From 470b09c178dadf15d4743430d398adcf3341e516 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 2 Jul 2024 08:28:07 -0700 Subject: [PATCH 01/56] Initial commit of refreshing Iceberg. --- .../impl/locations/TableLocationProvider.java | 6 +- .../impl/AbstractTableLocationProvider.java | 26 ++++- .../impl/CompositeTableDataService.java | 2 +- .../impl/FilteredTableDataService.java | 22 +++- .../impl/PollingTableLocationProvider.java | 22 +++- .../impl/TableLocationSubscriptionBuffer.java | 69 ++++++++--- .../impl/TestPartitionAwareSourceTable.java | 4 +- .../iceberg/layout/IcebergBaseLayout.java | 10 +- ...cebergRefreshingTableLocationProvider.java | 107 ++++++++++++++++++ .../location/IcebergTableLocationKey.java | 3 +- .../iceberg/util/IcebergCatalogAdapter.java | 83 ++++++++++---- .../iceberg/util/IcebergInstructions.java | 9 ++ .../deephaven/iceberg/util/IcebergTable.java | 74 ++++++++++++ 13 files changed, 380 insertions(+), 57 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index 84969d6249f..dd8b65826d4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -26,6 +26,10 @@ public interface TableLocationProvider extends NamedImplementation { */ interface Listener extends BasicTableDataListener { + void beginTransaction(); + + void endTransaction(); + /** * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location * subscription. This should occur at most once per location, but the order of delivery is not @@ -33,7 +37,7 @@ interface Listener extends BasicTableDataListener { * * @param tableLocationKey The new table location key */ - void handleTableLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey); + void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey); /** * Notify the listener of a {@link TableLocationKey} that has been removed. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index b474775955a..3bd12e5253a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -84,7 +84,9 @@ public final ImmutableTableKey getKey() { @Override protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) { - unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKey); + listener.beginTransaction(); + unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded); + listener.endTransaction(); } /** @@ -110,13 +112,26 @@ protected final void handleTableLocationKey(@NotNull final TableLocationKey loca visitLocationKey(locationKey); if (locationCreatedRecorder) { verifyPartitionKeys(locationKey); - if (subscriptions.deliverNotification(Listener::handleTableLocationKey, toKeyImmutable(result), true)) { + if (subscriptions.deliverNotification(Listener::handleTableLocationKeyAdded, toKeyImmutable(result), + true)) { onEmpty(); } } } } + protected final void beginTransaction() { + if (subscriptions != null) { + subscriptions.deliverNotification(Listener::beginTransaction, true); + } + } + + protected final void endTransaction() { + if (subscriptions != null) { + subscriptions.deliverNotification(Listener::endTransaction, true); + } + } + /** * Called after a table location has been visited by {@link #handleTableLocationKey(TableLocationKey)}, but * before notifications have been delivered to any subscriptions, if applicable. The default implementation does @@ -180,6 +195,13 @@ protected void doInitialization() { @Override @NotNull public final Collection getTableLocationKeys() { + // We need to prevent reading the map (and maybe mutating it?) during a transaction. + // We could transition between two maps, a stable copy and a shadow copy that is being mutated. + // Or we could hold a bigger lock while mutating the map, and hold the same lock here. Sounds like a job for a + // read-write lock (e.g. ReentrantReadWriteLock), maybe. If you want `FunctionalLock`, the pattern (but mostly + // not the code) from io.deephaven.engine.updategraph.UpdateGraphLock could help. + // I think we need the read-write lock for correctness, and I think we need to make it explicit. That is, the + // user needs to be able to get a read lock and hold it while it's operating on the returned collection. return unmodifiableTableLocationKeys; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index e9dbbc747ce..a9157da4b0f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -115,7 +115,7 @@ public void subscribe(@NotNull final Listener listener) { p.subscribe(listener); } else { p.refresh(); - p.getTableLocationKeys().forEach(listener::handleTableLocationKey); + p.getTableLocationKeys().forEach(listener::handleTableLocationKeyAdded); } }); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 8bfbeed184a..544852ecb21 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -157,12 +157,30 @@ private FilteringListener(@NotNull final TableLocationProvider.Listener outputLi } @Override - public void handleTableLocationKey(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void beginTransaction() { + // Delegate to the wrapped listener. + final TableLocationProvider.Listener outputListener = getWrapped(); + if (outputListener != null) { + outputListener.beginTransaction(); + } + } + + @Override + public void endTransaction() { + // Delegate to the wrapped listener. + final TableLocationProvider.Listener outputListener = getWrapped(); + if (outputListener != null) { + outputListener.endTransaction(); + } + } + + @Override + public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) { final TableLocationProvider.Listener outputListener = getWrapped(); // We can't try to clean up null listeners here, the underlying implementation may not allow concurrent // unsubscribe operations. if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { - outputListener.handleTableLocationKey(tableLocationKey); + outputListener.handleTableLocationKeyAdded(tableLocationKey); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java index 361a18c8c44..0f9302b7a77 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java @@ -3,14 +3,14 @@ // package io.deephaven.engine.table.impl.locations.impl; -import io.deephaven.engine.table.impl.locations.TableKey; -import io.deephaven.engine.table.impl.locations.TableLocation; -import io.deephaven.engine.table.impl.locations.TableLocationKey; -import io.deephaven.engine.table.impl.locations.TableLocationProvider; +import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.HashSet; +import java.util.Set; + /** * Polling-driven {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} * discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a @@ -46,9 +46,21 @@ public String getImplementationName() { return IMPLEMENTATION_NAME; } + // The simplest way to support "push" of new data availability is to provide a callback to the user that just calls + // `refresh`, which would need to become synchronized. Alternatively, we could make an Iceberg-specific aTLP + // implementation that exposes a more specific callback, e.g. with a snapshot ID, as well as the option to disable + // polling. We do need a mechanism to avoid going backwards, probably. @Override public void refresh() { - locationKeyFinder.findKeys(this::handleTableLocationKey); + beginTransaction(); + final Set missedKeys = new HashSet<>(getTableLocationKeys()); + locationKeyFinder.findKeys(tableLocationKey -> { + // noinspection SuspiciousMethodCalls + missedKeys.remove(tableLocationKey); + handleTableLocationKey(tableLocationKey); + }); + missedKeys.forEach(this::handleTableLocationKeyRemoved); + endTransaction(); setInitialized(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index fae32e8a0a1..05a32b5fb6d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -26,9 +26,14 @@ public class TableLocationSubscriptionBuffer implements TableLocationProvider.Li private boolean subscribed = false; private final Object updateLock = new Object(); - private Set pendingLocationKeys = EMPTY_TABLE_LOCATION_KEYS; + // These sets represent a completed transaction of adds and removes. + private Set pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + + private Set accumulatedLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; + private Set accumulatedLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + private TableDataException pendingException = null; public TableLocationSubscriptionBuffer(@NotNull final TableLocationProvider tableLocationProvider) { @@ -71,7 +76,7 @@ public synchronized LocationUpdate processPending() { // NB: Providers that don't support subscriptions don't tick - this single call to run is // sufficient. tableLocationProvider.refresh(); - tableLocationProvider.getTableLocationKeys().forEach(this::handleTableLocationKey); + tableLocationProvider.getTableLocationKeys().forEach(this::handleTableLocationKeyAdded); } subscribed = true; } @@ -79,8 +84,8 @@ public synchronized LocationUpdate processPending() { final Collection resultLocationsRemoved; final TableDataException resultException; synchronized (updateLock) { - resultLocationKeys = pendingLocationKeys; - pendingLocationKeys = EMPTY_TABLE_LOCATION_KEYS; + resultLocationKeys = pendingLocationsAdded; + pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; resultLocationsRemoved = pendingLocationsRemoved; pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; resultException = pendingException; @@ -105,7 +110,7 @@ public synchronized void reset() { subscribed = false; } synchronized (updateLock) { - pendingLocationKeys = EMPTY_TABLE_LOCATION_KEYS; + pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; pendingException = null; } @@ -116,27 +121,65 @@ public synchronized void reset() { // ------------------------------------------------------------------------------------------------------------------ @Override - public void handleTableLocationKey(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void beginTransaction() { + synchronized (updateLock) { + // Assert that we can start a new transaction. + Require.eq(EMPTY_TABLE_LOCATION_KEYS, "accumulatedLocationsAdded", accumulatedLocationsAdded); + Require.eq(EMPTY_TABLE_LOCATION_KEYS, "accumulatedLocationsRemoved", accumulatedLocationsRemoved); + } + } + + @Override + public void endTransaction() { + synchronized (updateLock) { + // If we already have a completed transaction, merge the accumulated sets into it. + if (pendingLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsAdded.addAll(accumulatedLocationsAdded); + } else { + pendingLocationsAdded = accumulatedLocationsAdded; + } + + if (pendingLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { + // Remove any locations that were added and then removed by this transaction. + for (final ImmutableTableLocationKey tableLocationKey : accumulatedLocationsRemoved) { + if (pendingLocationsAdded.remove(tableLocationKey)) { + continue; + } + pendingLocationsRemoved.add(tableLocationKey); + } + } else { + pendingLocationsRemoved = accumulatedLocationsRemoved; + } + + // Reset the accumulated sets to empty. + accumulatedLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; + accumulatedLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + } + } + + @Override + public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) { synchronized (updateLock) { - if (pendingLocationKeys == EMPTY_TABLE_LOCATION_KEYS) { - pendingLocationKeys = new HashSet<>(); + if (accumulatedLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { + accumulatedLocationsAdded = new HashSet<>(); } - pendingLocationKeys.add(tableLocationKey); + accumulatedLocationsAdded.add(tableLocationKey); } } @Override public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) { synchronized (updateLock) { + // TODO: is it possible that we add and remove a location within a single transaction? Paranoid? // If we remove something that was pending to be added, just discard both. - if (pendingLocationKeys.remove(tableLocationKey)) { + if (accumulatedLocationsRemoved.remove(tableLocationKey)) { return; } - if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { - pendingLocationsRemoved = new HashSet<>(); + if (accumulatedLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + accumulatedLocationsRemoved = new HashSet<>(); } - pendingLocationsRemoved.add(tableLocationKey); + accumulatedLocationsRemoved.add(tableLocationKey); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java index f6b838c0646..774f25f7d16 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java @@ -250,7 +250,7 @@ private void doInitializeCheck(final ImmutableTableLocationKey[] tableLocationKe @Override public Object invoke(Invocation invocation) { subscriptionBuffer = (TableLocationSubscriptionBuffer) invocation.getParameter(0); - Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKey); + Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded); return null; } }); @@ -407,7 +407,7 @@ public Object invoke(Invocation invocation) { private void doAddLocationsRefreshCheck(final ImmutableTableLocationKey[] tableLocationKeys, final Set expectPassFilters) { - Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKey); + Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded); expectPassFilters.forEach(tl -> checking(new Expectations() { { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index f5334cf866c..70a65230f29 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -33,11 +33,6 @@ public abstract class IcebergBaseLayout implements TableLocationKeyFinder cache; + /** + * The {@link Snapshot} from which to discover data files. + */ + Snapshot snapshot; + /** * The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. Only * accessed while synchronized on {@code this}. diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java new file mode 100644 index 00000000000..5b510904c3d --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java @@ -0,0 +1,107 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.layout; + +import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; +import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import org.apache.iceberg.Snapshot; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.HashSet; +import java.util.Set; + +/** + * Polling-driven {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} + * discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a + * {@link TableLocationFactory}. + */ +public class IcebergRefreshingTableLocationProvider + extends AbstractTableLocationProvider { + + private static final String IMPLEMENTATION_NAME = IcebergRefreshingTableLocationProvider.class.getSimpleName(); + + private final IcebergBaseLayout locationKeyFinder; + private final TableLocationFactory locationFactory; + private final TableDataRefreshService refreshService; + + private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken; + + public IcebergRefreshingTableLocationProvider(@NotNull final TK tableKey, + @NotNull final IcebergBaseLayout locationKeyFinder, + @NotNull final TableLocationFactory locationFactory, + @Nullable final TableDataRefreshService refreshService) { + super(tableKey, refreshService != null); + this.locationKeyFinder = locationKeyFinder; + this.locationFactory = locationFactory; + this.refreshService = refreshService; + } + + // ------------------------------------------------------------------------------------------------------------------ + // AbstractTableLocationProvider implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + // The simplest way to support "push" of new data availability is to provide a callback to the user that just calls + // `refresh`, which would need to become synchronized. Alternatively, we could make an Iceberg-specific aTLP + // implementation that exposes a more specific callback, e.g. with a snapshot ID, as well as the option to disable + // polling. We do need a mechanism to avoid going backwards, probably. + @Override + public void refresh() { + beginTransaction(); + final Set missedKeys = new HashSet<>(getTableLocationKeys()); + locationKeyFinder.findKeys(tableLocationKey -> { + // noinspection SuspiciousMethodCalls + missedKeys.remove(tableLocationKey); + handleTableLocationKey(tableLocationKey); + }); + missedKeys.forEach(this::handleTableLocationKeyRemoved); + endTransaction(); + setInitialized(); + } + + + public void update(final Snapshot snapshot) { + // Update the snapshot to the new one + locationKeyFinder.snapshot = snapshot; + // Call the refresh + refresh(); + } + + @Override + @NotNull + protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) { + // noinspection unchecked + return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService); + } + + // ------------------------------------------------------------------------------------------------------------------ + // SubscriptionAggregator implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + protected final void activateUnderlyingDataSource() { + subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this); + } + + @Override + protected final void deactivateUnderlyingDataSource() { + if (subscriptionToken != null) { + subscriptionToken.cancel(); + subscriptionToken = null; + } + } + + @Override + protected final boolean matchSubscriptionToken(final T token) { + return token == subscriptionToken; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java index d6d5d4fb514..0fb37d9132d 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java @@ -3,12 +3,13 @@ // package io.deephaven.iceberg.location; +import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableLocationKey; /** * {@link TableLocationKey} implementation for use with data stored in Iceberg tables. */ -public interface IcebergTableLocationKey extends TableLocationKey { +public interface IcebergTableLocationKey extends ImmutableTableLocationKey { /** * Get the read instructions for the location. * diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index c379c715c6d..f4daec870d2 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -3,21 +3,25 @@ // package io.deephaven.iceberg.util; +import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.PartitionAwareSourceTable; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.TableKey; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; +import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder; import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; -import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.iceberg.layout.IcebergBaseLayout; import io.deephaven.iceberg.layout.IcebergFlatLayout; import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout; +import io.deephaven.iceberg.layout.IcebergRefreshingTableLocationProvider; import io.deephaven.iceberg.location.IcebergTableLocationFactory; import io.deephaven.iceberg.location.IcebergTableLocationKey; import org.apache.iceberg.PartitionField; @@ -55,7 +59,7 @@ public class IcebergCatalogAdapter { /** * Create a single {@link TableDefinition} from a given Schema, PartitionSpec, and TableDefinition. Takes into - * account {@link Map column rename instructions} + * account {@link Map<> column rename instructions} * * @param schema The schema of the table. * @param partitionSpec The partition specification of the table. @@ -324,6 +328,13 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap); } + Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long snapshotId) { + return listSnapshots(tableIdentifier).stream() + .filter(snapshot -> snapshot.snapshotId() == snapshotId) + .findFirst() + .orElse(null); + } + /** * Read the latest static snapshot of an Iceberg table from the Iceberg catalog. * @@ -353,10 +364,10 @@ public Table readTable( @NotNull final IcebergInstructions instructions) { // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = listSnapshots(tableIdentifier).stream() - .filter(snapshot -> snapshot.snapshotId() == tableSnapshotId) - .findFirst() - .orElseThrow(() -> new IllegalArgumentException("Snapshot with id " + tableSnapshotId + " not found")); + final Snapshot tableSnapshot = getSnapshot(tableIdentifier, tableSnapshotId); + if (tableSnapshot == null) { + throw new IllegalArgumentException("Snapshot with id " + tableSnapshotId + " not found"); + } return readTableInternal(tableIdentifier, tableSnapshot, instructions); } @@ -381,7 +392,6 @@ private Table readTableInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, @NotNull final IcebergInstructions instructions) { - // Load the table from the catalog final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); @@ -427,7 +437,7 @@ private Table readTableInternal( } final String description; - final TableLocationKeyFinder keyFinder; + final IcebergBaseLayout keyFinder; final TableDataRefreshService refreshService; final UpdateSourceRegistrar updateSourceRegistrar; @@ -440,23 +450,46 @@ private Table readTableInternal( instructions); } - refreshService = null; - updateSourceRegistrar = null; - description = "Read static iceberg table with " + keyFinder; - - final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(), - refreshService); - - final PartitionAwareSourceTable result = new PartitionAwareSourceTable( - tableDef, - description, - RegionedTableComponentFactoryImpl.INSTANCE, - locationProvider, - updateSourceRegistrar); + if (instructions.isRefreshing()) { + description = "Read refreshing iceberg table with " + keyFinder; + updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + + final IcebergRefreshingTableLocationProvider locationProvider = + new IcebergRefreshingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + TableDataRefreshService.getSharedRefreshService()); + + return new IcebergTable( + tableIdentifier, + this, + tableDef, + description, + RegionedTableComponentFactoryImpl.INSTANCE, + locationProvider, + updateSourceRegistrar); + } else { + description = "Read static iceberg table with " + keyFinder; + updateSourceRegistrar = null; + + final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + null); + + return new PartitionAwareSourceTable( + tableDef, + description, + RegionedTableComponentFactoryImpl.INSTANCE, + locationProvider, + updateSourceRegistrar); + } + } - return result; + private static KnownLocationKeyFinder toKnownKeys( + final IcebergBaseLayout keyFinder) { + return KnownLocationKeyFinder.copyFrom(keyFinder, Comparator.naturalOrder()); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 4788e0e8714..57963a9e2d6 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -5,6 +5,7 @@ import io.deephaven.annotations.BuildableStyle; import io.deephaven.engine.table.TableDefinition; +import org.immutables.value.Value; import org.immutables.value.Value.Immutable; import java.util.Map; @@ -38,6 +39,11 @@ public static Builder builder() { */ public abstract Map columnRenames(); + @Value.Default + public Boolean isRefreshing() { + return false; + } + public interface Builder { @SuppressWarnings("unused") Builder tableDefinition(TableDefinition tableDefinition); @@ -51,6 +57,9 @@ public interface Builder { @SuppressWarnings("unused") Builder putAllColumnRenames(Map entries); + @SuppressWarnings("unused") + Builder isRefreshing(Boolean isRefreshing); + IcebergInstructions build(); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java new file mode 100644 index 00000000000..cbc27b50461 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java @@ -0,0 +1,74 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.PartitionAwareSourceTable; +import io.deephaven.engine.table.impl.SourceTableComponentFactory; +import io.deephaven.engine.table.impl.locations.TableKey; +import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.iceberg.layout.IcebergRefreshingTableLocationProvider; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.TableIdentifier; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.List; + +/** + * Table wrapper for refreshing Iceberg tables. + */ +public class IcebergTable extends PartitionAwareSourceTable { + private final TableIdentifier tableIdentifier; + private final IcebergCatalogAdapter adapter; + /** + * Location discovery. + */ + final IcebergRefreshingTableLocationProvider locationProvider; + + /** + * + * + * @param tableDefinition A TableDefinition + * @param description A human-readable description for this table + * @param componentFactory A component factory for creating column source managers + * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table + * @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live + */ + IcebergTable( + @NotNull TableIdentifier tableIdentifier, + @NotNull IcebergCatalogAdapter adapter, + @NotNull TableDefinition tableDefinition, + @NotNull String description, + @NotNull SourceTableComponentFactory componentFactory, + @NotNull IcebergRefreshingTableLocationProvider locationProvider, + @Nullable UpdateSourceRegistrar updateSourceRegistrar) { + super(tableDefinition, description, componentFactory, locationProvider, updateSourceRegistrar); + this.tableIdentifier = tableIdentifier; + this.adapter = adapter; + this.locationProvider = locationProvider; + } + + public void update() { + // Find the latest snapshot. + final List snapshots = adapter.listSnapshots(tableIdentifier); + update(snapshots.get(snapshots.size() - 1)); + } + + public void update(final long snapshotId) { + // Find the snapshot with the given snapshot id + final Snapshot tableSnapshot = adapter.getSnapshot(tableIdentifier, snapshotId); + if (tableSnapshot == null) { + throw new IllegalArgumentException("Snapshot with id " + snapshotId + " not found"); + } + + update(tableSnapshot); + } + + public void update(final @NotNull Snapshot snapshot) { + // Call the update function, this + locationProvider.update(snapshot); + } +} From 264fdb1eb5ff8ce4e1b4b667a1310efed7a1fffa Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 2 Jul 2024 09:02:59 -0700 Subject: [PATCH 02/56] Change IcebergInstructions refreshing indicator to enum instead of boolean --- .../iceberg/util/IcebergCatalogAdapter.java | 35 ++++++++++--------- .../iceberg/util/IcebergInstructions.java | 12 +++++-- 2 files changed, 28 insertions(+), 19 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index 97c467b04a4..69b5332caba 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -508,7 +508,23 @@ private Table readTableInternal( userInstructions); } - if (instructions.isRefreshing()) { + if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.STATIC) { + description = "Read static iceberg table with " + keyFinder; + updateSourceRegistrar = null; + + final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + null); + + return new PartitionAwareSourceTable( + tableDef, + description, + RegionedTableComponentFactoryImpl.INSTANCE, + locationProvider, + updateSourceRegistrar); + } else if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) { description = "Read refreshing iceberg table with " + keyFinder; updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); @@ -528,22 +544,9 @@ private Table readTableInternal( locationProvider, updateSourceRegistrar); } else { - description = "Read static iceberg table with " + keyFinder; - updateSourceRegistrar = null; - - final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(), - null); - - return new PartitionAwareSourceTable( - tableDef, - description, - RegionedTableComponentFactoryImpl.INSTANCE, - locationProvider, - updateSourceRegistrar); + throw new UnsupportedOperationException("Unsupported refreshing mode: " + instructions.refreshing()); } + } private static KnownLocationKeyFinder toKnownKeys( diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index ebc0dff128e..e1063d06d52 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -25,6 +25,12 @@ public abstract class IcebergInstructions { @SuppressWarnings("unused") public static final IcebergInstructions DEFAULT = builder().build(); + public enum IcebergRefreshing { + STATIC, + AUTO_REFRESHING, + MANUAL_REFRESHING + } + public static Builder builder() { return ImmutableIcebergInstructions.builder(); } @@ -47,8 +53,8 @@ public static Builder builder() { public abstract Map columnRenames(); @Value.Default - public Boolean isRefreshing() { - return false; + public IcebergRefreshing refreshing() { + return IcebergRefreshing.STATIC; } public interface Builder { @@ -65,7 +71,7 @@ public interface Builder { Builder putAllColumnRenames(Map entries); @SuppressWarnings("unused") - Builder isRefreshing(Boolean isRefreshing); + Builder refreshing(IcebergRefreshing refreshing); IcebergInstructions build(); } From 58d0a730bdea0487cc10041e25a4ecfb5410c978 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 3 Jul 2024 07:59:33 -0700 Subject: [PATCH 03/56] WIP, for review --- .../table/impl/ColumnSourceManager.java | 7 ++--- .../engine/table/impl/SourceTable.java | 16 +++++------- .../regioned/RegionedColumnSourceManager.java | 26 +++++++++++++++---- .../iceberg/util/IcebergInstructions.java | 4 +-- 4 files changed, 30 insertions(+), 23 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java index 1d5ba841b7a..bc56a42d246 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java @@ -7,10 +7,7 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.TrackingWritableRowSet; import io.deephaven.engine.rowset.WritableRowSet; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.DataIndex; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableListener; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableLocation; import org.jetbrains.annotations.NotNull; @@ -53,7 +50,7 @@ public interface ColumnSourceManager extends LivenessReferent { * * @return The set of added row keys, to be owned by the caller */ - WritableRowSet refresh(); + TableUpdate refresh(); /** * Advise this ColumnSourceManager that an error has occurred, and that it will no longer be {@link #refresh() diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index e3508fc45c7..5398bb80a86 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -9,6 +9,7 @@ import io.deephaven.engine.rowset.TrackingWritableRowSet; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.TableUpdateListener; import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableDataException; @@ -106,7 +107,6 @@ public abstract class SourceTable> exte } setRefreshing(isRefreshing); - setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); } /** @@ -219,10 +219,6 @@ protected void instrumentedRefresh() { final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending(); final ImmutableTableLocationKey[] removedKeys = maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); - if (removedKeys.length > 0) { - throw new TableLocationRemovedException("Source table does not support removed locations", - removedKeys); - } maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); // This class previously had functionality to notify "location listeners", but it was never used. @@ -232,13 +228,13 @@ protected void instrumentedRefresh() { return; } - final RowSet added = columnSourceManager.refresh(); - if (added.isEmpty()) { + final TableUpdate update = columnSourceManager.refresh(); + if (update.empty()) { return; } - - rowSet.insert(added); - notifyListeners(added, RowSetFactory.empty(), RowSetFactory.empty()); + rowSet.insert(update.added()); + rowSet.remove(update.removed()); + notifyListeners(update); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index d510f21a002..0ae2eaf1ba0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -71,6 +71,8 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col private final KeyedObjectHashMap includedTableLocations = new KeyedObjectHashMap<>(INCLUDED_TABLE_LOCATION_ENTRY_KEY); + private final List removedTableLocations = new ArrayList<>(); + /** * Table locations that provide the regions backing our column sources, in insertion order. */ @@ -206,7 +208,8 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati log.debug().append("EMPTY_LOCATION_REMOVED:").append(locationKey.toString()).endl(); } } else if (includedLocation != null) { - includedLocation.invalidate(); + removedTableLocations.add(includedLocation); + // includedLocation.invalidate(); return true; } @@ -219,7 +222,8 @@ public synchronized TrackingWritableRowSet initialize() { // Do our first pass over the locations to include as many as possible and build the initial row set // noinspection resource - final TrackingWritableRowSet initialRowSet = update(true).toTracking(); + final TableUpdate update = update(true); + final TrackingWritableRowSet initialRowSet = update.added().writableCast().toTracking(); // Add single-column data indexes for all partitioning columns, whether refreshing or not columnDefinitions.stream().filter(ColumnDefinition::isPartitioning).forEach(cd -> { @@ -261,7 +265,7 @@ public synchronized TrackingWritableRowSet initialize() { } @Override - public synchronized WritableRowSet refresh() { + public synchronized TableUpdate refresh() { if (!isRefreshing) { throw new UnsupportedOperationException("Cannot refresh a static table"); } @@ -280,8 +284,19 @@ public void deliverError(@NotNull final Throwable error, @Nullable final TableLi } } - private WritableRowSet update(final boolean initializing) { + private TableUpdate update(final boolean initializing) { final RowSetBuilderSequential addedRowSetBuilder = RowSetFactory.builderSequential(); + final RowSetBuilderSequential removedRowSetBuilder = RowSetFactory.builderSequential(); + + // Sort the removed locations by region index, so that we can process them in order. + removedTableLocations.sort(Comparator.comparingInt(e -> e.regionIndex)); + for (final IncludedTableLocationEntry removedLocation : removedTableLocations) { + final long regionFirstKey = RegionedColumnSource.getFirstRowKey(removedLocation.regionIndex); + removedLocation.location.getRowSet() + .forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> removedRowSetBuilder + .appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey)); + } + removedTableLocations.clear(); final RowSetBuilderSequential modifiedRegionBuilder = initializing ? null : RowSetFactory.builderSequential(); @@ -373,7 +388,8 @@ private WritableRowSet update(final boolean initializing) { includedLocationsTable.notifyListeners(update); } } - return addedRowSetBuilder.build(); + return new TableUpdateImpl(addedRowSetBuilder.build(), removedRowSetBuilder.build(), RowSetFactory.empty(), + RowSetShiftData.EMPTY, ModifiedColumnSet.EMPTY); } @Override diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index e1063d06d52..d6596b254de 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -26,9 +26,7 @@ public abstract class IcebergInstructions { public static final IcebergInstructions DEFAULT = builder().build(); public enum IcebergRefreshing { - STATIC, - AUTO_REFRESHING, - MANUAL_REFRESHING + STATIC, AUTO_REFRESHING, MANUAL_REFRESHING } public static Builder builder() { From e0904742be80360735e5e3d1e66f9c4bbb34c07f Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 22 Jul 2024 19:47:36 -0700 Subject: [PATCH 04/56] Manual and auto-refreshing working, better documentation. --- .../regioned/RegionedColumnSourceManager.java | 36 +++++-- ...cebergRefreshingTableLocationProvider.java | 99 ++++++++++++++++--- .../iceberg/util/IcebergCatalogAdapter.java | 79 ++++++++++----- .../iceberg/util/IcebergInstructions.java | 17 ++++ .../deephaven/iceberg/util/IcebergTable.java | 24 +---- 5 files changed, 190 insertions(+), 65 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 0ae2eaf1ba0..6d70dbf912f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -24,6 +24,7 @@ import io.deephaven.io.logger.Logger; import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.ReferentialIntegrity; +import io.deephaven.util.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -71,8 +72,17 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col private final KeyedObjectHashMap includedTableLocations = new KeyedObjectHashMap<>(INCLUDED_TABLE_LOCATION_ENTRY_KEY); + /** + * List of locations that were removed this cycle. Will be cleared after each update. + */ private final List removedTableLocations = new ArrayList<>(); + /** + * The next region index to assign to a location. We increment for each new location and will not reuse indices from + * regions that were removed. + */ + private final MutableInt nextRegionIndex = new MutableInt(0); + /** * Table locations that provide the regions backing our column sources, in insertion order. */ @@ -208,6 +218,7 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati log.debug().append("EMPTY_LOCATION_REMOVED:").append(locationKey.toString()).endl(); } } else if (includedLocation != null) { + orderedIncludedTableLocations.remove(includedLocation); removedTableLocations.add(includedLocation); // includedLocation.invalidate(); return true; @@ -288,6 +299,8 @@ private TableUpdate update(final boolean initializing) { final RowSetBuilderSequential addedRowSetBuilder = RowSetFactory.builderSequential(); final RowSetBuilderSequential removedRowSetBuilder = RowSetFactory.builderSequential(); + final RowSetBuilderSequential removedRegionBuilder = RowSetFactory.builderSequential(); + // Sort the removed locations by region index, so that we can process them in order. removedTableLocations.sort(Comparator.comparingInt(e -> e.regionIndex)); for (final IncludedTableLocationEntry removedLocation : removedTableLocations) { @@ -295,6 +308,7 @@ private TableUpdate update(final boolean initializing) { removedLocation.location.getRowSet() .forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> removedRowSetBuilder .appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey)); + removedRegionBuilder.appendKey(removedLocation.regionIndex); } removedTableLocations.clear(); @@ -333,6 +347,8 @@ private TableUpdate update(final boolean initializing) { } } + final RowSetBuilderSequential addedRegionBuilder = RowSetFactory.builderSequential(); + final int previousNumRegions = includedTableLocations.size(); final int newNumRegions = previousNumRegions + (entriesToInclude == null ? 0 : entriesToInclude.size()); if (entriesToInclude != null) { @@ -356,11 +372,13 @@ private TableUpdate update(final boolean initializing) { // @formatter:on locationSource.set(entry.regionIndex, entry.location); rowSetSource.set(entry.regionIndex, entry.location.getRowSet()); + addedRegionBuilder.appendKey(entry.regionIndex); } } + final RowSet addedRegions = addedRegionBuilder.build(); - if (previousNumRegions != newNumRegions) { - includedLocationsTable.getRowSet().writableCast().insertRange(previousNumRegions, newNumRegions - 1); + if (addedRegions.isNonempty()) { + includedLocationsTable.getRowSet().writableCast().insert(addedRegions); } if (initializing) { @@ -374,14 +392,16 @@ private TableUpdate update(final boolean initializing) { } } else { final RowSet modifiedRegions = modifiedRegionBuilder.build(); - if (previousNumRegions == newNumRegions && modifiedRegions.isEmpty()) { + final RowSet removedRegions = removedRegionBuilder.build(); + if (addedRegions.isEmpty() && modifiedRegions.isEmpty() && removedRegions.isEmpty()) { + addedRegions.close(); modifiedRegions.close(); + removedRegions.close(); } else { + includedLocationsTable.getRowSet().writableCast().remove(removedRegions); final TableUpdate update = new TableUpdateImpl( - previousNumRegions == newNumRegions - ? RowSetFactory.empty() - : RowSetFactory.fromRange(previousNumRegions, newNumRegions - 1), - RowSetFactory.empty(), + addedRegions, + removedRegions, modifiedRegions, RowSetShiftData.EMPTY, modifiedRegions.isNonempty() ? rowSetModifiedColumnSet : ModifiedColumnSet.EMPTY); @@ -488,7 +508,7 @@ private class IncludedTableLocationEntry implements Comparable> columnLocationStates = new ArrayList<>(); /** diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java index 5b510904c3d..43aa3d78939 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java @@ -8,17 +8,25 @@ import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.iceberg.util.IcebergCatalogAdapter; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.HashSet; +import java.util.List; import java.util.Set; /** - * Polling-driven {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} + *

+ * Refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} * discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a * {@link TableLocationFactory}. + *

+ *

+ * Supports both automatic and manual refreshing cases, distinguished by the {@code autoRefresh} parameter. + *

*/ public class IcebergRefreshingTableLocationProvider extends AbstractTableLocationProvider { @@ -28,17 +36,27 @@ public class IcebergRefreshingTableLocationProvider locationFactory; private final TableDataRefreshService refreshService; + private final IcebergCatalogAdapter adapter; + private final TableIdentifier tableIdentifier; + private final boolean autoRefresh; private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken; - public IcebergRefreshingTableLocationProvider(@NotNull final TK tableKey, + public IcebergRefreshingTableLocationProvider( + @NotNull final TK tableKey, @NotNull final IcebergBaseLayout locationKeyFinder, @NotNull final TableLocationFactory locationFactory, - @Nullable final TableDataRefreshService refreshService) { + @Nullable final TableDataRefreshService refreshService, + @NotNull final IcebergCatalogAdapter adapter, + @NotNull final TableIdentifier tableIdentifier, + final boolean autoRefresh) { super(tableKey, refreshService != null); this.locationKeyFinder = locationKeyFinder; this.locationFactory = locationFactory; this.refreshService = refreshService; + this.adapter = adapter; + this.tableIdentifier = tableIdentifier; + this.autoRefresh = autoRefresh; } // ------------------------------------------------------------------------------------------------------------------ @@ -50,12 +68,69 @@ public String getImplementationName() { return IMPLEMENTATION_NAME; } - // The simplest way to support "push" of new data availability is to provide a callback to the user that just calls - // `refresh`, which would need to become synchronized. Alternatively, we could make an Iceberg-specific aTLP - // implementation that exposes a more specific callback, e.g. with a snapshot ID, as well as the option to disable - // polling. We do need a mechanism to avoid going backwards, probably. @Override - public void refresh() { + public synchronized void refresh() { + if (autoRefresh) { + final Snapshot latestSnapshot = adapter.getCurrentSnapshot(tableIdentifier); + if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) { + locationKeyFinder.snapshot = latestSnapshot; + } + } + refreshSnapshot(); + } + + /** + * Update the table location provider with the latest snapshot from the catalog. + */ + public synchronized void update() { + update(adapter.getCurrentSnapshot(tableIdentifier)); + } + + /** + * Update the table location provider with a specific snapshot from the catalog. If the {@code snapshotId} is not + * found in the list of snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot + * must also be newer (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} + * is thrown. + */ + public synchronized void update(final long snapshotId) { + final List snapshots = adapter.listSnapshots(tableIdentifier); + + final Snapshot snapshot = snapshots.stream() + .filter(s -> s.snapshotId() == snapshotId).findFirst() + .orElse(null); + + if (snapshot == null) { + throw new IllegalArgumentException( + "Snapshot " + snapshotId + " was not found in the list of snapshots for table " + tableIdentifier + + ". Snapshots: " + snapshots); + } + update(snapshot); + } + + /** + * Update the table location provider with a specific snapshot from the catalog. The input snapshot must be newer + * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown. + * + * @param snapshot + */ + public synchronized void update(final Snapshot snapshot) { + // Verify that the input snapshot is newer (higher in sequence number) than the current snapshot. + if (snapshot.sequenceNumber() <= locationKeyFinder.snapshot.sequenceNumber()) { + throw new IllegalArgumentException( + "Snapshot sequence number " + snapshot.sequenceNumber() + + " is older than the current snapshot sequence number " + + locationKeyFinder.snapshot.sequenceNumber() + " for table " + tableIdentifier); + } + // Update the snapshot. + locationKeyFinder.snapshot = snapshot; + refreshSnapshot(); + } + + /** + * Refresh the table location provider with the latest snapshot from the catalog. This method will identify new + * locations and removed locations. + */ + private void refreshSnapshot() { beginTransaction(); final Set missedKeys = new HashSet<>(getTableLocationKeys()); locationKeyFinder.findKeys(tableLocationKey -> { @@ -68,14 +143,6 @@ public void refresh() { setInitialized(); } - - public void update(final Snapshot snapshot) { - // Update the snapshot to the new one - locationKeyFinder.snapshot = snapshot; - // Call the refresh - refresh(); - } - @Override @NotNull protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index 69b5332caba..76d099e9499 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder; import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; +import io.deephaven.engine.table.impl.locations.util.ExecutorTableDataRefreshService; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; @@ -360,6 +361,17 @@ Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long .orElse(null); } + /** + * Get the current {@link Snapshot snapshot} of a given Iceberg table. + * + * @param tableIdentifier The identifier of the table. + * @return The current snapshot of the table. + */ + public Snapshot getCurrentSnapshot(@NotNull final TableIdentifier tableIdentifier) { + final List snapshots = listSnapshots(tableIdentifier); + return snapshots.get(snapshots.size() - 1); + } + /** * Read the latest static snapshot of an Iceberg table from the Iceberg catalog. * @@ -496,8 +508,6 @@ private Table readTableInternal( final String description; final IcebergBaseLayout keyFinder; - final TableDataRefreshService refreshService; - final UpdateSourceRegistrar updateSourceRegistrar; if (partitionSpec.isUnpartitioned()) { // Create the flat layout location key finder @@ -510,11 +520,10 @@ private Table readTableInternal( if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.STATIC) { description = "Read static iceberg table with " + keyFinder; - updateSourceRegistrar = null; final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( StandaloneTableKey.getInstance(), - keyFinder, + toKnownKeys(keyFinder), new IcebergTableLocationFactory(), null); @@ -523,30 +532,47 @@ private Table readTableInternal( description, RegionedTableComponentFactoryImpl.INSTANCE, locationProvider, - updateSourceRegistrar); - } else if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) { - description = "Read refreshing iceberg table with " + keyFinder; - updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + null); + } - final IcebergRefreshingTableLocationProvider locationProvider = - new IcebergRefreshingTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(), - TableDataRefreshService.getSharedRefreshService()); + final UpdateSourceRegistrar updateSourceRegistrar; + final IcebergRefreshingTableLocationProvider locationProvider; - return new IcebergTable( - tableIdentifier, + if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) { + description = "Read manual refreshing iceberg table with " + keyFinder; + updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + + locationProvider = new IcebergRefreshingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + TableDataRefreshService.getSharedRefreshService(), this, - tableDef, - description, - RegionedTableComponentFactoryImpl.INSTANCE, - locationProvider, - updateSourceRegistrar); + tableIdentifier, + false); } else { - throw new UnsupportedOperationException("Unsupported refreshing mode: " + instructions.refreshing()); + description = "Read automatic refreshing iceberg table with " + keyFinder; + updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + + final TableDataRefreshService refreshService = + new ExecutorTableDataRefreshService("Local", instructions.autoRefreshMs(), 30_000L, 10); + + locationProvider = new IcebergRefreshingTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + refreshService, + this, + tableIdentifier, + true); } + return new IcebergTable( + tableDef, + description, + RegionedTableComponentFactoryImpl.INSTANCE, + locationProvider, + updateSourceRegistrar); } private static KnownLocationKeyFinder toKnownKeys( @@ -561,4 +587,13 @@ private static KnownLocationKeyFinder toKnownKeys( public Catalog catalog() { return catalog; } + + /** + * Returns the underlying Iceberg {@link FileIO fileIO} used by this adapter. + */ + @SuppressWarnings("unused") + public FileIO fileIO() { + return fileIO; + } + } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index d6596b254de..dc7515d7498 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -25,6 +25,7 @@ public abstract class IcebergInstructions { @SuppressWarnings("unused") public static final IcebergInstructions DEFAULT = builder().build(); + @SuppressWarnings("unused") public enum IcebergRefreshing { STATIC, AUTO_REFRESHING, MANUAL_REFRESHING } @@ -50,11 +51,24 @@ public static Builder builder() { */ public abstract Map columnRenames(); + /** + * The {@link IcebergRefreshing} mode to use when reading the Iceberg data files. Default is + * {@link IcebergRefreshing#STATIC}. + */ @Value.Default public IcebergRefreshing refreshing() { return IcebergRefreshing.STATIC; } + /** + * When {@link #refreshing()} is set to {@code IcebergRefreshing.AUTO_REFRESHING}, specifies the number of + * milliseconds to wait before refreshing the Iceberg data files. Default is 60_000 milliseconds. + */ + @Value.Default + public long autoRefreshMs() { + return 60_000L; // 60 second default + } + public interface Builder { @SuppressWarnings("unused") Builder tableDefinition(TableDefinition tableDefinition); @@ -71,6 +85,9 @@ public interface Builder { @SuppressWarnings("unused") Builder refreshing(IcebergRefreshing refreshing); + @SuppressWarnings("unused") + Builder autoRefreshMs(long autoRefreshMs); + IcebergInstructions build(); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java index cbc27b50461..9c422b96968 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java @@ -15,14 +15,10 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.List; - /** * Table wrapper for refreshing Iceberg tables. */ public class IcebergTable extends PartitionAwareSourceTable { - private final TableIdentifier tableIdentifier; - private final IcebergCatalogAdapter adapter; /** * Location discovery. */ @@ -38,37 +34,27 @@ public class IcebergTable extends PartitionAwareSourceTable { * @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live */ IcebergTable( - @NotNull TableIdentifier tableIdentifier, - @NotNull IcebergCatalogAdapter adapter, @NotNull TableDefinition tableDefinition, @NotNull String description, @NotNull SourceTableComponentFactory componentFactory, @NotNull IcebergRefreshingTableLocationProvider locationProvider, @Nullable UpdateSourceRegistrar updateSourceRegistrar) { super(tableDefinition, description, componentFactory, locationProvider, updateSourceRegistrar); - this.tableIdentifier = tableIdentifier; - this.adapter = adapter; this.locationProvider = locationProvider; } + @SuppressWarnings("unused") public void update() { - // Find the latest snapshot. - final List snapshots = adapter.listSnapshots(tableIdentifier); - update(snapshots.get(snapshots.size() - 1)); + locationProvider.update(); } + @SuppressWarnings("unused") public void update(final long snapshotId) { - // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = adapter.getSnapshot(tableIdentifier, snapshotId); - if (tableSnapshot == null) { - throw new IllegalArgumentException("Snapshot with id " + snapshotId + " not found"); - } - - update(tableSnapshot); + locationProvider.update(snapshotId); } + @SuppressWarnings("unused") public void update(final @NotNull Snapshot snapshot) { - // Call the update function, this locationProvider.update(snapshot); } } From 57021ad244f4e9b1a84b239289c9953ef93f0eb6 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 23 Jul 2024 16:57:03 -0700 Subject: [PATCH 05/56] Addressed more PR comments, some remaining. --- .../engine/table/impl/SourceTable.java | 9 +- .../impl/locations/TableLocationProvider.java | 42 ++++--- .../impl/AbstractTableLocationProvider.java | 23 ++-- .../impl/CompositeTableDataService.java | 2 +- .../impl/FilteredTableDataService.java | 20 ++-- .../impl/TableLocationSubscriptionBuffer.java | 109 +++++++++++------- .../regioned/RegionedColumnSourceManager.java | 19 ++- ...cebergRefreshingTableLocationProvider.java | 63 +--------- .../IcebergStaticTableLocationProvider.java | 65 +++++++++++ .../IcebergTableLocationProviderBase.java | 97 ++++++++++++++++ .../iceberg/util/IcebergCatalogAdapter.java | 70 ++++++----- .../deephaven/iceberg/util/IcebergTable.java | 59 +++------- .../iceberg/util/IcebergTableRefreshing.java | 59 ++++++++++ .../iceberg/util/IcebergTableStatic.java | 49 ++++++++ 14 files changed, 467 insertions(+), 219 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index 5398bb80a86..a9a6dce38cf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -7,19 +7,15 @@ import io.deephaven.base.verify.Require; import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.TrackingWritableRowSet; -import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.TableUpdateListener; import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocationProvider; -import io.deephaven.engine.table.impl.locations.TableLocationRemovedException; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; import io.deephaven.engine.table.impl.locations.impl.TableLocationSubscriptionBuffer; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.TestUseOnly; import org.apache.commons.lang3.mutable.Mutable; @@ -230,10 +226,13 @@ protected void instrumentedRefresh() { final TableUpdate update = columnSourceManager.refresh(); if (update.empty()) { + update.release(); return; } - rowSet.insert(update.added()); + + Assert.eqTrue(update.shifted().empty(), "update.shifted().empty()"); rowSet.remove(update.removed()); + rowSet.insert(update.added()); notifyListeners(update); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index dd8b65826d4..1390bb3b1f4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -26,25 +26,41 @@ public interface TableLocationProvider extends NamedImplementation { */ interface Listener extends BasicTableDataListener { - void beginTransaction(); + /** + * Begin a transaction that collects location key additions and removals to be processed atomically. + * + * @param token A token to identify the transaction. + */ + void beginTransaction(final Object token); - void endTransaction(); + /** + * End the transaction and process the location changes. + * + * @param token A token to identify the transaction. + */ + void endTransaction(final Object token); /** * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location * subscription. This should occur at most once per location, but the order of delivery is not * guaranteed. * - * @param tableLocationKey The new table location key + * @param tableLocationKey The new table location key. + * @param transactionToken The token identifying the transaction. */ - void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey); + void handleTableLocationKeyAdded( + @NotNull final ImmutableTableLocationKey tableLocationKey, + final Object transactionToken); /** * Notify the listener of a {@link TableLocationKey} that has been removed. * - * @param tableLocationKey The table location key that was removed + * @param tableLocationKey The table location key that was removed. + * @param transactionToken The token identifying the transaction. */ - void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey); + void handleTableLocationKeyRemoved( + @NotNull final ImmutableTableLocationKey tableLocationKey, + final Object transactionToken); } /** @@ -90,7 +106,7 @@ interface Listener extends BasicTableDataListener { * that {@link #refresh()} or {@link #subscribe(Listener)} has been called prior to calls to the various table * location fetch methods. * - * @return this, to allow method chaining + * @return this, to allow method chaining. */ TableLocationProvider ensureInitialized(); @@ -99,7 +115,7 @@ interface Listener extends BasicTableDataListener { * size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is * guaranteed to succeed for all results. * - * @return A collection of keys for locations available from this provider + * @return A collection of keys for locations available from this provider. */ @NotNull Collection getTableLocationKeys(); @@ -107,13 +123,13 @@ interface Listener extends BasicTableDataListener { /** * Check if this provider knows the supplied location key. * - * @param tableLocationKey The key to test for - * @return Whether the key is known to this provider + * @param tableLocationKey The key to test. + * @return Whether the key is known to this provider. */ boolean hasTableLocationKey(@NotNull final TableLocationKey tableLocationKey); /** - * @param tableLocationKey A {@link TableLocationKey} specifying the location to get + * @param tableLocationKey A {@link TableLocationKey} specifying the location to get. * @return The {@link TableLocation} matching the given key */ @NotNull @@ -126,8 +142,8 @@ default TableLocation getTableLocation(@NotNull TableLocationKey tableLocationKe } /** - * @param tableLocationKey A {@link TableLocationKey} specifying the location to get - * @return The {@link TableLocation} matching the given key if present, else null + * @param tableLocationKey A {@link TableLocationKey} specifying the location to get. + * @return The {@link TableLocation} matching the given key if present, else null. */ @Nullable TableLocation getTableLocationIfPresent(@NotNull TableLocationKey tableLocationKey); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 3bd12e5253a..dd3eab3ca54 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -84,9 +84,9 @@ public final ImmutableTableKey getKey() { @Override protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) { - listener.beginTransaction(); - unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded); - listener.endTransaction(); + listener.beginTransaction(this); + unmodifiableTableLocationKeys.forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, this)); + listener.endTransaction(this); } /** @@ -112,8 +112,10 @@ protected final void handleTableLocationKey(@NotNull final TableLocationKey loca visitLocationKey(locationKey); if (locationCreatedRecorder) { verifyPartitionKeys(locationKey); - if (subscriptions.deliverNotification(Listener::handleTableLocationKeyAdded, toKeyImmutable(result), - true)) { + if (subscriptions.deliverNotification( + (listener, tlk) -> listener.handleTableLocationKeyAdded(tlk, + AbstractTableLocationProvider.this), + toKeyImmutable(result), true)) { onEmpty(); } } @@ -122,13 +124,13 @@ protected final void handleTableLocationKey(@NotNull final TableLocationKey loca protected final void beginTransaction() { if (subscriptions != null) { - subscriptions.deliverNotification(Listener::beginTransaction, true); + subscriptions.deliverNotification(listener -> listener.beginTransaction(this), true); } } protected final void endTransaction() { if (subscriptions != null) { - subscriptions.deliverNotification(Listener::endTransaction, true); + subscriptions.deliverNotification(listener -> listener.endTransaction(this), true); } } @@ -237,8 +239,8 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t /** * Remove a {@link TableLocationKey} and its corresponding {@link TableLocation} (if it was created). All * subscribers to this TableLocationProvider will be - * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(ImmutableTableLocationKey) notified}. If the - * TableLocation was created, all of its subscribers will additionally be + * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(ImmutableTableLocationKey, Object)} + * notified}. If the TableLocation was created, all of its subscribers will additionally be * {@link TableLocation.Listener#handleUpdate() notified} that it no longer exists. This TableLocationProvider will * continue to update other locations and will no longer provide or request information about the removed location. * @@ -279,7 +281,8 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey) protected void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey locationKey) { if (supportsSubscriptions()) { synchronized (subscriptions) { - if (subscriptions.deliverNotification(Listener::handleTableLocationKeyRemoved, locationKey, true)) { + if (subscriptions.deliverNotification( + (listener, tlk) -> listener.handleTableLocationKeyRemoved(tlk, this), locationKey, true)) { onEmpty(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index a9157da4b0f..a9e782496ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -115,7 +115,7 @@ public void subscribe(@NotNull final Listener listener) { p.subscribe(listener); } else { p.refresh(); - p.getTableLocationKeys().forEach(listener::handleTableLocationKeyAdded); + p.getTableLocationKeys().forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, this)); } }); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 544852ecb21..7c5c090f780 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -157,38 +157,42 @@ private FilteringListener(@NotNull final TableLocationProvider.Listener outputLi } @Override - public void beginTransaction() { + public void beginTransaction(final Object token) { // Delegate to the wrapped listener. final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null) { - outputListener.beginTransaction(); + outputListener.beginTransaction(token); } } @Override - public void endTransaction() { + public void endTransaction(final Object token) { // Delegate to the wrapped listener. final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null) { - outputListener.endTransaction(); + outputListener.endTransaction(token); } } @Override - public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyAdded( + @NotNull final ImmutableTableLocationKey tableLocationKey, + final Object transactionToken) { final TableLocationProvider.Listener outputListener = getWrapped(); // We can't try to clean up null listeners here, the underlying implementation may not allow concurrent // unsubscribe operations. if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { - outputListener.handleTableLocationKeyAdded(tableLocationKey); + outputListener.handleTableLocationKeyAdded(tableLocationKey, transactionToken); } } @Override - public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyRemoved( + @NotNull final ImmutableTableLocationKey tableLocationKey, + final Object transactionToken) { final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { - outputListener.handleTableLocationKeyRemoved(tableLocationKey); + outputListener.handleTableLocationKeyRemoved(tableLocationKey, transactionToken); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index 05a32b5fb6d..c9998af0ee6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -9,10 +9,7 @@ import io.deephaven.engine.table.impl.locations.TableLocationProvider; import org.jetbrains.annotations.NotNull; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; +import java.util.*; /** * Intermediates between push-based subscription to a TableLocationProvider and polling on update source refresh. @@ -27,12 +24,14 @@ public class TableLocationSubscriptionBuffer implements TableLocationProvider.Li private final Object updateLock = new Object(); - // These sets represent a completed transaction of adds and removes. + // These sets represent adds and removes from completed transactions. private Set pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; - private Set accumulatedLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; - private Set accumulatedLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + // These sets represent open transactions that are being accumulated. + private final Set transactionTokens = new HashSet<>(); + private final Map> accumulatedLocationsAdded = new HashMap<>(); + private final Map> accumulatedLocationsRemoved = new HashMap<>(); private TableDataException pendingException = null; @@ -65,7 +64,7 @@ public Collection getPendingRemovedLocationKeys() { * reset). No order is maintained internally. If a pending exception is thrown, this signals that the subscription * is no longer valid and no subsequent location keys will be returned. * - * @return The collection of pending location keys + * @return The collection of pending location keys. */ public synchronized LocationUpdate processPending() { // TODO: Should I change this to instead re-use the collection? @@ -76,7 +75,8 @@ public synchronized LocationUpdate processPending() { // NB: Providers that don't support subscriptions don't tick - this single call to run is // sufficient. tableLocationProvider.refresh(); - tableLocationProvider.getTableLocationKeys().forEach(this::handleTableLocationKeyAdded); + tableLocationProvider.getTableLocationKeys() + .forEach(tlk -> handleTableLocationKeyAdded(tlk, tableLocationProvider)); } subscribed = true; } @@ -121,65 +121,96 @@ public synchronized void reset() { // ------------------------------------------------------------------------------------------------------------------ @Override - public void beginTransaction() { + public void beginTransaction(final Object token) { synchronized (updateLock) { - // Assert that we can start a new transaction. - Require.eq(EMPTY_TABLE_LOCATION_KEYS, "accumulatedLocationsAdded", accumulatedLocationsAdded); - Require.eq(EMPTY_TABLE_LOCATION_KEYS, "accumulatedLocationsRemoved", accumulatedLocationsRemoved); + // Assert that we can start a new transaction with this token. + Require.eqFalse(transactionTokens.contains(token), "transactionTokens.contains(token)"); + Require.eqFalse(accumulatedLocationsAdded.containsKey(token), + "accumulatedLocationsAdded.containsKey(token)"); + Require.eqFalse(accumulatedLocationsRemoved.containsKey(token), + "accumulatedLocationsRemoved.containsKey(token)"); + + transactionTokens.add(token); + accumulatedLocationsAdded.put(token, EMPTY_TABLE_LOCATION_KEYS); + accumulatedLocationsRemoved.put(token, EMPTY_TABLE_LOCATION_KEYS); } } @Override - public void endTransaction() { + public void endTransaction(final Object token) { synchronized (updateLock) { - // If we already have a completed transaction, merge the accumulated sets into it. - if (pendingLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) { - pendingLocationsAdded.addAll(accumulatedLocationsAdded); - } else { - pendingLocationsAdded = accumulatedLocationsAdded; - } + // Assert that this transaction is open. + Require.eqTrue(transactionTokens.contains(token), "transactionTokens.contains(token)"); + + final Set tokenLocationsAdded = accumulatedLocationsAdded.get(token); + final Set tokenLocationsRemoved = accumulatedLocationsRemoved.get(token); if (pendingLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { - // Remove any locations that were added and then removed by this transaction. - for (final ImmutableTableLocationKey tableLocationKey : accumulatedLocationsRemoved) { + // Handle any locations that were pending as adds but removed by this transaction. + for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsRemoved) { if (pendingLocationsAdded.remove(tableLocationKey)) { continue; } pendingLocationsRemoved.add(tableLocationKey); } } else { - pendingLocationsRemoved = accumulatedLocationsRemoved; + pendingLocationsRemoved = tokenLocationsRemoved; } - // Reset the accumulated sets to empty. - accumulatedLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; - accumulatedLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + if (pendingLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) { + // Handle any locations that were pending as removes but added again by this transaction. + for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsAdded) { + if (pendingLocationsRemoved.remove(tableLocationKey)) { + continue; + } + pendingLocationsAdded.add(tableLocationKey); + } + } else { + pendingLocationsAdded = tokenLocationsAdded; + } + + // Clear all the storage for this transaction. + transactionTokens.remove(token); + accumulatedLocationsAdded.remove(token); + accumulatedLocationsRemoved.remove(token); } } @Override - public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyAdded( + @NotNull final ImmutableTableLocationKey tableLocationKey, + final Object transactionToken) { synchronized (updateLock) { - if (accumulatedLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { - accumulatedLocationsAdded = new HashSet<>(); + if (accumulatedLocationsAdded.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { + accumulatedLocationsAdded.put(transactionToken, new HashSet<>()); } - accumulatedLocationsAdded.add(tableLocationKey); + final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken); + final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken); + + // A single transaction should never add and remove the same location, + Require.eqFalse(locationsRemoved.contains(tableLocationKey), + "locationsRemoved.contains(tableLocationKey)"); + + locationsAdded.add(tableLocationKey); } } @Override - public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyRemoved( + @NotNull final ImmutableTableLocationKey tableLocationKey, + final Object transactionToken) { synchronized (updateLock) { - // TODO: is it possible that we add and remove a location within a single transaction? Paranoid? - // If we remove something that was pending to be added, just discard both. - if (accumulatedLocationsRemoved.remove(tableLocationKey)) { - return; + if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { + accumulatedLocationsRemoved.put(transactionToken, new HashSet<>()); } + final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken); + final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken); - if (accumulatedLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { - accumulatedLocationsRemoved = new HashSet<>(); - } - accumulatedLocationsRemoved.add(tableLocationKey); + // A single transaction should never add and remove the same location, + Require.eqFalse(locationsAdded.contains(tableLocationKey), + "locationsAdded.contains(tableLocationKey)"); + + locationsRemoved.add(tableLocationKey); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 6d70dbf912f..0bcb4e66724 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -4,6 +4,7 @@ package io.deephaven.engine.table.impl.sources.regioned; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.liveness.LivenessArtifact; import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.*; @@ -18,6 +19,7 @@ import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; import io.deephaven.engine.table.impl.sources.ObjectArraySource; import io.deephaven.engine.table.impl.util.DelayedErrorNotifier; +import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; import io.deephaven.internal.log.LoggerFactory; @@ -220,7 +222,12 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati } else if (includedLocation != null) { orderedIncludedTableLocations.remove(includedLocation); removedTableLocations.add(includedLocation); - // includedLocation.invalidate(); + + final UpdateCommitter committer = new UpdateCommitter<>(this, + ExecutionContext.getContext().getUpdateGraph(), + (ignored) -> includedLocation.invalidate()); + + committer.maybeActivate(); return true; } @@ -233,8 +240,10 @@ public synchronized TrackingWritableRowSet initialize() { // Do our first pass over the locations to include as many as possible and build the initial row set // noinspection resource - final TableUpdate update = update(true); + final TableUpdateImpl update = update(true); final TrackingWritableRowSet initialRowSet = update.added().writableCast().toTracking(); + update.added = null; + update.release(); // Add single-column data indexes for all partitioning columns, whether refreshing or not columnDefinitions.stream().filter(ColumnDefinition::isPartitioning).forEach(cd -> { @@ -295,7 +304,7 @@ public void deliverError(@NotNull final Throwable error, @Nullable final TableLi } } - private TableUpdate update(final boolean initializing) { + private TableUpdateImpl update(final boolean initializing) { final RowSetBuilderSequential addedRowSetBuilder = RowSetFactory.builderSequential(); final RowSetBuilderSequential removedRowSetBuilder = RowSetFactory.builderSequential(); @@ -305,9 +314,7 @@ private TableUpdate update(final boolean initializing) { removedTableLocations.sort(Comparator.comparingInt(e -> e.regionIndex)); for (final IncludedTableLocationEntry removedLocation : removedTableLocations) { final long regionFirstKey = RegionedColumnSource.getFirstRowKey(removedLocation.regionIndex); - removedLocation.location.getRowSet() - .forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> removedRowSetBuilder - .appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey)); + removedRowSetBuilder.appendRowSequenceWithOffset(removedLocation.location.getRowSet(), regionFirstKey); removedRegionBuilder.appendKey(removedLocation.regionIndex); } removedTableLocations.clear(); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java index 43aa3d78939..e6958dc63b3 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java @@ -4,7 +4,6 @@ package io.deephaven.iceberg.layout; import io.deephaven.engine.table.impl.locations.*; -import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; @@ -29,15 +28,10 @@ *

*/ public class IcebergRefreshingTableLocationProvider - extends AbstractTableLocationProvider { + extends IcebergTableLocationProviderBase { private static final String IMPLEMENTATION_NAME = IcebergRefreshingTableLocationProvider.class.getSimpleName(); - private final IcebergBaseLayout locationKeyFinder; - private final TableLocationFactory locationFactory; - private final TableDataRefreshService refreshService; - private final IcebergCatalogAdapter adapter; - private final TableIdentifier tableIdentifier; private final boolean autoRefresh; private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken; @@ -50,12 +44,7 @@ public IcebergRefreshingTableLocationProvider( @NotNull final IcebergCatalogAdapter adapter, @NotNull final TableIdentifier tableIdentifier, final boolean autoRefresh) { - super(tableKey, refreshService != null); - this.locationKeyFinder = locationKeyFinder; - this.locationFactory = locationFactory; - this.refreshService = refreshService; - this.adapter = adapter; - this.tableIdentifier = tableIdentifier; + super(tableKey, locationKeyFinder, locationFactory, refreshService, adapter, tableIdentifier); this.autoRefresh = autoRefresh; } @@ -79,19 +68,12 @@ public synchronized void refresh() { refreshSnapshot(); } - /** - * Update the table location provider with the latest snapshot from the catalog. - */ + @Override public synchronized void update() { update(adapter.getCurrentSnapshot(tableIdentifier)); } - /** - * Update the table location provider with a specific snapshot from the catalog. If the {@code snapshotId} is not - * found in the list of snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot - * must also be newer (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} - * is thrown. - */ + @Override public synchronized void update(final long snapshotId) { final List snapshots = adapter.listSnapshots(tableIdentifier); @@ -107,12 +89,7 @@ public synchronized void update(final long snapshotId) { update(snapshot); } - /** - * Update the table location provider with a specific snapshot from the catalog. The input snapshot must be newer - * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown. - * - * @param snapshot - */ + @Override public synchronized void update(final Snapshot snapshot) { // Verify that the input snapshot is newer (higher in sequence number) than the current snapshot. if (snapshot.sequenceNumber() <= locationKeyFinder.snapshot.sequenceNumber()) { @@ -134,7 +111,6 @@ private void refreshSnapshot() { beginTransaction(); final Set missedKeys = new HashSet<>(getTableLocationKeys()); locationKeyFinder.findKeys(tableLocationKey -> { - // noinspection SuspiciousMethodCalls missedKeys.remove(tableLocationKey); handleTableLocationKey(tableLocationKey); }); @@ -142,33 +118,4 @@ private void refreshSnapshot() { endTransaction(); setInitialized(); } - - @Override - @NotNull - protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) { - // noinspection unchecked - return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService); - } - - // ------------------------------------------------------------------------------------------------------------------ - // SubscriptionAggregator implementation - // ------------------------------------------------------------------------------------------------------------------ - - @Override - protected final void activateUnderlyingDataSource() { - subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this); - } - - @Override - protected final void deactivateUnderlyingDataSource() { - if (subscriptionToken != null) { - subscriptionToken.cancel(); - subscriptionToken = null; - } - } - - @Override - protected final boolean matchSubscriptionToken(final T token) { - return token == subscriptionToken; - } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java new file mode 100644 index 00000000000..4a93d985af0 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -0,0 +1,65 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.layout; + +import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.TableIdentifier; +import org.jetbrains.annotations.NotNull; + +/** + *

+ * Static {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} discovery to + * a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a {@link TableLocationFactory}. + *

+ */ +public class IcebergStaticTableLocationProvider + extends IcebergTableLocationProviderBase { + + private static final String IMPLEMENTATION_NAME = IcebergStaticTableLocationProvider.class.getSimpleName(); + + public IcebergStaticTableLocationProvider( + @NotNull final TK tableKey, + @NotNull final IcebergBaseLayout locationKeyFinder, + @NotNull final TableLocationFactory locationFactory, + @NotNull final IcebergCatalogAdapter adapter, + @NotNull final TableIdentifier tableIdentifier) { + super(tableKey, locationKeyFinder, locationFactory, null, adapter, tableIdentifier); + } + + // ------------------------------------------------------------------------------------------------------------------ + // AbstractTableLocationProvider implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + @Override + public void refresh() { + beginTransaction(); + locationKeyFinder.findKeys(this::handleTableLocationKey); + endTransaction(); + setInitialized(); + } + + @Override + public void update() { + throw new IllegalStateException("A static table location provider cannot be updated"); + } + + @Override + public void update(long snapshotId) { + throw new IllegalStateException("A static table location provider cannot be updated"); + } + + @Override + public void update(Snapshot snapshot) { + throw new IllegalStateException("A static table location provider cannot be updated"); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java new file mode 100644 index 00000000000..52c98718317 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java @@ -0,0 +1,97 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.layout; + +import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; +import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.TableIdentifier; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public abstract class IcebergTableLocationProviderBase + extends AbstractTableLocationProvider { + + final IcebergBaseLayout locationKeyFinder; + final TableLocationFactory locationFactory; + final TableDataRefreshService refreshService; + final IcebergCatalogAdapter adapter; + final TableIdentifier tableIdentifier; + + private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken; + + public IcebergTableLocationProviderBase( + @NotNull final TK tableKey, + @NotNull final IcebergBaseLayout locationKeyFinder, + @NotNull final TableLocationFactory locationFactory, + @Nullable final TableDataRefreshService refreshService, + @NotNull final IcebergCatalogAdapter adapter, + @NotNull final TableIdentifier tableIdentifier) { + super(tableKey, refreshService != null); + this.locationKeyFinder = locationKeyFinder; + this.locationFactory = locationFactory; + this.refreshService = refreshService; + this.adapter = adapter; + this.tableIdentifier = tableIdentifier; + } + + // ------------------------------------------------------------------------------------------------------------------ + // AbstractTableLocationProvider implementation + // ------------------------------------------------------------------------------------------------------------------ + + /** + * Update the table location provider with the latest snapshot from the catalog. + */ + public abstract void update(); + + /** + * Update the table location provider with a specific snapshot from the catalog. If the {@code snapshotId} is not + * found in the list of snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot + * must also be newer (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} + * is thrown. + * + * @param snapshotId The identifier of the snapshot to use when updating the table. + */ + public abstract void update(final long snapshotId); + + /** + * Update the table location provider with a specific snapshot from the catalog. The input snapshot must be newer + * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown. + * + * @param snapshot The snapshot to use when updating the table. + */ + public abstract void update(final Snapshot snapshot); + + @Override + @NotNull + protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) { + // noinspection unchecked + return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService); + } + + // ------------------------------------------------------------------------------------------------------------------ + // SubscriptionAggregator implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + protected final void activateUnderlyingDataSource() { + subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this); + } + + @Override + protected final void deactivateUnderlyingDataSource() { + if (subscriptionToken != null) { + subscriptionToken.cancel(); + subscriptionToken = null; + } + } + + @Override + protected final boolean matchSubscriptionToken(final T token) { + return token == subscriptionToken; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index 76d099e9499..bf23dbd6e85 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -6,23 +6,17 @@ import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.PartitionAwareSourceTable; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableKey; -import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder; -import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; import io.deephaven.engine.table.impl.locations.util.ExecutorTableDataRefreshService; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; -import io.deephaven.iceberg.layout.IcebergBaseLayout; -import io.deephaven.iceberg.layout.IcebergFlatLayout; -import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout; -import io.deephaven.iceberg.layout.IcebergRefreshingTableLocationProvider; +import io.deephaven.iceberg.layout.*; import io.deephaven.iceberg.location.IcebergTableLocationFactory; import io.deephaven.iceberg.location.IcebergTableLocationKey; import org.apache.iceberg.PartitionField; @@ -354,11 +348,18 @@ public Table listSnapshotsAsTable(@NotNull final String tableIdentifier) { return listSnapshotsAsTable(TableIdentifier.parse(tableIdentifier)); } - Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long snapshotId) { + /** + * Retrieve a specific {@link Snapshot snapshot} of an Iceberg table. + * + * @param tableIdentifier The identifier of the table from which to load the snapshot. + * @param snapshotId The identifier of the snapshot to load. + * + * @return An Optional containing the requested snapshot if it exists. + */ + Optional getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long snapshotId) { return listSnapshots(tableIdentifier).stream() .filter(snapshot -> snapshot.snapshotId() == snapshotId) - .findFirst() - .orElse(null); + .findFirst(); } /** @@ -380,7 +381,7 @@ public Snapshot getCurrentSnapshot(@NotNull final TableIdentifier tableIdentifie * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + public IcebergTable readTable( @NotNull final TableIdentifier tableIdentifier, @Nullable final IcebergInstructions instructions) { return readTableInternal(tableIdentifier, null, instructions); @@ -394,7 +395,7 @@ public Table readTable( * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + public IcebergTable readTable( @NotNull final String tableIdentifier, @Nullable final IcebergInstructions instructions) { return readTable(TableIdentifier.parse(tableIdentifier), instructions); @@ -409,16 +410,15 @@ public Table readTable( * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + public IcebergTable readTable( @NotNull final TableIdentifier tableIdentifier, final long tableSnapshotId, @Nullable final IcebergInstructions instructions) { // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = getSnapshot(tableIdentifier, tableSnapshotId); - if (tableSnapshot == null) { - throw new IllegalArgumentException("Snapshot with id " + tableSnapshotId + " not found"); - } + final Snapshot tableSnapshot = + getSnapshot(tableIdentifier, tableSnapshotId).orElseThrow(() -> new IllegalArgumentException( + "Snapshot with id " + tableSnapshotId + " not found for table " + tableIdentifier)); return readTableInternal(tableIdentifier, tableSnapshot, instructions); } @@ -432,7 +432,7 @@ public Table readTable( * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + public IcebergTable readTable( @NotNull final String tableIdentifier, final long tableSnapshotId, @Nullable final IcebergInstructions instructions) { @@ -448,14 +448,14 @@ public Table readTable( * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + public IcebergTable readTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { return readTableInternal(tableIdentifier, tableSnapshot, instructions); } - private Table readTableInternal( + private IcebergTable readTableInternal( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { @@ -506,7 +506,6 @@ private Table readTableInternal( tableDef = icebergTableDef; } - final String description; final IcebergBaseLayout keyFinder; if (partitionSpec.isUnpartitioned()) { @@ -519,27 +518,25 @@ private Table readTableInternal( } if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.STATIC) { - description = "Read static iceberg table with " + keyFinder; - - final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>( - StandaloneTableKey.getInstance(), - toKnownKeys(keyFinder), - new IcebergTableLocationFactory(), - null); - - return new PartitionAwareSourceTable( + final IcebergTableLocationProviderBase locationProvider = + new IcebergStaticTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + this, + tableIdentifier); + + return new IcebergTableStatic( tableDef, - description, + tableIdentifier.toString(), RegionedTableComponentFactoryImpl.INSTANCE, - locationProvider, - null); + locationProvider); } final UpdateSourceRegistrar updateSourceRegistrar; final IcebergRefreshingTableLocationProvider locationProvider; if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) { - description = "Read manual refreshing iceberg table with " + keyFinder; updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); locationProvider = new IcebergRefreshingTableLocationProvider<>( @@ -551,7 +548,6 @@ private Table readTableInternal( tableIdentifier, false); } else { - description = "Read automatic refreshing iceberg table with " + keyFinder; updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); final TableDataRefreshService refreshService = @@ -567,9 +563,9 @@ private Table readTableInternal( true); } - return new IcebergTable( + return new IcebergTableRefreshing( tableDef, - description, + tableIdentifier.toString(), RegionedTableComponentFactoryImpl.INSTANCE, locationProvider, updateSourceRegistrar); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java index 9c422b96968..a0ae36a6eaa 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java @@ -3,58 +3,33 @@ // package io.deephaven.iceberg.util; -import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.impl.PartitionAwareSourceTable; -import io.deephaven.engine.table.impl.SourceTableComponentFactory; -import io.deephaven.engine.table.impl.locations.TableKey; -import io.deephaven.engine.updategraph.UpdateSourceRegistrar; -import io.deephaven.iceberg.layout.IcebergRefreshingTableLocationProvider; -import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.engine.table.Table; import org.apache.iceberg.Snapshot; -import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -/** - * Table wrapper for refreshing Iceberg tables. - */ -public class IcebergTable extends PartitionAwareSourceTable { +public interface IcebergTable extends Table { /** - * Location discovery. + * Update the table with the latest snapshot from the catalog. */ - final IcebergRefreshingTableLocationProvider locationProvider; + @SuppressWarnings("unused") + void update(); /** + * Update the table with a specific snapshot from the catalog. If the {@code snapshotId} is not found in the list of + * snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot must also be newer + * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown. * - * - * @param tableDefinition A TableDefinition - * @param description A human-readable description for this table - * @param componentFactory A component factory for creating column source managers - * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table - * @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live + * @param snapshotId The identifier of the snapshot to use when updating the table. */ - IcebergTable( - @NotNull TableDefinition tableDefinition, - @NotNull String description, - @NotNull SourceTableComponentFactory componentFactory, - @NotNull IcebergRefreshingTableLocationProvider locationProvider, - @Nullable UpdateSourceRegistrar updateSourceRegistrar) { - super(tableDefinition, description, componentFactory, locationProvider, updateSourceRegistrar); - this.locationProvider = locationProvider; - } - - @SuppressWarnings("unused") - public void update() { - locationProvider.update(); - } - @SuppressWarnings("unused") - public void update(final long snapshotId) { - locationProvider.update(snapshotId); - } + void update(final long snapshotId); + /** + * Update the table with a specific snapshot from the catalog. The input snapshot must be newer (higher in sequence + * number) than the current snapshot or an {@link IllegalArgumentException} is thrown. + * + * @param snapshot The snapshot to use when updating the table. + */ @SuppressWarnings("unused") - public void update(final @NotNull Snapshot snapshot) { - locationProvider.update(snapshot); - } + void update(final @NotNull Snapshot snapshot); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java new file mode 100644 index 00000000000..72623b9e200 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java @@ -0,0 +1,59 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.PartitionAwareSourceTable; +import io.deephaven.engine.table.impl.SourceTableComponentFactory; +import io.deephaven.engine.table.impl.locations.TableKey; +import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.iceberg.layout.IcebergTableLocationProviderBase; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import org.apache.iceberg.Snapshot; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Table wrapper for refreshing Iceberg tables. + */ +public class IcebergTableRefreshing extends PartitionAwareSourceTable implements IcebergTable { + /** + * Location discovery. + */ + final IcebergTableLocationProviderBase locationProvider; + + /** + * + * + * @param tableDefinition A TableDefinition + * @param description A human-readable description for this table + * @param componentFactory A component factory for creating column source managers + * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table + * @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live + */ + IcebergTableRefreshing( + @NotNull TableDefinition tableDefinition, + @NotNull String description, + @NotNull SourceTableComponentFactory componentFactory, + @NotNull IcebergTableLocationProviderBase locationProvider, + @Nullable UpdateSourceRegistrar updateSourceRegistrar) { + super(tableDefinition, description, componentFactory, locationProvider, updateSourceRegistrar); + this.locationProvider = locationProvider; + } + + @Override + public void update() { + locationProvider.update(); + } + + @Override + public void update(final long snapshotId) { + locationProvider.update(snapshotId); + } + + @Override + public void update(final @NotNull Snapshot snapshot) { + locationProvider.update(snapshot); + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java new file mode 100644 index 00000000000..0884a6379c5 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java @@ -0,0 +1,49 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.PartitionAwareSourceTable; +import io.deephaven.engine.table.impl.SourceTableComponentFactory; +import io.deephaven.engine.table.impl.locations.TableKey; +import io.deephaven.iceberg.layout.IcebergTableLocationProviderBase; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import org.apache.iceberg.Snapshot; +import org.jetbrains.annotations.NotNull; + +/** + * Table wrapper for refreshing Iceberg tables. + */ +public class IcebergTableStatic extends PartitionAwareSourceTable implements IcebergTable { + /** + * + * + * @param tableDefinition A TableDefinition + * @param description A human-readable description for this table + * @param componentFactory A component factory for creating column source managers + * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table + */ + IcebergTableStatic( + @NotNull TableDefinition tableDefinition, + @NotNull String description, + @NotNull SourceTableComponentFactory componentFactory, + @NotNull IcebergTableLocationProviderBase locationProvider) { + super(tableDefinition, description, componentFactory, locationProvider, null); + } + + @Override + public void update() { + throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description); + } + + @Override + public void update(final long snapshotId) { + throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description); + } + + @Override + public void update(final @NotNull Snapshot snapshot) { + throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description); + } +} From fb882e8e76956f940fc74362ae307b5d3afa6137 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 26 Jul 2024 12:11:48 -0700 Subject: [PATCH 06/56] WIP, some PR comments addressed. --- .../util/datastructures/SubscriptionSet.java | 31 +++++++ .../impl/locations/TableLocationProvider.java | 49 +++++++++-- .../impl/AbstractTableLocationProvider.java | 86 ++++++++++++++++--- .../impl/FilteredTableDataService.java | 8 +- .../impl/PollingTableLocationProvider.java | 2 +- .../impl/TableLocationSubscriptionBuffer.java | 55 +++++++++--- .../TableBackedTableLocationProvider.java | 2 +- ...cebergRefreshingTableLocationProvider.java | 2 +- .../IcebergStaticTableLocationProvider.java | 2 +- 9 files changed, 196 insertions(+), 41 deletions(-) diff --git a/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java b/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java index 0d9fc96a8c0..d5c1f711a93 100644 --- a/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java +++ b/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java @@ -7,6 +7,7 @@ import io.deephaven.base.reference.WeakReferenceWrapper; import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; +import org.apache.commons.lang3.function.TriConsumer; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -219,6 +220,36 @@ public final boolean deliverNotification( return initialSize > 0 && size == 0; } + /** + * Dispatch a binary notification to all subscribers. Clean up any GC'd subscriptions. + * + * @param procedure The notification procedure to invoke + * @param notification The notification to deliver + * @param token An additional token to deliver (usually a transaction token) + * @param activeOnly Whether to restrict this notification to active subscriptions only + * @return Whether this operation caused the set to become empty + */ + public final boolean deliverNotification( + @NotNull final TriConsumer procedure, + @Nullable final NOTIFICATION_TYPE notification, + @Nullable final Object token, + final boolean activeOnly) { + final int initialSize = size; + for (int si = 0; si < size;) { + final Entry currentEntry = subscriptions[si]; + final LISTENER_TYPE currentListener = currentEntry.getListener(); + if (currentListener == null) { + removeAt(si); + continue; // si is not incremented in this case - we'll reconsider the same slot if necessary. + } + if (!activeOnly || currentEntry.isActive()) { + procedure.accept(currentListener, notification, token); + } + ++si; + } + return initialSize > 0 && size == 0; + } + private void removeAt(final int subscriptionIndex) { final int lastSubscriptionIndex = --size; subscriptions[subscriptionIndex] = subscriptions[lastSubscriptionIndex]; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index 1390bb3b1f4..9ba7a350d81 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -31,14 +31,29 @@ interface Listener extends BasicTableDataListener { * * @param token A token to identify the transaction. */ - void beginTransaction(final Object token); + void beginTransaction(@NotNull Object token); + + /** + * Begin a transaction that collects location key additions and removals to be processed atomically. Uses + * {@code this} as the token. + */ + default void beginTransaction() { + beginTransaction(this); + } /** * End the transaction and process the location changes. * * @param token A token to identify the transaction. */ - void endTransaction(final Object token); + void endTransaction(@NotNull Object token); + + /** + * End the transaction and process the location changes. Uses {@code this} as the token. + */ + default void endTransaction() { + endTransaction(this); + } /** * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location @@ -49,8 +64,19 @@ interface Listener extends BasicTableDataListener { * @param transactionToken The token identifying the transaction. */ void handleTableLocationKeyAdded( - @NotNull final ImmutableTableLocationKey tableLocationKey, - final Object transactionToken); + @NotNull ImmutableTableLocationKey tableLocationKey, + @Nullable Object transactionToken); + + /** + * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location + * subscription. This should occur at most once per location, but the order of delivery is not + * guaranteed. Uses {@code this} as the token. + * + * @param tableLocationKey The new table location key. + */ + default void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey) { + handleTableLocationKeyAdded(tableLocationKey, this); + } /** * Notify the listener of a {@link TableLocationKey} that has been removed. @@ -59,8 +85,17 @@ void handleTableLocationKeyAdded( * @param transactionToken The token identifying the transaction. */ void handleTableLocationKeyRemoved( - @NotNull final ImmutableTableLocationKey tableLocationKey, - final Object transactionToken); + @NotNull ImmutableTableLocationKey tableLocationKey, + @Nullable Object transactionToken); + + /** + * Notify the listener of a {@link TableLocationKey} that has been removed. Uses {@code this} as the token. + * + * @param tableLocationKey The table location key that was removed. + */ + default void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey) { + handleTableLocationKeyRemoved(tableLocationKey, this); + } } /** @@ -126,7 +161,7 @@ void handleTableLocationKeyRemoved( * @param tableLocationKey The key to test. * @return Whether the key is known to this provider. */ - boolean hasTableLocationKey(@NotNull final TableLocationKey tableLocationKey); + boolean hasTableLocationKey(@NotNull TableLocationKey tableLocationKey); /** * @param tableLocationKey A {@link TableLocationKey} specifying the location to get. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index dd3eab3ca54..1a3f7ead2da 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -16,7 +16,7 @@ * Partial {@link TableLocationProvider} implementation for standalone use or as part of a {@link TableDataService}. *

* Presents an interface similar to {@link TableLocationProvider.Listener} for subclasses to use when communicating with - * the parent; see {@link #handleTableLocationKey(TableLocationKey)}. + * the parent; see {@link #handleTableLocationKeyAdded(TableLocationKey)}. *

* Note that subclasses are responsible for determining when it's appropriate to call {@link #setInitialized()} and/or * override {@link #doInitialization()}. @@ -84,18 +84,32 @@ public final ImmutableTableKey getKey() { @Override protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) { - listener.beginTransaction(this); - unmodifiableTableLocationKeys.forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, this)); - listener.endTransaction(this); + listener.beginTransaction(); + unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded); + listener.endTransaction(); + } + + /** + * Deliver a possibly-new key. This method passes {@code this} as the transaction token. + * + * @param locationKey The new key + * @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools. + */ + protected final void handleTableLocationKeyAdded( + @NotNull final TableLocationKey locationKey) { + handleTableLocationKeyAdded(locationKey, this); } /** * Deliver a possibly-new key. * * @param locationKey The new key + * @param transactionToken The token identifying the transaction * @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools. */ - protected final void handleTableLocationKey(@NotNull final TableLocationKey locationKey) { + protected final void handleTableLocationKeyAdded( + @NotNull final TableLocationKey locationKey, + @Nullable final Object transactionToken) { if (!supportsSubscriptions()) { tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable); visitLocationKey(toKeyImmutable(locationKey)); @@ -113,30 +127,57 @@ protected final void handleTableLocationKey(@NotNull final TableLocationKey loca if (locationCreatedRecorder) { verifyPartitionKeys(locationKey); if (subscriptions.deliverNotification( - (listener, tlk) -> listener.handleTableLocationKeyAdded(tlk, - AbstractTableLocationProvider.this), - toKeyImmutable(result), true)) { + Listener::handleTableLocationKeyAdded, + toKeyImmutable(result), + transactionToken, + true)) { onEmpty(); } } } } + /** + * Internal method to begin an atomic transaction of location adds and removes. This method passes {@code this} as + * the transaction token. + */ protected final void beginTransaction() { + beginTransaction(this); + } + + /** + * Internal method to begin an atomic transaction of location adds and removes. + * + * @param token A token to identify the transaction + */ + protected final void beginTransaction(@NotNull final Object token) { if (subscriptions != null) { - subscriptions.deliverNotification(listener -> listener.beginTransaction(this), true); + subscriptions.deliverNotification(Listener::beginTransaction, token, true); } } + /** + * Internal method to end an atomic transaction of location adds and removes. This method passes {@code this} as the + * transaction token. + */ protected final void endTransaction() { + endTransaction(this); + } + + /** + * Internal method to end an atomic transaction of location adds and removes. + * + * @param token A token to identify the transaction + */ + protected final void endTransaction(@NotNull final Object token) { if (subscriptions != null) { - subscriptions.deliverNotification(listener -> listener.endTransaction(this), true); + subscriptions.deliverNotification(Listener::endTransaction, token, true); } } /** - * Called after a table location has been visited by {@link #handleTableLocationKey(TableLocationKey)}, but - * before notifications have been delivered to any subscriptions, if applicable. The default implementation does + * Called after a table location has been visited by {@link #handleTableLocationKeyAdded(TableLocationKey)}, + * but before notifications have been delivered to any subscriptions, if applicable. The default implementation does * nothing, and may be overridden to implement additional features. * * @param locationKey The {@link TableLocationKey} that was visited. @@ -273,16 +314,33 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey) } } + /** + * Notify subscribers that {@code locationKey} was removed. This method passes {@code this} as the transaction + * token. + * + * @param locationKey the TableLocation that was removed + */ + protected final void handleTableLocationKeyRemoved( + @NotNull final ImmutableTableLocationKey locationKey) { + handleTableLocationKeyRemoved(locationKey, this); + } + /** * Notify subscribers that {@code locationKey} was removed. * * @param locationKey the TableLocation that was removed + * @param transactionToken The token identifying the transaction */ - protected void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey locationKey) { + protected void handleTableLocationKeyRemoved( + @NotNull final ImmutableTableLocationKey locationKey, + @Nullable final Object transactionToken) { if (supportsSubscriptions()) { synchronized (subscriptions) { if (subscriptions.deliverNotification( - (listener, tlk) -> listener.handleTableLocationKeyRemoved(tlk, this), locationKey, true)) { + Listener::handleTableLocationKeyRemoved, + locationKey, + transactionToken, + true)) { onEmpty(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 7c5c090f780..6592ea11a12 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -157,7 +157,7 @@ private FilteringListener(@NotNull final TableLocationProvider.Listener outputLi } @Override - public void beginTransaction(final Object token) { + public void beginTransaction(@NotNull final Object token) { // Delegate to the wrapped listener. final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null) { @@ -166,7 +166,7 @@ public void beginTransaction(final Object token) { } @Override - public void endTransaction(final Object token) { + public void endTransaction(@NotNull final Object token) { // Delegate to the wrapped listener. final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null) { @@ -177,7 +177,7 @@ public void endTransaction(final Object token) { @Override public void handleTableLocationKeyAdded( @NotNull final ImmutableTableLocationKey tableLocationKey, - final Object transactionToken) { + @Nullable final Object transactionToken) { final TableLocationProvider.Listener outputListener = getWrapped(); // We can't try to clean up null listeners here, the underlying implementation may not allow concurrent // unsubscribe operations. @@ -189,7 +189,7 @@ public void handleTableLocationKeyAdded( @Override public void handleTableLocationKeyRemoved( @NotNull final ImmutableTableLocationKey tableLocationKey, - final Object transactionToken) { + @Nullable final Object transactionToken) { final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { outputListener.handleTableLocationKeyRemoved(tableLocationKey, transactionToken); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java index 0f9302b7a77..5ac5c65c4e0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java @@ -57,7 +57,7 @@ public void refresh() { locationKeyFinder.findKeys(tableLocationKey -> { // noinspection SuspiciousMethodCalls missedKeys.remove(tableLocationKey); - handleTableLocationKey(tableLocationKey); + handleTableLocationKeyAdded(tableLocationKey); }); missedKeys.forEach(this::handleTableLocationKeyRemoved); endTransaction(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index c9998af0ee6..29e654a3633 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocationProvider; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.*; @@ -121,7 +122,7 @@ public synchronized void reset() { // ------------------------------------------------------------------------------------------------------------------ @Override - public void beginTransaction(final Object token) { + public void beginTransaction(@NotNull final Object token) { synchronized (updateLock) { // Assert that we can start a new transaction with this token. Require.eqFalse(transactionTokens.contains(token), "transactionTokens.contains(token)"); @@ -137,7 +138,7 @@ public void beginTransaction(final Object token) { } @Override - public void endTransaction(final Object token) { + public void endTransaction(@NotNull final Object token) { synchronized (updateLock) { // Assert that this transaction is open. Require.eqTrue(transactionTokens.contains(token), "transactionTokens.contains(token)"); @@ -179,18 +180,31 @@ public void endTransaction(final Object token) { @Override public void handleTableLocationKeyAdded( @NotNull final ImmutableTableLocationKey tableLocationKey, - final Object transactionToken) { + @Nullable Object transactionToken) { synchronized (updateLock) { + if (transactionToken == null) { + // If we're not in a transaction, modify the pending locations directly. + if (pendingLocationsRemoved.remove(tableLocationKey)) { + return; + } + pendingLocationsAdded.add(tableLocationKey); + return; + } + if (accumulatedLocationsAdded.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { accumulatedLocationsAdded.put(transactionToken, new HashSet<>()); } final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken); - final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken); - - // A single transaction should never add and remove the same location, - Require.eqFalse(locationsRemoved.contains(tableLocationKey), - "locationsRemoved.contains(tableLocationKey)"); + // When adding a location in a transaction, check for logical consistency. + // 1. If the location was already added in this transaction, we have a problem. A transaction should not + // add the same location twice. + // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a + // logical error (although it may not be supported by all consumers). + if (accumulatedLocationsAdded.containsKey(tableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + tableLocationKey + + " was added twice in the same transaction."); + } locationsAdded.add(tableLocationKey); } } @@ -200,16 +214,33 @@ public void handleTableLocationKeyRemoved( @NotNull final ImmutableTableLocationKey tableLocationKey, final Object transactionToken) { synchronized (updateLock) { + if (transactionToken == null) { + // If we're not in a transaction, modify the pending locations directly. + if (pendingLocationsRemoved.remove(tableLocationKey)) { + return; + } + pendingLocationsAdded.add(tableLocationKey); + return; + } + if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { accumulatedLocationsRemoved.put(transactionToken, new HashSet<>()); } final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken); final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken); - // A single transaction should never add and remove the same location, - Require.eqFalse(locationsAdded.contains(tableLocationKey), - "locationsAdded.contains(tableLocationKey)"); - + // When removing a location in a transaction, check for logical consistency. + // 1. If the location was already removed in this transaction, we have a problem. A transaction should not + // remove the same location twice. + // 2. If the location was already added in this transaction, we have a problem. A transaction should not + // add then remove the same location. + if (accumulatedLocationsRemoved.containsKey(tableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + tableLocationKey + + " was removed twice in the same transaction."); + } else if (accumulatedLocationsAdded.containsKey(tableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + tableLocationKey + + " was removed after being added in the same transaction."); + } locationsRemoved.add(tableLocationKey); } } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java index 09687f81f1c..6d742706c2f 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java @@ -43,7 +43,7 @@ private void processPending(@NotNull final Stream tableStream) { .withAttributes(Map.of(LOCATION_ID_ATTR, nextId.getAndIncrement()))) .peek(table -> Assert.assertion(table.isAppendOnly(), "table is append only")) .map(TableBackedTableLocationKey::new) - .forEach(this::handleTableLocationKey); + .forEach(this::handleTableLocationKeyAdded); } public synchronized void addPending(@NotNull final Table toAdd) { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java index e6958dc63b3..58d7d83b7be 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java @@ -112,7 +112,7 @@ private void refreshSnapshot() { final Set missedKeys = new HashSet<>(getTableLocationKeys()); locationKeyFinder.findKeys(tableLocationKey -> { missedKeys.remove(tableLocationKey); - handleTableLocationKey(tableLocationKey); + handleTableLocationKeyAdded(tableLocationKey); }); missedKeys.forEach(this::handleTableLocationKeyRemoved); endTransaction(); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index 4a93d985af0..163952571a2 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -43,7 +43,7 @@ public String getImplementationName() { @Override public void refresh() { beginTransaction(); - locationKeyFinder.findKeys(this::handleTableLocationKey); + locationKeyFinder.findKeys(this::handleTableLocationKeyAdded); endTransaction(); setInitialized(); } From 5bbdeb215058b1bb5609ac7a84cdcfe79a6b18a3 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 26 Jul 2024 20:38:00 -0700 Subject: [PATCH 07/56] WIP, even more PR comments addressed. --- .../impl/CompositeTableDataService.java | 4 +- .../impl/TableLocationSubscriptionBuffer.java | 84 ++++++++---- .../util/ExecutorTableDataRefreshService.java | 22 +++- .../util/TableDataRefreshService.java | 18 +++ ...ebergAutoRefreshTableLocationProvider.java | 120 ++++++++++++++++++ ...rgManualRefreshTableLocationProvider.java} | 55 ++++---- .../IcebergStaticTableLocationProvider.java | 25 +++- .../IcebergTableLocationProviderBase.java | 33 +---- .../iceberg/util/IcebergCatalogAdapter.java | 50 +++----- .../iceberg/util/IcebergInstructions.java | 19 ++- ...eRefreshing.java => IcebergTableImpl.java} | 13 +- .../iceberg/util/IcebergTableStatic.java | 49 ------- 12 files changed, 306 insertions(+), 186 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java rename extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/{IcebergRefreshingTableLocationProvider.java => IcebergManualRefreshTableLocationProvider.java} (70%) rename extensions/iceberg/src/main/java/io/deephaven/iceberg/util/{IcebergTableRefreshing.java => IcebergTableImpl.java} (79%) delete mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index a9e782496ca..3abcbf623e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -115,7 +115,9 @@ public void subscribe(@NotNull final Listener listener) { p.subscribe(listener); } else { p.refresh(); - p.getTableLocationKeys().forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, this)); + listener.beginTransaction(p); + p.getTableLocationKeys().forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, p)); + listener.endTransaction(p); } }); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index 29e654a3633..b9a455ce7af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -76,8 +76,9 @@ public synchronized LocationUpdate processPending() { // NB: Providers that don't support subscriptions don't tick - this single call to run is // sufficient. tableLocationProvider.refresh(); + // TODO: cast this to AbstractTableLocationProvider and call begin/endTransaction? tableLocationProvider.getTableLocationKeys() - .forEach(tlk -> handleTableLocationKeyAdded(tlk, tableLocationProvider)); + .forEach(tlk -> handleTableLocationKeyAdded(tlk, null)); } subscribed = true; } @@ -146,28 +147,39 @@ public void endTransaction(@NotNull final Object token) { final Set tokenLocationsAdded = accumulatedLocationsAdded.get(token); final Set tokenLocationsRemoved = accumulatedLocationsRemoved.get(token); - if (pendingLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { - // Handle any locations that were pending as adds but removed by this transaction. + if (tokenLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsRemoved) { + // If we have a pending add that is removed by this transaction, we can remove it from the pending + // list because it is cancelled by this remove. This also covers the case where a `replace` + // operation has occurred in a previous transaction. if (pendingLocationsAdded.remove(tableLocationKey)) { continue; } + // Verify that we don't have stacked removes (without intervening adds). + if (pendingLocationsRemoved.contains(tableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + tableLocationKey + + " was already removed by a previous transaction."); + } + if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsRemoved = new HashSet<>(); + } pendingLocationsRemoved.add(tableLocationKey); } - } else { - pendingLocationsRemoved = tokenLocationsRemoved; } - if (pendingLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) { - // Handle any locations that were pending as removes but added again by this transaction. + if (tokenLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) { for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsAdded) { - if (pendingLocationsRemoved.remove(tableLocationKey)) { - continue; + // Verify that we don't have stacked adds (without intervening removes). + if (pendingLocationsAdded.contains(tableLocationKey) + && !pendingLocationsRemoved.contains(tableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + tableLocationKey + + " was already added by a previous transaction."); + } + if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsAdded = new HashSet<>(); } pendingLocationsAdded.add(tableLocationKey); } - } else { - pendingLocationsAdded = tokenLocationsAdded; } // Clear all the storage for this transaction. @@ -182,10 +194,22 @@ public void handleTableLocationKeyAdded( @NotNull final ImmutableTableLocationKey tableLocationKey, @Nullable Object transactionToken) { synchronized (updateLock) { + // When adding a location in a transaction, check for logical consistency. + // 1. If the location was already added in this transaction, we have a problem. A transaction should not + // add the same location twice. + // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a + // logical error (although it may not be supported by all consumers). + if (transactionToken == null) { // If we're not in a transaction, modify the pending locations directly. - if (pendingLocationsRemoved.remove(tableLocationKey)) { - return; + // Need to verify that we don't have stacked adds (without intervening removes). + if (pendingLocationsAdded.contains(tableLocationKey) + && !pendingLocationsRemoved.contains(tableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + tableLocationKey + + " was already added by a previous transaction."); + } + if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsAdded = new HashSet<>(); } pendingLocationsAdded.add(tableLocationKey); return; @@ -196,14 +220,9 @@ public void handleTableLocationKeyAdded( } final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken); - // When adding a location in a transaction, check for logical consistency. - // 1. If the location was already added in this transaction, we have a problem. A transaction should not - // add the same location twice. - // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a - // logical error (although it may not be supported by all consumers). if (accumulatedLocationsAdded.containsKey(tableLocationKey)) { throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was added twice in the same transaction."); + + " was added multiple times in the same transaction."); } locationsAdded.add(tableLocationKey); } @@ -214,29 +233,38 @@ public void handleTableLocationKeyRemoved( @NotNull final ImmutableTableLocationKey tableLocationKey, final Object transactionToken) { synchronized (updateLock) { + // When removing a location in a transaction, check for logical consistency. + // 1. If the location was already removed in this transaction, we have a problem. A transaction should not + // remove the same location twice. + // 2. If the location was already added in this transaction, we have a problem. A transaction should not + // add then remove the same location. + if (transactionToken == null) { // If we're not in a transaction, modify the pending locations directly. - if (pendingLocationsRemoved.remove(tableLocationKey)) { + // If we have a pending add, it is being cancelled by this remove. + if (pendingLocationsAdded.remove(tableLocationKey)) { return; } - pendingLocationsAdded.add(tableLocationKey); + // Verify that we don't have stacked removes (without intervening adds). + if (pendingLocationsRemoved.contains(tableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + tableLocationKey + + " was already removed by a previous transaction."); + } + if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsRemoved = new HashSet<>(); + } + pendingLocationsRemoved.add(tableLocationKey); return; } if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { accumulatedLocationsRemoved.put(transactionToken, new HashSet<>()); } - final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken); final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken); - // When removing a location in a transaction, check for logical consistency. - // 1. If the location was already removed in this transaction, we have a problem. A transaction should not - // remove the same location twice. - // 2. If the location was already added in this transaction, we have a problem. A transaction should not - // add then remove the same location. if (accumulatedLocationsRemoved.containsKey(tableLocationKey)) { throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was removed twice in the same transaction."); + + " was removed multiple times in the same transaction."); } else if (accumulatedLocationsAdded.containsKey(tableLocationKey)) { throw new IllegalStateException("TableLocationKey " + tableLocationKey + " was removed after being added in the same transaction."); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java index 8c4e016607c..8db2b625354 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java @@ -19,7 +19,6 @@ import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; /** * {@link TableDataRefreshService} implementation that uses a {@link ScheduledThreadPoolExecutor}. @@ -76,7 +75,8 @@ private abstract class ScheduledSubscriptionTask { - private ScheduledTableLocationProviderRefresh(@NotNull AbstractTableLocationProvider tableLocationProvider) { + private ScheduledTableLocationProviderRefresh( + @NotNull final AbstractTableLocationProvider tableLocationProvider) { super(tableLocationProvider, tableLocationProviderRefreshIntervalMillis); providerSubscriptions.increment(1); } + private ScheduledTableLocationProviderRefresh( + @NotNull final AbstractTableLocationProvider tableLocationProvider, + final long refreshIntervalMillis) { + super(tableLocationProvider, refreshIntervalMillis); + providerSubscriptions.increment(1); + } + + @Override protected void refresh() { final long startTimeNanos = System.nanoTime(); @@ -156,6 +165,13 @@ public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( return new ScheduledTableLocationProviderRefresh(tableLocationProvider); } + @Override + public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( + @NotNull final AbstractTableLocationProvider tableLocationProvider, + final long refreshIntervalMillis) { + return new ScheduledTableLocationProviderRefresh(tableLocationProvider, refreshIntervalMillis); + } + @Override public CancellableSubscriptionToken scheduleTableLocationRefresh( @NotNull final AbstractTableLocation tableLocation) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java index c6b823c3139..093426d0bb6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java @@ -40,6 +40,17 @@ interface CancellableSubscriptionToken { CancellableSubscriptionToken scheduleTableLocationProviderRefresh( @NotNull AbstractTableLocationProvider tableLocationProvider); + /** + * Schedule run for an AbstractTableLocationProvider. + * + * @param tableLocationProvider The table location provider + * @param refreshIntervalMs The interval in milliseconds between refreshes + * @return A subscription token to be used for matching, which also supports cancellation + */ + CancellableSubscriptionToken scheduleTableLocationProviderRefresh( + @NotNull AbstractTableLocationProvider tableLocationProvider, + long refreshIntervalMs); + /** * Schedule run for an AbstractTableLocation. * @@ -126,6 +137,13 @@ public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( throw new UnsupportedOperationException(); } + @Override + public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( + @NotNull final AbstractTableLocationProvider tableLocationProvider, + final long refreshIntervalMs) { + throw new UnsupportedOperationException(); + } + @Override public CancellableSubscriptionToken scheduleTableLocationRefresh( @NotNull final AbstractTableLocation tableLocation) { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java new file mode 100644 index 00000000000..bc9fb230eda --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -0,0 +1,120 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.layout; + +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.TableIdentifier; +import org.jetbrains.annotations.NotNull; + +import java.util.HashSet; +import java.util.Set; + +/** + *

+ * Automatically refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location + * key} discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a + * {@link TableLocationFactory}. + *

+ */ +public class IcebergAutoRefreshTableLocationProvider + extends IcebergTableLocationProviderBase { + + private static final String IMPLEMENTATION_NAME = IcebergAutoRefreshTableLocationProvider.class.getSimpleName(); + + private final long refreshIntervalMs; + + private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken; + + public IcebergAutoRefreshTableLocationProvider( + @NotNull final TK tableKey, + @NotNull final IcebergBaseLayout locationKeyFinder, + @NotNull final TableLocationFactory locationFactory, + @NotNull final TableDataRefreshService refreshService, + final long refreshIntervalMs, + @NotNull final IcebergCatalogAdapter adapter, + @NotNull final TableIdentifier tableIdentifier) { + super(tableKey, locationKeyFinder, locationFactory, refreshService, true, adapter, tableIdentifier); + + Assert.neqNull(refreshService, "refreshService"); + this.refreshIntervalMs = refreshIntervalMs; + } + + // ------------------------------------------------------------------------------------------------------------------ + // AbstractTableLocationProvider implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + @Override + public synchronized void refresh() { + final Snapshot latestSnapshot = adapter.getCurrentSnapshot(tableIdentifier); + if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) { + locationKeyFinder.snapshot = latestSnapshot; + refreshSnapshot(); + } + } + + @Override + public void update() { + throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated manually"); + } + + @Override + public void update(long snapshotId) { + throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated"); + } + + @Override + public void update(Snapshot snapshot) { + throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated"); + } + + /** + * Refresh the table location provider with the latest snapshot from the catalog. This method will identify new + * locations and removed locations. + */ + private void refreshSnapshot() { + beginTransaction(); + final Set missedKeys = new HashSet<>(getTableLocationKeys()); + locationKeyFinder.findKeys(tableLocationKey -> { + missedKeys.remove(tableLocationKey); + handleTableLocationKeyAdded(tableLocationKey); + }); + missedKeys.forEach(this::handleTableLocationKeyRemoved); + endTransaction(); + setInitialized(); + } + + // ------------------------------------------------------------------------------------------------------------------ + // SubscriptionAggregator implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + protected final void activateUnderlyingDataSource() { + refreshSnapshot(); + subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this, refreshIntervalMs); + } + + @Override + protected final void deactivateUnderlyingDataSource() { + if (subscriptionToken != null) { + subscriptionToken.cancel(); + subscriptionToken = null; + } + } + + @Override + protected final boolean matchSubscriptionToken(final T token) { + return token == subscriptionToken; + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java similarity index 70% rename from extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java rename to extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index 58d7d83b7be..9287cfa64dc 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -6,12 +6,10 @@ import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; -import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.iceberg.util.IcebergCatalogAdapter; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import java.util.HashSet; import java.util.List; @@ -19,33 +17,23 @@ /** *

- * Refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} + * Manually refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} * discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a * {@link TableLocationFactory}. *

- *

- * Supports both automatic and manual refreshing cases, distinguished by the {@code autoRefresh} parameter. - *

*/ -public class IcebergRefreshingTableLocationProvider +public class IcebergManualRefreshTableLocationProvider extends IcebergTableLocationProviderBase { - private static final String IMPLEMENTATION_NAME = IcebergRefreshingTableLocationProvider.class.getSimpleName(); - - private final boolean autoRefresh; + private static final String IMPLEMENTATION_NAME = IcebergManualRefreshTableLocationProvider.class.getSimpleName(); - private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken; - - public IcebergRefreshingTableLocationProvider( + public IcebergManualRefreshTableLocationProvider( @NotNull final TK tableKey, @NotNull final IcebergBaseLayout locationKeyFinder, @NotNull final TableLocationFactory locationFactory, - @Nullable final TableDataRefreshService refreshService, @NotNull final IcebergCatalogAdapter adapter, - @NotNull final TableIdentifier tableIdentifier, - final boolean autoRefresh) { - super(tableKey, locationKeyFinder, locationFactory, refreshService, adapter, tableIdentifier); - this.autoRefresh = autoRefresh; + @NotNull final TableIdentifier tableIdentifier) { + super(tableKey, locationKeyFinder, locationFactory, null, true, adapter, tableIdentifier); } // ------------------------------------------------------------------------------------------------------------------ @@ -59,13 +47,8 @@ public String getImplementationName() { @Override public synchronized void refresh() { - if (autoRefresh) { - final Snapshot latestSnapshot = adapter.getCurrentSnapshot(tableIdentifier); - if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) { - locationKeyFinder.snapshot = latestSnapshot; - } - } - refreshSnapshot(); + // There should be no refresh service for this provider. + throw new UnsupportedOperationException(); } @Override @@ -95,7 +78,7 @@ public synchronized void update(final Snapshot snapshot) { if (snapshot.sequenceNumber() <= locationKeyFinder.snapshot.sequenceNumber()) { throw new IllegalArgumentException( "Snapshot sequence number " + snapshot.sequenceNumber() - + " is older than the current snapshot sequence number " + + " must be higher than the current snapshot sequence number " + locationKeyFinder.snapshot.sequenceNumber() + " for table " + tableIdentifier); } // Update the snapshot. @@ -118,4 +101,24 @@ private void refreshSnapshot() { endTransaction(); setInitialized(); } + + // ------------------------------------------------------------------------------------------------------------------ + // SubscriptionAggregator implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + protected void activateUnderlyingDataSource() { + refreshSnapshot(); + activationSuccessful(this); + } + + @Override + protected void deactivateUnderlyingDataSource() { + // NOP for manually refreshing Iceberg table location provider. + } + + @Override + protected boolean matchSubscriptionToken(T token) { + return token == this; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index 163952571a2..92798252dad 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -6,7 +6,6 @@ import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; -import io.deephaven.iceberg.util.IcebergCatalogAdapter; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; @@ -26,9 +25,8 @@ public IcebergStaticTableLocationProvider( @NotNull final TK tableKey, @NotNull final IcebergBaseLayout locationKeyFinder, @NotNull final TableLocationFactory locationFactory, - @NotNull final IcebergCatalogAdapter adapter, @NotNull final TableIdentifier tableIdentifier) { - super(tableKey, locationKeyFinder, locationFactory, null, adapter, tableIdentifier); + super(tableKey, locationKeyFinder, locationFactory, null, false, null, tableIdentifier); } // ------------------------------------------------------------------------------------------------------------------ @@ -62,4 +60,25 @@ public void update(long snapshotId) { public void update(Snapshot snapshot) { throw new IllegalStateException("A static table location provider cannot be updated"); } + + // ------------------------------------------------------------------------------------------------------------------ + // SubscriptionAggregator implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + protected void activateUnderlyingDataSource() { + throw new IllegalStateException( + "activateUnderlyingDataSource() called on a static Iceberg table location provider"); + } + + @Override + protected void deactivateUnderlyingDataSource() { + throw new IllegalStateException( + "deactivateUnderlyingDataSource() called on a static Iceberg table location provider"); + } + + @Override + protected boolean matchSubscriptionToken(final T token) { + return false; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java index 52c98718317..9e6577d500b 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java @@ -22,16 +22,15 @@ public abstract class IcebergTableLocationProviderBase locationFactory, @Nullable final TableDataRefreshService refreshService, - @NotNull final IcebergCatalogAdapter adapter, + final boolean isRefreshing, + @Nullable final IcebergCatalogAdapter adapter, @NotNull final TableIdentifier tableIdentifier) { - super(tableKey, refreshService != null); + super(tableKey, isRefreshing); this.locationKeyFinder = locationKeyFinder; this.locationFactory = locationFactory; this.refreshService = refreshService; @@ -39,10 +38,6 @@ public IcebergTableLocationProviderBase( this.tableIdentifier = tableIdentifier; } - // ------------------------------------------------------------------------------------------------------------------ - // AbstractTableLocationProvider implementation - // ------------------------------------------------------------------------------------------------------------------ - /** * Update the table location provider with the latest snapshot from the catalog. */ @@ -72,26 +67,4 @@ protected TableLocation makeTableLocation(@NotNull final TableLocationKey locati // noinspection unchecked return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService); } - - // ------------------------------------------------------------------------------------------------------------------ - // SubscriptionAggregator implementation - // ------------------------------------------------------------------------------------------------------------------ - - @Override - protected final void activateUnderlyingDataSource() { - subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this); - } - - @Override - protected final void deactivateUnderlyingDataSource() { - if (subscriptionToken != null) { - subscriptionToken.cancel(); - subscriptionToken = null; - } - } - - @Override - protected final boolean matchSubscriptionToken(final T token) { - return token == subscriptionToken; - } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index bf23dbd6e85..e96cd2f8b43 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -11,7 +11,6 @@ import io.deephaven.engine.table.impl.locations.TableKey; import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; -import io.deephaven.engine.table.impl.locations.util.ExecutorTableDataRefreshService; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; @@ -370,6 +369,9 @@ Optional getSnapshot(@NotNull final TableIdentifier tableIdentifier, f */ public Snapshot getCurrentSnapshot(@NotNull final TableIdentifier tableIdentifier) { final List snapshots = listSnapshots(tableIdentifier); + if (snapshots.isEmpty()) { + throw new IllegalStateException("No snapshots found for table " + tableIdentifier); + } return snapshots.get(snapshots.size() - 1); } @@ -517,53 +519,44 @@ private IcebergTable readTableInternal( userInstructions); } - if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.STATIC) { + if (instructions.updateMode() == IcebergInstructions.IcebergUpdateMode.STATIC) { final IcebergTableLocationProviderBase locationProvider = new IcebergStaticTableLocationProvider<>( StandaloneTableKey.getInstance(), keyFinder, new IcebergTableLocationFactory(), - this, tableIdentifier); - return new IcebergTableStatic( + return new IcebergTableImpl( tableDef, tableIdentifier.toString(), RegionedTableComponentFactoryImpl.INSTANCE, - locationProvider); + locationProvider, + null); } - final UpdateSourceRegistrar updateSourceRegistrar; - final IcebergRefreshingTableLocationProvider locationProvider; - - if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) { - updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + final IcebergTableLocationProviderBase locationProvider; - locationProvider = new IcebergRefreshingTableLocationProvider<>( + if (instructions.updateMode() == IcebergInstructions.IcebergUpdateMode.MANUAL_REFRESHING) { + locationProvider = new IcebergManualRefreshTableLocationProvider<>( StandaloneTableKey.getInstance(), keyFinder, new IcebergTableLocationFactory(), - TableDataRefreshService.getSharedRefreshService(), this, - tableIdentifier, - false); + tableIdentifier); } else { - updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); - - final TableDataRefreshService refreshService = - new ExecutorTableDataRefreshService("Local", instructions.autoRefreshMs(), 30_000L, 10); - - locationProvider = new IcebergRefreshingTableLocationProvider<>( + locationProvider = new IcebergAutoRefreshTableLocationProvider<>( StandaloneTableKey.getInstance(), keyFinder, new IcebergTableLocationFactory(), - refreshService, + TableDataRefreshService.getSharedRefreshService(), + instructions.autoRefreshMs(), this, - tableIdentifier, - true); + tableIdentifier); } - return new IcebergTableRefreshing( + return new IcebergTableImpl( tableDef, tableIdentifier.toString(), RegionedTableComponentFactoryImpl.INSTANCE, @@ -583,13 +576,4 @@ private static KnownLocationKeyFinder toKnownKeys( public Catalog catalog() { return catalog; } - - /** - * Returns the underlying Iceberg {@link FileIO fileIO} used by this adapter. - */ - @SuppressWarnings("unused") - public FileIO fileIO() { - return fileIO; - } - } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index dc7515d7498..462ca838d2e 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -26,7 +26,7 @@ public abstract class IcebergInstructions { public static final IcebergInstructions DEFAULT = builder().build(); @SuppressWarnings("unused") - public enum IcebergRefreshing { + public enum IcebergUpdateMode { STATIC, AUTO_REFRESHING, MANUAL_REFRESHING } @@ -52,16 +52,16 @@ public static Builder builder() { public abstract Map columnRenames(); /** - * The {@link IcebergRefreshing} mode to use when reading the Iceberg data files. Default is - * {@link IcebergRefreshing#STATIC}. + * The {@link IcebergUpdateMode} mode to use when reading the Iceberg data files. Default is + * {@link IcebergUpdateMode#STATIC}. */ @Value.Default - public IcebergRefreshing refreshing() { - return IcebergRefreshing.STATIC; + public IcebergUpdateMode updateMode() { + return IcebergUpdateMode.STATIC; } /** - * When {@link #refreshing()} is set to {@code IcebergRefreshing.AUTO_REFRESHING}, specifies the number of + * When {@link #updateMode()} is set to {@code IcebergUpdateMode.AUTO_REFRESHING}, specifies the number of * milliseconds to wait before refreshing the Iceberg data files. Default is 60_000 milliseconds. */ @Value.Default @@ -83,7 +83,12 @@ public interface Builder { Builder putAllColumnRenames(Map entries); @SuppressWarnings("unused") - Builder refreshing(IcebergRefreshing refreshing); + Builder updateMode(IcebergUpdateMode refreshing); + + @SuppressWarnings("unused") + default Builder updateMode(IcebergUpdateMode updateMode, long autoRefreshMs) { + return this.updateMode(updateMode).autoRefreshMs(autoRefreshMs); + } @SuppressWarnings("unused") Builder autoRefreshMs(long autoRefreshMs); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java similarity index 79% rename from extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java rename to extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java index 72623b9e200..a73a55d40be 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java @@ -15,24 +15,25 @@ import org.jetbrains.annotations.Nullable; /** - * Table wrapper for refreshing Iceberg tables. + * Table implementation for static and refreshing Iceberg tables. */ -public class IcebergTableRefreshing extends PartitionAwareSourceTable implements IcebergTable { +public class IcebergTableImpl extends PartitionAwareSourceTable implements IcebergTable { /** * Location discovery. */ final IcebergTableLocationProviderBase locationProvider; /** + * Create an instance of the class with the provided parameters. * - * - * @param tableDefinition A TableDefinition + * @param tableDefinition The {@link TableDefinition} describing the table schema * @param description A human-readable description for this table * @param componentFactory A component factory for creating column source managers - * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table + * @param locationProvider A {@link io.deephaven.engine.table.impl.locations.TableLocationProvider}, for use in + * discovering the locations that compose this table * @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live */ - IcebergTableRefreshing( + IcebergTableImpl( @NotNull TableDefinition tableDefinition, @NotNull String description, @NotNull SourceTableComponentFactory componentFactory, diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java deleted file mode 100644 index 0884a6379c5..00000000000 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java +++ /dev/null @@ -1,49 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.iceberg.util; - -import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.impl.PartitionAwareSourceTable; -import io.deephaven.engine.table.impl.SourceTableComponentFactory; -import io.deephaven.engine.table.impl.locations.TableKey; -import io.deephaven.iceberg.layout.IcebergTableLocationProviderBase; -import io.deephaven.iceberg.location.IcebergTableLocationKey; -import org.apache.iceberg.Snapshot; -import org.jetbrains.annotations.NotNull; - -/** - * Table wrapper for refreshing Iceberg tables. - */ -public class IcebergTableStatic extends PartitionAwareSourceTable implements IcebergTable { - /** - * - * - * @param tableDefinition A TableDefinition - * @param description A human-readable description for this table - * @param componentFactory A component factory for creating column source managers - * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table - */ - IcebergTableStatic( - @NotNull TableDefinition tableDefinition, - @NotNull String description, - @NotNull SourceTableComponentFactory componentFactory, - @NotNull IcebergTableLocationProviderBase locationProvider) { - super(tableDefinition, description, componentFactory, locationProvider, null); - } - - @Override - public void update() { - throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description); - } - - @Override - public void update(final long snapshotId) { - throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description); - } - - @Override - public void update(final @NotNull Snapshot snapshot) { - throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description); - } -} From 3da205ccee7d2e8a2004e1ef05d9fffcbf6fd3fb Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Sat, 27 Jul 2024 13:25:37 -0700 Subject: [PATCH 08/56] Nearly all PR comments addressed. --- .../impl/locations/TableLocationProvider.java | 12 ++++++- .../impl/TableLocationSubscriptionBuffer.java | 26 +++++++------- .../regioned/RegionedColumnSourceManager.java | 35 +++++++++++-------- ...ebergAutoRefreshTableLocationProvider.java | 6 ++-- 4 files changed, 48 insertions(+), 31 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index 9ba7a350d81..eab5f6ad72f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -56,10 +56,14 @@ default void endTransaction() { } /** + *

* Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location * subscription. This should occur at most once per location, but the order of delivery is not * guaranteed. - * + *

+ *

+ * If transactionToken is {@code null}, the key will be added to the pending additions immediately. + *

* @param tableLocationKey The new table location key. * @param transactionToken The token identifying the transaction. */ @@ -79,7 +83,12 @@ default void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tabl } /** + *

* Notify the listener of a {@link TableLocationKey} that has been removed. + *

+ *

+ * If transactionToken is {@code null}, the key will be added to the pending removals immediately. + *

* * @param tableLocationKey The table location key that was removed. * @param transactionToken The token identifying the transaction. @@ -93,6 +102,7 @@ void handleTableLocationKeyRemoved( * * @param tableLocationKey The table location key that was removed. */ + @SuppressWarnings("unused") default void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey) { handleTableLocationKeyRemoved(tableLocationKey, this); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index b9a455ce7af..1962ae39ce1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.locations.impl; +import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableDataException; @@ -125,11 +126,13 @@ public synchronized void reset() { @Override public void beginTransaction(@NotNull final Object token) { synchronized (updateLock) { - // Assert that we can start a new transaction with this token. - Require.eqFalse(transactionTokens.contains(token), "transactionTokens.contains(token)"); - Require.eqFalse(accumulatedLocationsAdded.containsKey(token), + // Verify that we can start a new transaction with this token. + if (transactionTokens.contains(token)) { + throw new IllegalStateException("A transaction with token " + token + " is currently open."); + } + Assert.eqFalse(accumulatedLocationsAdded.containsKey(token), "accumulatedLocationsAdded.containsKey(token)"); - Require.eqFalse(accumulatedLocationsRemoved.containsKey(token), + Assert.eqFalse(accumulatedLocationsRemoved.containsKey(token), "accumulatedLocationsRemoved.containsKey(token)"); transactionTokens.add(token); @@ -141,11 +144,13 @@ public void beginTransaction(@NotNull final Object token) { @Override public void endTransaction(@NotNull final Object token) { synchronized (updateLock) { - // Assert that this transaction is open. - Require.eqTrue(transactionTokens.contains(token), "transactionTokens.contains(token)"); + // Verify that this transaction is open. + if (!transactionTokens.remove(token)) { + throw new IllegalStateException("No transaction with token " + token + " is currently open."); + } - final Set tokenLocationsAdded = accumulatedLocationsAdded.get(token); - final Set tokenLocationsRemoved = accumulatedLocationsRemoved.get(token); + final Set tokenLocationsAdded = accumulatedLocationsAdded.remove(token); + final Set tokenLocationsRemoved = accumulatedLocationsRemoved.remove(token); if (tokenLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsRemoved) { @@ -181,11 +186,6 @@ public void endTransaction(@NotNull final Object token) { pendingLocationsAdded.add(tableLocationKey); } } - - // Clear all the storage for this transaction. - transactionTokens.remove(token); - accumulatedLocationsAdded.remove(token); - accumulatedLocationsRemoved.remove(token); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 0bcb4e66724..5743c08f4b6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -121,6 +121,9 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col @ReferentialIntegrity private Runnable delayedErrorReference; + final List invalidatedLocations; + final UpdateCommitter invalidateCommitter; + /** * Construct a column manager with the specified component factory and definitions. * @@ -179,6 +182,14 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col rowSetModifiedColumnSet = null; } } + + invalidatedLocations = new ArrayList<>(); + invalidateCommitter = new UpdateCommitter<>(this, + ExecutionContext.getContext().getUpdateGraph(), + (ignored) -> { + invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate); + invalidatedLocations.clear(); + }); } @Override @@ -223,11 +234,9 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati orderedIncludedTableLocations.remove(includedLocation); removedTableLocations.add(includedLocation); - final UpdateCommitter committer = new UpdateCommitter<>(this, - ExecutionContext.getContext().getUpdateGraph(), - (ignored) -> includedLocation.invalidate()); - - committer.maybeActivate(); + // Mark this location for invalidation. + invalidatedLocations.add(includedLocation); + invalidateCommitter.maybeActivate(); return true; } @@ -356,13 +365,13 @@ private TableUpdateImpl update(final boolean initializing) { final RowSetBuilderSequential addedRegionBuilder = RowSetFactory.builderSequential(); - final int previousNumRegions = includedTableLocations.size(); - final int newNumRegions = previousNumRegions + (entriesToInclude == null ? 0 : entriesToInclude.size()); + final int prevMaxIndex = nextRegionIndex.get(); + final int maxIndex = nextRegionIndex.get() + (entriesToInclude == null ? 0 : entriesToInclude.size()); if (entriesToInclude != null) { partitioningColumnValueSources.values().forEach( - (final WritableColumnSource wcs) -> wcs.ensureCapacity(newNumRegions)); - locationSource.ensureCapacity(newNumRegions); - rowSetSource.ensureCapacity(newNumRegions); + (final WritableColumnSource wcs) -> wcs.ensureCapacity(maxIndex)); + locationSource.ensureCapacity(maxIndex); + rowSetSource.ensureCapacity(maxIndex); for (final EmptyTableLocationEntry entryToInclude : entriesToInclude) { final IncludedTableLocationEntry entry = new IncludedTableLocationEntry(entryToInclude); @@ -389,7 +398,7 @@ private TableUpdateImpl update(final boolean initializing) { } if (initializing) { - Assert.eqZero(previousNumRegions, "previousNumRegions"); + Assert.eqZero(prevMaxIndex, "previousNumRegions"); if (isRefreshing) { rowSetSource.startTrackingPrevValues(); includedLocationsTable.getRowSet().writableCast().initializePreviousValue(); @@ -401,9 +410,7 @@ private TableUpdateImpl update(final boolean initializing) { final RowSet modifiedRegions = modifiedRegionBuilder.build(); final RowSet removedRegions = removedRegionBuilder.build(); if (addedRegions.isEmpty() && modifiedRegions.isEmpty() && removedRegions.isEmpty()) { - addedRegions.close(); - modifiedRegions.close(); - removedRegions.close(); + SafeCloseable.closeAll(addedRegions, modifiedRegions, removedRegions); } else { includedLocationsTable.getRowSet().writableCast().remove(removedRegions); final TableUpdate update = new TableUpdateImpl( diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index bc9fb230eda..ad7212e1c3f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -66,17 +66,17 @@ public synchronized void refresh() { @Override public void update() { - throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated manually"); + throw new IllegalStateException("An automatically refreshing Iceberg table cannot be manually updated"); } @Override public void update(long snapshotId) { - throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated"); + throw new IllegalStateException("An automatically refreshing Iceberg table cannot be manually updated"); } @Override public void update(Snapshot snapshot) { - throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated"); + throw new IllegalStateException("An automatically refreshing Iceberg table cannot be manually updated"); } /** From 08dd329dbfbb2fb5b7b3c307fa21c0397ab12040 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 29 Jul 2024 12:07:50 -0700 Subject: [PATCH 09/56] Adjustment to IcebergInstructions update mode. --- .../impl/locations/TableLocationProvider.java | 1 + ...ergManualRefreshTableLocationProvider.java | 6 +- .../iceberg/util/IcebergCatalogAdapter.java | 7 ++- .../iceberg/util/IcebergInstructions.java | 22 -------- .../iceberg/util/IcebergUpdateMode.java | 56 +++++++++++++++++++ 5 files changed, 64 insertions(+), 28 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index eab5f6ad72f..d6158090289 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -64,6 +64,7 @@ default void endTransaction() { *

* If transactionToken is {@code null}, the key will be added to the pending additions immediately. *

+ * * @param tableLocationKey The new table location key. * @param transactionToken The token identifying the transaction. */ diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index 9287cfa64dc..bfa5e99e60d 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -77,9 +77,9 @@ public synchronized void update(final Snapshot snapshot) { // Verify that the input snapshot is newer (higher in sequence number) than the current snapshot. if (snapshot.sequenceNumber() <= locationKeyFinder.snapshot.sequenceNumber()) { throw new IllegalArgumentException( - "Snapshot sequence number " + snapshot.sequenceNumber() - + " must be higher than the current snapshot sequence number " - + locationKeyFinder.snapshot.sequenceNumber() + " for table " + tableIdentifier); + "Update snapshot sequence number (" + snapshot.sequenceNumber() + + ") must be higher than the current snapshot sequence number (" + + locationKeyFinder.snapshot.sequenceNumber() + ") for table " + tableIdentifier); } // Update the snapshot. locationKeyFinder.snapshot = snapshot; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index 75b602813e4..8ba56e5cc70 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -550,7 +550,8 @@ private IcebergTable readTableInternal( userInstructions); } - if (instructions.updateMode() == IcebergInstructions.IcebergUpdateMode.STATIC) { + if (instructions == null + || instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) { final IcebergTableLocationProviderBase locationProvider = new IcebergStaticTableLocationProvider<>( StandaloneTableKey.getInstance(), @@ -569,7 +570,7 @@ private IcebergTable readTableInternal( final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); final IcebergTableLocationProviderBase locationProvider; - if (instructions.updateMode() == IcebergInstructions.IcebergUpdateMode.MANUAL_REFRESHING) { + if (instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) { locationProvider = new IcebergManualRefreshTableLocationProvider<>( StandaloneTableKey.getInstance(), keyFinder, @@ -582,7 +583,7 @@ private IcebergTable readTableInternal( keyFinder, new IcebergTableLocationFactory(), TableDataRefreshService.getSharedRefreshService(), - instructions.autoRefreshMs(), + instructions.updateMode().autoRefreshMs(), this, tableIdentifier); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 462ca838d2e..5b6438475b2 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -25,11 +25,6 @@ public abstract class IcebergInstructions { @SuppressWarnings("unused") public static final IcebergInstructions DEFAULT = builder().build(); - @SuppressWarnings("unused") - public enum IcebergUpdateMode { - STATIC, AUTO_REFRESHING, MANUAL_REFRESHING - } - public static Builder builder() { return ImmutableIcebergInstructions.builder(); } @@ -60,15 +55,6 @@ public IcebergUpdateMode updateMode() { return IcebergUpdateMode.STATIC; } - /** - * When {@link #updateMode()} is set to {@code IcebergUpdateMode.AUTO_REFRESHING}, specifies the number of - * milliseconds to wait before refreshing the Iceberg data files. Default is 60_000 milliseconds. - */ - @Value.Default - public long autoRefreshMs() { - return 60_000L; // 60 second default - } - public interface Builder { @SuppressWarnings("unused") Builder tableDefinition(TableDefinition tableDefinition); @@ -85,14 +71,6 @@ public interface Builder { @SuppressWarnings("unused") Builder updateMode(IcebergUpdateMode refreshing); - @SuppressWarnings("unused") - default Builder updateMode(IcebergUpdateMode updateMode, long autoRefreshMs) { - return this.updateMode(updateMode).autoRefreshMs(autoRefreshMs); - } - - @SuppressWarnings("unused") - Builder autoRefreshMs(long autoRefreshMs); - IcebergInstructions build(); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java new file mode 100644 index 00000000000..32f67c742ec --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java @@ -0,0 +1,56 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value; + +@Value.Immutable +@BuildableStyle +public abstract class IcebergUpdateMode { + public static final IcebergUpdateMode STATIC = builder() + .updateType(IcebergUpdateType.STATIC).build(); + @SuppressWarnings("unused") + public static final IcebergUpdateMode AUTO_REFRESHING = builder() + .updateType(IcebergUpdateType.AUTO_REFRESHING).build(); + @SuppressWarnings("unused") + public static final IcebergUpdateMode MANUAL_REFRESHING = builder() + .updateType(IcebergUpdateType.MANUAL_REFRESHING).build(); + + public enum IcebergUpdateType { + STATIC, AUTO_REFRESHING, MANUAL_REFRESHING + } + + public static Builder builder() { + return ImmutableIcebergUpdateMode.builder(); + } + + @SuppressWarnings("unused") + public static IcebergUpdateMode autoRefreshing(final long refreshMs) { + return ImmutableIcebergUpdateMode.builder() + .updateType(IcebergUpdateType.AUTO_REFRESHING) + .autoRefreshMs(refreshMs) + .build(); + } + + @Value.Default + public IcebergUpdateType updateType() { + return IcebergUpdateType.STATIC; + } + + @Value.Default + public long autoRefreshMs() { + return 60_000L; + } + + public interface Builder { + @SuppressWarnings("unused") + Builder updateType(IcebergUpdateType updateType); + + @SuppressWarnings("unused") + Builder autoRefreshMs(long autoRefreshMs); + + IcebergUpdateMode build(); + } +} From 7af0d1da546a6c10ee2bb3c6f22b1910c5bb2fad Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 29 Jul 2024 14:40:20 -0700 Subject: [PATCH 10/56] Added python wrapper for Iceberg refreshing tables. --- .../iceberg/util/IcebergUpdateMode.java | 15 ++-- py/server/deephaven/experimental/iceberg.py | 80 ++++++++++++++++++- 2 files changed, 84 insertions(+), 11 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java index 32f67c742ec..246e81ac65e 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java @@ -9,14 +9,15 @@ @Value.Immutable @BuildableStyle public abstract class IcebergUpdateMode { - public static final IcebergUpdateMode STATIC = builder() - .updateType(IcebergUpdateType.STATIC).build(); + private static final long REFRESH_INTERVAL_MS = 60_000L; + + public static final IcebergUpdateMode STATIC = builder().updateType(IcebergUpdateType.STATIC).build(); @SuppressWarnings("unused") - public static final IcebergUpdateMode AUTO_REFRESHING = builder() - .updateType(IcebergUpdateType.AUTO_REFRESHING).build(); + public static final IcebergUpdateMode AUTO_REFRESHING = + builder().updateType(IcebergUpdateType.AUTO_REFRESHING).build(); @SuppressWarnings("unused") - public static final IcebergUpdateMode MANUAL_REFRESHING = builder() - .updateType(IcebergUpdateType.MANUAL_REFRESHING).build(); + public static final IcebergUpdateMode MANUAL_REFRESHING = + builder().updateType(IcebergUpdateType.MANUAL_REFRESHING).build(); public enum IcebergUpdateType { STATIC, AUTO_REFRESHING, MANUAL_REFRESHING @@ -41,7 +42,7 @@ public IcebergUpdateType updateType() { @Value.Default public long autoRefreshMs() { - return 60_000L; + return REFRESH_INTERVAL_MS; } public interface Builder { diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 7506bc95a25..42e2c8292bf 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -2,6 +2,7 @@ # Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending # """ This module adds Iceberg table support into Deephaven. """ +from __future__ import annotations from typing import List, Optional, Union, Dict, Sequence import jpy @@ -17,7 +18,9 @@ from deephaven.table import Table _JIcebergInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergInstructions") +_JIcebergUpdateMode = jpy.get_type("io.deephaven.iceberg.util.IcebergUpdateMode") _JIcebergCatalogAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergCatalogAdapter") +_JIcebergTable = jpy.get_type("io.deephaven.iceberg.util.IcebergTable") # IcebergToolsS3 is an optional library try: @@ -30,6 +33,39 @@ _JSnapshot = jpy.get_type("org.apache.iceberg.Snapshot") +class IcebergUpdateMode(JObjectWrapper): + """ + This class specifies the update mode for the Iceberg table to be loaded into Deephaven. The modes are: + + - `STATIC`: The table is loaded once and does not change + - `MANUAL_REFRESHING`: The table can be manually refreshed by the user. + - `AUTO_REFRESHING`: The table will be automatically refreshed at a specified interval (use + `auto_refreshing(auto_refresh_ms: int)` to specify an interval rather than use the system default + of 60 seconds). + """ + j_object_type = _JIcebergUpdateMode + + def __init__(self, mode: _JIcebergUpdateMode): + self._j_object = mode + + @classmethod + def auto_refreshing(cls, auto_refresh_ms: int) -> IcebergUpdateMode: + """ + Creates an IcebergUpdateMode with auto-refreshing mode enabled using the provided refresh interval. + + :param auto_refresh_ms (int): the refresh interval in milliseconds. + """ + return IcebergUpdateMode(_JIcebergUpdateMode.autoRefreshing(auto_refresh_ms)) + + @property + def j_object(self) -> jpy.JType: + return self._j_object + +IcebergUpdateMode.STATIC=IcebergUpdateMode(_JIcebergUpdateMode.STATIC) +IcebergUpdateMode.MANUAL_REFRESHING=IcebergUpdateMode(_JIcebergUpdateMode.MANUAL_REFRESHING) +IcebergUpdateMode.AUTO_REFRESHING=IcebergUpdateMode(_JIcebergUpdateMode.AUTO_REFRESHING) + + class IcebergInstructions(JObjectWrapper): """ This class specifies the instructions for reading an Iceberg table into Deephaven. These include column rename @@ -41,7 +77,8 @@ class IcebergInstructions(JObjectWrapper): def __init__(self, table_definition: Optional[Union[Dict[str, DType], List[Column]]] = None, data_instructions: Optional[s3.S3Instructions] = None, - column_renames: Optional[Dict[str, str]] = None): + column_renames: Optional[Dict[str, str]] = None, + update_mode: Optional[IcebergUpdateMode] = None): """ Initializes the instructions using the provided parameters. @@ -53,6 +90,8 @@ def __init__(self, reading files from a non-local file system, like S3. column_renames (Optional[Dict[str, str]]): A dictionary of old to new column names that will be renamed in the output table. + update_mode (Optional[IcebergUpdateMode]): The update mode for the table. If omitted, the default update + mode of `IcebergUpdateMode.STATIC` is used. Raises: DHError: If unable to build the instructions object. @@ -71,6 +110,9 @@ def __init__(self, for old_name, new_name in column_renames.items(): builder.putColumnRenames(old_name, new_name) + if update_mode is not None: + builder.updateMode(update_mode.j_object) + self._j_object = builder.build() except Exception as e: raise DHError(e, "Failed to build Iceberg instructions") from e @@ -80,6 +122,36 @@ def j_object(self) -> jpy.JType: return self._j_object +class IcebergTable(Table): + """ + IcebergTable is a subclass of Table that allows the users to dynamically update the table with new snapshots from + the Iceberg catalog. + """ + j_object_type = _JIcebergTable + + def __init__(self, j_table: jpy.JType): + super().__init__(j_table) + + def update(self, snapshot_id:Optional[int] = None): + """ + Updates the table with a specific snapshot. If no snapshot is provided, the most recent snapshot is used. + + NOTE: this method is only valid when the table is in `MANUAL_REFRESHING` mode. `STATIC` and `AUTO_REFRESHING` + Iceberg tables cannot be updated manually and will throw an exception if this method is called. + + Args: + snapshot_id (Optional[int]): the snapshot id to update to; if omitted the most recent snapshot will be used. + """ + if snapshot_id is not None: + self.j_object.update(snapshot_id) + return + self.j_object.update() + + @property + def j_object(self) -> jpy.JType: + return self.j_table + + class IcebergCatalogAdapter(JObjectWrapper): """ This class provides an interface for interacting with Iceberg catalogs. It allows listing namespaces, tables and @@ -135,7 +207,7 @@ def snapshots(self, table_identifier: str) -> Table: return self.j_object.listSnapshotsAsTable(table_identifier) - def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> Table: + def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> IcebergTable: """ Reads the table from the catalog using the provided instructions. Optionally, a snapshot id can be provided to read a specific snapshot of the table. @@ -157,8 +229,8 @@ def read_table(self, table_identifier: str, instructions: Optional[IcebergInstru instructions_object = _JIcebergInstructions.DEFAULT if snapshot_id is not None: - return Table(self.j_object.readTable(table_identifier, snapshot_id, instructions_object)) - return Table(self.j_object.readTable(table_identifier, instructions_object)) + return IcebergTable(self.j_object.readTable(table_identifier, snapshot_id, instructions_object)) + return IcebergTable(self.j_object.readTable(table_identifier, instructions_object)) @property def j_object(self) -> jpy.JType: From 2d79c38b93a85fd96fdd4654d9eb2ad72bb2202d Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 29 Jul 2024 15:20:15 -0700 Subject: [PATCH 11/56] Changes to mocked tests for ColumnSourceManager and PartitionAwareSourceTable. --- .../impl/TestPartitionAwareSourceTable.java | 16 ++++++++++------ .../TestRegionedColumnSourceManager.java | 14 +++++++------- 2 files changed, 17 insertions(+), 13 deletions(-) diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java index db0142c1c9b..e5933d866bb 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java @@ -6,9 +6,7 @@ import io.deephaven.base.Pair; import io.deephaven.base.verify.Assert; import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.rowset.WritableRowSet; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.perf.PerformanceEntry; import io.deephaven.engine.table.vectors.ColumnVectors; @@ -24,7 +22,6 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.engine.rowset.RowSequence; import io.deephaven.qst.column.Column; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.vector.ObjectVector; @@ -250,7 +247,9 @@ private void doInitializeCheck(final ImmutableTableLocationKey[] tableLocationKe @Override public Object invoke(Invocation invocation) { subscriptionBuffer = (TableLocationSubscriptionBuffer) invocation.getParameter(0); + subscriptionBuffer.beginTransaction(); Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded); + subscriptionBuffer.endTransaction(); return null; } }); @@ -320,7 +319,8 @@ private void doRefreshChangedCheck() { checking(new Expectations() { { oneOf(columnSourceManager).refresh(); - will(returnValue(toAdd.copy())); + will(returnValue(new TableUpdateImpl(toAdd.copy(), RowSetFactory.empty(), RowSetFactory.empty(), + RowSetShiftData.EMPTY, ModifiedColumnSet.ALL))); checking(new Expectations() { { oneOf(listener).getNotification(with(any(TableUpdateImpl.class))); @@ -360,7 +360,9 @@ private void doRefreshUnchangedCheck() { checking(new Expectations() { { oneOf(columnSourceManager).refresh(); - will(returnValue(RowSetFactory.empty())); + will(returnValue( + new TableUpdateImpl(RowSetFactory.empty(), RowSetFactory.empty(), RowSetFactory.empty(), + RowSetShiftData.EMPTY, ModifiedColumnSet.ALL))); } }); @@ -407,7 +409,9 @@ public Object invoke(Invocation invocation) { private void doAddLocationsRefreshCheck(final ImmutableTableLocationKey[] tableLocationKeys, final Set expectPassFilters) { + subscriptionBuffer.beginTransaction(); Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded); + subscriptionBuffer.endTransaction(); expectPassFilters.forEach(tl -> checking(new Expectations() { { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java index 55ffd9960b5..ff44314501a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java @@ -525,19 +525,19 @@ public void testRefreshing() { // Refresh them setSizeExpectations(true, true, 5, 1000); - updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added())); checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations()); // Refresh them with no change setSizeExpectations(true, true, 5, 1000); - updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added())); checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations()); // Refresh them with a change for the subscription-supporting one setSizeExpectations(true, true, 5, 1001); - updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added())); checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations()); @@ -568,26 +568,26 @@ public void testRefreshing() { // Test run with new locations included setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE); - updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added())); checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B), SUT.includedLocations()); // Test no-op run setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE); - updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added())); checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B), SUT.includedLocations()); // Test run with a location updated from null to not setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 2); - updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added())); checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B), SUT.includedLocations()); // Test run with a location updated setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 10000002); - updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added())); checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B), SUT.includedLocations()); From 3809f212672266b8e98b663709f590c2235304a9 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 30 Jul 2024 10:11:29 -0700 Subject: [PATCH 12/56] Added DHError handler and add'l documentation to python `snapshots()` call. --- .../iceberg/util/IcebergCatalogAdapter.java | 11 +++++++-- py/server/deephaven/experimental/iceberg.py | 23 +++++++++++++++---- 2 files changed, 27 insertions(+), 7 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index 8ba56e5cc70..e57b984e05f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -291,8 +291,15 @@ public List listSnapshots(@NotNull final TableIdentifier tableIdentifi /** * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting - * table will be static and contain the same information as {@link #listSnapshots(TableIdentifier)}. - * + * table will be static and contain contains the following columns: + *
    + *
  • Id: the snapshot identifier (can be used for updating the table or loading a specific snapshot).
  • + *
  • TimestampMs: the timestamp of the snapshot.
  • + *
  • Operation: the data operation that created this snapshot.
  • + *
  • Summary: additional information about the snapshot from the Iceberg metadata.
  • + *
  • SnapshotObject: a Java object containing the Iceberg API snapshot.
  • + *
+ * * @param tableIdentifier The identifier of the table from which to gather snapshots. * @return A list of all tables in the given namespace. */ diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 42e2c8292bf..9a3b1a9a1ce 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -141,11 +141,18 @@ def update(self, snapshot_id:Optional[int] = None): Args: snapshot_id (Optional[int]): the snapshot id to update to; if omitted the most recent snapshot will be used. + + Raises: + DHError: If unable to update the Iceberg table. + """ - if snapshot_id is not None: - self.j_object.update(snapshot_id) - return - self.j_object.update() + try: + if snapshot_id is not None: + self.j_object.update(snapshot_id) + return + self.j_object.update() + except Exception as e: + raise DHError(e, "Failed to update Iceberg table") from e @property def j_object(self) -> jpy.JType: @@ -196,7 +203,13 @@ def tables(self, namespace: str) -> Table: def snapshots(self, table_identifier: str) -> Table: """ - Returns information on the snapshots of the specified table as a Deephaven table. + Returns information on the snapshots of the specified table as a Deephaven table. The table contains the + following columns: + - `Id`: the snapshot identifier (can be used for updating the table or loading a specific snapshot). + - `TimestampMs`: the timestamp of the snapshot. + - `Operation`: the data operation that created this snapshot. + - `Summary`: additional information about this snapshot from the Iceberg metadata. + - `SnapshotObject`: a Java object containing the Iceberg API snapshot. Args: table_identifier (str): the table from which to list snapshots. From 5273a15e4847f262fcb972ac04cc80e5a57cc184 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 30 Jul 2024 10:18:25 -0700 Subject: [PATCH 13/56] Fixed typo in JavaDoc --- .../java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index e57b984e05f..c7458288165 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -220,6 +220,7 @@ public Table listNamespacesAsTable(@NotNull final Namespace namespace) { * * @return A {@link Table table} of all namespaces. */ + @SuppressWarnings("unused") public Table listNamespacesAsTable(@NotNull final String... namespace) { return listNamespacesAsTable(Namespace.of(namespace)); } @@ -291,7 +292,7 @@ public List listSnapshots(@NotNull final TableIdentifier tableIdentifi /** * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting - * table will be static and contain contains the following columns: + * table will be static and contain the following columns: *
    *
  • Id: the snapshot identifier (can be used for updating the table or loading a specific snapshot).
  • *
  • TimestampMs: the timestamp of the snapshot.
  • From b9e2c6e3ba9f43e2ec69abe9176433503c87d054 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 31 Jul 2024 11:15:26 -0700 Subject: [PATCH 14/56] WIP --- .../impl/locations/TableLocationProvider.java | 4 +-- .../impl/AbstractTableLocationProvider.java | 28 ++----------------- 2 files changed, 4 insertions(+), 28 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index d6158090289..d23b057ec13 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -80,7 +80,7 @@ void handleTableLocationKeyAdded( * @param tableLocationKey The new table location key. */ default void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey) { - handleTableLocationKeyAdded(tableLocationKey, this); + handleTableLocationKeyAdded(tableLocationKey, null); } /** @@ -105,7 +105,7 @@ void handleTableLocationKeyRemoved( */ @SuppressWarnings("unused") default void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey) { - handleTableLocationKeyRemoved(tableLocationKey, this); + handleTableLocationKeyRemoved(tableLocationKey, null); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 1a3f7ead2da..c9d8c7c0087 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -16,7 +16,7 @@ * Partial {@link TableLocationProvider} implementation for standalone use or as part of a {@link TableDataService}. *

    * Presents an interface similar to {@link TableLocationProvider.Listener} for subclasses to use when communicating with - * the parent; see {@link #handleTableLocationKeyAdded(TableLocationKey)}. + * the parent; see {@link #handleTableLocationKeyAdded(TableLocationKey, Object). *

    * Note that subclasses are responsible for determining when it's appropriate to call {@link #setInitialized()} and/or * override {@link #doInitialization()}. @@ -84,20 +84,7 @@ public final ImmutableTableKey getKey() { @Override protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) { - listener.beginTransaction(); unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded); - listener.endTransaction(); - } - - /** - * Deliver a possibly-new key. This method passes {@code this} as the transaction token. - * - * @param locationKey The new key - * @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools. - */ - protected final void handleTableLocationKeyAdded( - @NotNull final TableLocationKey locationKey) { - handleTableLocationKeyAdded(locationKey, this); } /** @@ -176,7 +163,7 @@ protected final void endTransaction(@NotNull final Object token) { } /** - * Called after a table location has been visited by {@link #handleTableLocationKeyAdded(TableLocationKey)}, + * Called after a table location has been visited by {@link #handleTableLocationKeyAdded(TableLocationKey, Object)}, * but before notifications have been delivered to any subscriptions, if applicable. The default implementation does * nothing, and may be overridden to implement additional features. * @@ -314,17 +301,6 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey) } } - /** - * Notify subscribers that {@code locationKey} was removed. This method passes {@code this} as the transaction - * token. - * - * @param locationKey the TableLocation that was removed - */ - protected final void handleTableLocationKeyRemoved( - @NotNull final ImmutableTableLocationKey locationKey) { - handleTableLocationKeyRemoved(locationKey, this); - } - /** * Notify subscribers that {@code locationKey} was removed. * From 9937f79627ba626febf8b97454015922b66938d5 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 31 Jul 2024 12:37:19 -0700 Subject: [PATCH 15/56] Suggestion from review --- .../impl/AbstractTableLocationProvider.java | 44 +++++++++++-------- 1 file changed, 26 insertions(+), 18 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index c9d8c7c0087..d996e35d053 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -163,9 +163,10 @@ protected final void endTransaction(@NotNull final Object token) { } /** - * Called after a table location has been visited by {@link #handleTableLocationKeyAdded(TableLocationKey, Object)}, - * but before notifications have been delivered to any subscriptions, if applicable. The default implementation does - * nothing, and may be overridden to implement additional features. + * Called after a table location has been visited by + * {@link #handleTableLocationKeyAdded(TableLocationKey, Object)}, but before notifications have been delivered to + * any subscriptions, if applicable. The default implementation does nothing, and may be overridden to implement + * additional features. * * @param locationKey The {@link TableLocationKey} that was visited. */ @@ -289,32 +290,31 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t * @param locationKey The {@link TableLocationKey} to remove */ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey) { - final Object removedLocation = tableLocations.remove(locationKey); - - if (removedLocation != null) { - handleTableLocationKeyRemoved(locationKey.makeImmutable()); - if (removedLocation instanceof AbstractTableLocation) { - final AbstractTableLocation abstractLocation = (AbstractTableLocation) removedLocation; - abstractLocation.handleUpdate(null, System.currentTimeMillis()); - abstractLocation.clearColumnLocations(); - } - } + handleTableLocationKeyRemoved(locationKey, null); } /** - * Notify subscribers that {@code locationKey} was removed. + * Handle a removal, optionally as part of a transaction. Notify subscribers that {@code locationKey} was removed if + * necessary. See {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics. * * @param locationKey the TableLocation that was removed * @param transactionToken The token identifying the transaction */ protected void handleTableLocationKeyRemoved( - @NotNull final ImmutableTableLocationKey locationKey, + @NotNull final TableLocationKey locationKey, @Nullable final Object transactionToken) { - if (supportsSubscriptions()) { - synchronized (subscriptions) { + if (!supportsSubscriptions()) { + maybeClearLocationForRemoval(tableLocations.remove(locationKey)); + return; + } + + synchronized (subscriptions) { + final Object removedLocation = tableLocations.remove(locationKey); + if (removedLocation != null) { + maybeClearLocationForRemoval(removedLocation); if (subscriptions.deliverNotification( Listener::handleTableLocationKeyRemoved, - locationKey, + locationKey.makeImmutable(), transactionToken, true)) { onEmpty(); @@ -323,6 +323,14 @@ protected void handleTableLocationKeyRemoved( } } + private static void maybeClearLocationForRemoval(@Nullable final Object removedLocation) { + if (removedLocation instanceof AbstractTableLocation) { + final AbstractTableLocation abstractLocation = (AbstractTableLocation) removedLocation; + abstractLocation.handleUpdate(null, System.currentTimeMillis()); + abstractLocation.clearColumnLocations(); + } + } + private void verifyPartitionKeys(@NotNull final TableLocationKey locationKey) { if (partitionKeys == null) { partitionKeys = new ArrayList<>(locationKey.getPartitionKeys()); From cd08038aecfc010aa95701b1add618253bb1b834 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 31 Jul 2024 13:25:34 -0700 Subject: [PATCH 16/56] WIP, changes to revert some transaction token code. --- .../impl/locations/TableLocationProvider.java | 15 ------ .../impl/AbstractTableLocationProvider.java | 49 +++++++++++-------- .../impl/PollingTableLocationProvider.java | 2 - .../impl/TestPartitionAwareSourceTable.java | 4 -- ...ebergAutoRefreshTableLocationProvider.java | 8 +-- ...ergManualRefreshTableLocationProvider.java | 12 ++--- .../IcebergStaticTableLocationProvider.java | 6 +-- 7 files changed, 42 insertions(+), 54 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index d23b057ec13..86ba191fba2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -33,14 +33,6 @@ interface Listener extends BasicTableDataListener { */ void beginTransaction(@NotNull Object token); - /** - * Begin a transaction that collects location key additions and removals to be processed atomically. Uses - * {@code this} as the token. - */ - default void beginTransaction() { - beginTransaction(this); - } - /** * End the transaction and process the location changes. * @@ -48,13 +40,6 @@ default void beginTransaction() { */ void endTransaction(@NotNull Object token); - /** - * End the transaction and process the location changes. Uses {@code this} as the token. - */ - default void endTransaction() { - endTransaction(this); - } - /** *

    * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index d996e35d053..21ae98fdb9e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -3,8 +3,11 @@ // package io.deephaven.engine.table.impl.locations.impl; +import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSourceManager; +import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; import org.jetbrains.annotations.NotNull; @@ -43,6 +46,9 @@ public abstract class AbstractTableLocationProvider (Collection) (Collection) Collections .unmodifiableCollection(tableLocations.keySet()); + final List locationsToClear; + final UpdateCommitter locationClearCommitter; + private volatile boolean initialized; private List partitionKeys; @@ -58,6 +64,18 @@ protected AbstractTableLocationProvider(@NotNull final TableKey tableKey, final super(supportsSubscriptions); this.tableKey = tableKey.makeImmutable(); this.partitionKeys = null; + + locationsToClear = new ArrayList<>(); + locationClearCommitter = new UpdateCommitter<>(this, + ExecutionContext.getContext().getUpdateGraph(), + (ignored) -> { + locationsToClear.forEach(location -> { + location.handleUpdate(null, System.currentTimeMillis()); + location.clearColumnLocations(); + + }); + locationsToClear.clear(); + }); } /** @@ -87,6 +105,10 @@ protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded); } + protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey locationKey) { + handleTableLocationKeyAdded(locationKey, null); + } + /** * Deliver a possibly-new key. * @@ -124,14 +146,6 @@ protected final void handleTableLocationKeyAdded( } } - /** - * Internal method to begin an atomic transaction of location adds and removes. This method passes {@code this} as - * the transaction token. - */ - protected final void beginTransaction() { - beginTransaction(this); - } - /** * Internal method to begin an atomic transaction of location adds and removes. * @@ -143,14 +157,6 @@ protected final void beginTransaction(@NotNull final Object token) { } } - /** - * Internal method to end an atomic transaction of location adds and removes. This method passes {@code this} as the - * transaction token. - */ - protected final void endTransaction() { - endTransaction(this); - } - /** * Internal method to end an atomic transaction of location adds and removes. * @@ -293,6 +299,10 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey) handleTableLocationKeyRemoved(locationKey, null); } + protected final void handleTableLocationKeyRemoved(@NotNull final TableLocationKey locationKey) { + handleTableLocationKeyRemoved(locationKey, null); + } + /** * Handle a removal, optionally as part of a transaction. Notify subscribers that {@code locationKey} was removed if * necessary. See {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics. @@ -323,11 +333,10 @@ protected void handleTableLocationKeyRemoved( } } - private static void maybeClearLocationForRemoval(@Nullable final Object removedLocation) { + private void maybeClearLocationForRemoval(@Nullable final Object removedLocation) { if (removedLocation instanceof AbstractTableLocation) { - final AbstractTableLocation abstractLocation = (AbstractTableLocation) removedLocation; - abstractLocation.handleUpdate(null, System.currentTimeMillis()); - abstractLocation.clearColumnLocations(); + locationsToClear.add((AbstractTableLocation) removedLocation); + locationClearCommitter.maybeActivate(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java index 5ac5c65c4e0..c9d213713d5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java @@ -52,7 +52,6 @@ public String getImplementationName() { // polling. We do need a mechanism to avoid going backwards, probably. @Override public void refresh() { - beginTransaction(); final Set missedKeys = new HashSet<>(getTableLocationKeys()); locationKeyFinder.findKeys(tableLocationKey -> { // noinspection SuspiciousMethodCalls @@ -60,7 +59,6 @@ public void refresh() { handleTableLocationKeyAdded(tableLocationKey); }); missedKeys.forEach(this::handleTableLocationKeyRemoved); - endTransaction(); setInitialized(); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java index e5933d866bb..6bed409a747 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java @@ -247,9 +247,7 @@ private void doInitializeCheck(final ImmutableTableLocationKey[] tableLocationKe @Override public Object invoke(Invocation invocation) { subscriptionBuffer = (TableLocationSubscriptionBuffer) invocation.getParameter(0); - subscriptionBuffer.beginTransaction(); Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded); - subscriptionBuffer.endTransaction(); return null; } }); @@ -409,9 +407,7 @@ public Object invoke(Invocation invocation) { private void doAddLocationsRefreshCheck(final ImmutableTableLocationKey[] tableLocationKeys, final Set expectPassFilters) { - subscriptionBuffer.beginTransaction(); Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded); - subscriptionBuffer.endTransaction(); expectPassFilters.forEach(tl -> checking(new Expectations() { { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index ad7212e1c3f..4a826d460f5 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -84,14 +84,14 @@ public void update(Snapshot snapshot) { * locations and removed locations. */ private void refreshSnapshot() { - beginTransaction(); + beginTransaction(this); final Set missedKeys = new HashSet<>(getTableLocationKeys()); locationKeyFinder.findKeys(tableLocationKey -> { missedKeys.remove(tableLocationKey); - handleTableLocationKeyAdded(tableLocationKey); + handleTableLocationKeyAdded(tableLocationKey, this); }); - missedKeys.forEach(this::handleTableLocationKeyRemoved); - endTransaction(); + missedKeys.forEach(tlk -> handleTableLocationKeyRemoved(tlk, this)); + endTransaction(this); setInitialized(); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index bfa5e99e60d..c4639a3edf5 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -91,14 +91,14 @@ public synchronized void update(final Snapshot snapshot) { * locations and removed locations. */ private void refreshSnapshot() { - beginTransaction(); + beginTransaction(this); final Set missedKeys = new HashSet<>(getTableLocationKeys()); - locationKeyFinder.findKeys(tableLocationKey -> { - missedKeys.remove(tableLocationKey); - handleTableLocationKeyAdded(tableLocationKey); + locationKeyFinder.findKeys(tlk -> { + missedKeys.remove(tlk); + handleTableLocationKeyAdded(tlk, this); }); - missedKeys.forEach(this::handleTableLocationKeyRemoved); - endTransaction(); + missedKeys.forEach(tlk -> handleTableLocationKeyRemoved(tlk, this)); + endTransaction(this); setInitialized(); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index 92798252dad..a9f52487681 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -40,9 +40,9 @@ public String getImplementationName() { @Override public void refresh() { - beginTransaction(); - locationKeyFinder.findKeys(this::handleTableLocationKeyAdded); - endTransaction(); + beginTransaction(this); + locationKeyFinder.findKeys(tlk -> handleTableLocationKeyAdded(tlk, this)); + endTransaction(this); setInitialized(); } From f28325f46ace6059d1a755f6f9dac6aa4f61ed83 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 21 Aug 2024 12:28:46 -0700 Subject: [PATCH 17/56] Correct logic across multiple transactions. --- .../locations/impl/TableLocationSubscriptionBuffer.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index 1962ae39ce1..44ac8119c11 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -175,8 +175,7 @@ public void endTransaction(@NotNull final Object token) { if (tokenLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) { for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsAdded) { // Verify that we don't have stacked adds (without intervening removes). - if (pendingLocationsAdded.contains(tableLocationKey) - && !pendingLocationsRemoved.contains(tableLocationKey)) { + if (pendingLocationsAdded.contains(tableLocationKey)) { throw new IllegalStateException("TableLocationKey " + tableLocationKey + " was already added by a previous transaction."); } @@ -203,8 +202,7 @@ public void handleTableLocationKeyAdded( if (transactionToken == null) { // If we're not in a transaction, modify the pending locations directly. // Need to verify that we don't have stacked adds (without intervening removes). - if (pendingLocationsAdded.contains(tableLocationKey) - && !pendingLocationsRemoved.contains(tableLocationKey)) { + if (pendingLocationsAdded.contains(tableLocationKey)) { throw new IllegalStateException("TableLocationKey " + tableLocationKey + " was already added by a previous transaction."); } From cd31d82a4818e862498d613e7caccd582dd76ff8 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 23 Aug 2024 11:18:26 -0700 Subject: [PATCH 18/56] Moved transaction accumulation to AbstractTableLocationProvider --- .../util/datastructures/SubscriptionSet.java | 14 +- .../impl/locations/TableLocationProvider.java | 16 +- .../impl/AbstractTableLocationProvider.java | 185 ++++++++++++++---- .../impl/FilteredTableDataService.java | 10 + .../impl/TableLocationSubscriptionBuffer.java | 184 ++++++----------- 5 files changed, 242 insertions(+), 167 deletions(-) diff --git a/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java b/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java index 1024f255ca2..cc76ffe0c44 100644 --- a/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java +++ b/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java @@ -221,18 +221,18 @@ public final boolean deliverNotification( } /** - * Dispatch a binary notification to all subscribers. Clean up any GC'd subscriptions. + * Dispatch a notification to all subscribers. Clean up any GC'd subscriptions. * * @param procedure The notification procedure to invoke - * @param notification The notification to deliver - * @param token An additional token to deliver (usually a transaction token) + * @param firstNotification The first item to deliver + * @param secondNotification The second item to deliver (must be of the same type as {@code firstNotification}) * @param activeOnly Whether to restrict this notification to active subscriptions only * @return Whether this operation caused the set to become empty */ public final boolean deliverNotification( - @NotNull final TriConsumer procedure, - @Nullable final NOTIFICATION_TYPE notification, - @Nullable final Object token, + @NotNull final TriConsumer procedure, + @Nullable final NOTIFICATION_TYPE firstNotification, + @Nullable final NOTIFICATION_TYPE secondNotification, final boolean activeOnly) { final int initialSize = size; for (int si = 0; si < size;) { @@ -243,7 +243,7 @@ public final boolean deliverNotification( continue; // si is not incremented in this case - we'll reconsider the same slot if necessary. } if (!activeOnly || currentEntry.isActive()) { - procedure.accept(currentListener, notification, token); + procedure.accept(currentListener, firstNotification, secondNotification); } ++si; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index 86ba191fba2..2339ec2efaf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -49,7 +49,7 @@ interface Listener extends BasicTableDataListener { *

    * If transactionToken is {@code null}, the key will be added to the pending additions immediately. *

    - * + * * @param tableLocationKey The new table location key. * @param transactionToken The token identifying the transaction. */ @@ -92,6 +92,20 @@ void handleTableLocationKeyRemoved( default void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey) { handleTableLocationKeyRemoved(tableLocationKey, null); } + + /** + *

    + * Notify the listener of collections of {@link TableLocationKey TableLocationKeys} added or removed while + * initiating or maintaining the location subscription. This should occur at most once per location, but the + * order of delivery is not guaranteed. + *

    + * + * @param addedKeys Collection of table location keys that were added. + * @param removedKeys Collection of table location keys that were removed. + */ + void handleTableLocationKeysUpdate( + @Nullable Collection addedKeys, + @Nullable Collection removedKeys); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 21ae98fdb9e..8df894bd253 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -3,10 +3,10 @@ // package io.deephaven.engine.table.impl.locations.impl; +import io.deephaven.base.verify.Assert; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.*; -import io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSourceManager; import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; @@ -28,8 +28,17 @@ public abstract class AbstractTableLocationProvider extends SubscriptionAggregator implements TableLocationProvider { + private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet(); + private final ImmutableTableKey tableKey; + // These sets represent open transactions that are being accumulated. + private final Set transactionTokens = new HashSet<>(); + private final Map> accumulatedLocationsAdded = new HashMap<>(); + private final Map> accumulatedLocationsRemoved = new HashMap<>(); + + private final Object transactionLock = new Object(); + /** * Map from {@link TableLocationKey} to itself, or to a {@link TableLocation}. The values are {@link TableLocation}s * if: @@ -102,13 +111,95 @@ public final ImmutableTableKey getKey() { @Override protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) { - unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded); + listener.handleTableLocationKeysUpdate(unmodifiableTableLocationKeys, null); } protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey locationKey) { handleTableLocationKeyAdded(locationKey, null); } + /** + * Internal method to begin an atomic transaction of location adds and removes. + * + * @param token A token to identify the transaction + */ + protected void beginTransaction(@NotNull final Object token) { + synchronized (transactionLock) { + // Verify that we can start a new transaction with this token. + if (transactionTokens.contains(token)) { + throw new IllegalStateException("A transaction with token " + token + " is currently open."); + } + Assert.eqFalse(accumulatedLocationsAdded.containsKey(token), + "accumulatedLocationsAdded.containsKey(token)"); + Assert.eqFalse(accumulatedLocationsRemoved.containsKey(token), + "accumulatedLocationsRemoved.containsKey(token)"); + + transactionTokens.add(token); + accumulatedLocationsAdded.put(token, EMPTY_TABLE_LOCATION_KEYS); + accumulatedLocationsRemoved.put(token, EMPTY_TABLE_LOCATION_KEYS); + } + } + + /** + * Internal method to end an atomic transaction of location adds and removes. + * + * @param token A token to identify the transaction + */ + protected void endTransaction(@NotNull final Object token) { + final Set locationsAdded; + final Set locationsRemoved; + synchronized (transactionLock) { + // Verify that this transaction is open. + if (!transactionTokens.remove(token)) { + throw new IllegalStateException("No transaction with token " + token + " is currently open."); + } + + locationsAdded = accumulatedLocationsAdded.remove(token); + locationsRemoved = accumulatedLocationsRemoved.remove(token); + } + + final Collection addedImmutableKeys = new ArrayList<>(locationsAdded.size()); + final Collection removedImmutableKeys = new ArrayList<>(locationsRemoved.size()); + + // Process the accumulated adds and removes under a lock on `tableLocations` to keep modifications atomic to + // other holders of this lock. + synchronized (tableLocations) { + if (locationsAdded != EMPTY_TABLE_LOCATION_KEYS || locationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { + for (TableLocationKey locationKey : locationsAdded) { + locationCreatedRecorder = false; + final Object result = tableLocations.putIfAbsent(locationKey, this::observeInsert); + visitLocationKey(locationKey); + if (locationCreatedRecorder) { + verifyPartitionKeys(locationKey); + addedImmutableKeys.add(toKeyImmutable(result)); + } + } + + for (TableLocationKey locationKey : locationsRemoved) { + final Object removedLocation = tableLocations.remove(locationKey); + if (removedLocation != null) { + maybeClearLocationForRemoval(removedLocation); + removedImmutableKeys.add(toKeyImmutable(locationKey)); + } + } + } + } + + if (subscriptions != null) { + synchronized (subscriptions) { + // Push the notifications to the subscribers. + if ((!addedImmutableKeys.isEmpty() || !removedImmutableKeys.isEmpty()) + && subscriptions.deliverNotification( + Listener::handleTableLocationKeysUpdate, + addedImmutableKeys, + removedImmutableKeys, + true)) { + onEmpty(); + } + } + } + } + /** * Deliver a possibly-new key. * @@ -119,12 +210,35 @@ protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey protected final void handleTableLocationKeyAdded( @NotNull final TableLocationKey locationKey, @Nullable final Object transactionToken) { + if (!supportsSubscriptions()) { tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable); visitLocationKey(toKeyImmutable(locationKey)); return; } + if (transactionToken != null) { + // When adding a location in a transaction, check for logical consistency. + // 1. If the location was already added in this transaction, we have a problem. A transaction should not + // add the same location twice. + // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a + // logical error (although it may not be supported by all consumers). + synchronized (transactionLock) { + if (accumulatedLocationsAdded.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { + accumulatedLocationsAdded.put(transactionToken, new HashSet<>()); + } + final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken); + + if (accumulatedLocationsAdded.containsKey(locationKey)) { + throw new IllegalStateException("TableLocationKey " + locationKey + + " was added multiple times in the same transaction."); + } + locationsAdded.add(locationKey); + } + return; + } + + // If we're not in a transaction, we should push this key immediately. synchronized (subscriptions) { // Since we're holding the lock on subscriptions, the following code is overly complicated - we could // certainly just deliver the notification in observeInsert. That said, I'm happier with this approach, @@ -138,7 +252,6 @@ protected final void handleTableLocationKeyAdded( if (subscriptions.deliverNotification( Listener::handleTableLocationKeyAdded, toKeyImmutable(result), - transactionToken, true)) { onEmpty(); } @@ -146,28 +259,6 @@ protected final void handleTableLocationKeyAdded( } } - /** - * Internal method to begin an atomic transaction of location adds and removes. - * - * @param token A token to identify the transaction - */ - protected final void beginTransaction(@NotNull final Object token) { - if (subscriptions != null) { - subscriptions.deliverNotification(Listener::beginTransaction, token, true); - } - } - - /** - * Internal method to end an atomic transaction of location adds and removes. - * - * @param token A token to identify the transaction - */ - protected final void endTransaction(@NotNull final Object token) { - if (subscriptions != null) { - subscriptions.deliverNotification(Listener::endTransaction, token, true); - } - } - /** * Called after a table location has been visited by * {@link #handleTableLocationKeyAdded(TableLocationKey, Object)}, but before notifications have been delivered to @@ -232,14 +323,12 @@ protected void doInitialization() { @Override @NotNull public final Collection getTableLocationKeys() { - // We need to prevent reading the map (and maybe mutating it?) during a transaction. - // We could transition between two maps, a stable copy and a shadow copy that is being mutated. - // Or we could hold a bigger lock while mutating the map, and hold the same lock here. Sounds like a job for a - // read-write lock (e.g. ReentrantReadWriteLock), maybe. If you want `FunctionalLock`, the pattern (but mostly - // not the code) from io.deephaven.engine.updategraph.UpdateGraphLock could help. - // I think we need the read-write lock for correctness, and I think we need to make it explicit. That is, the - // user needs to be able to get a read lock and hold it while it's operating on the returned collection. - return unmodifiableTableLocationKeys; + // This lock is held while `endTransaction()` updates `tableLocations` with the accumulated adds/removes. + // Locking here ensures that this call won't return while `tableLocations` (and `unmodifiableTableLocationKeys`) + // contain a partial transaction. + synchronized (tableLocations) { + return unmodifiableTableLocationKeys; + } } @Override @@ -274,8 +363,8 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t /** * Remove a {@link TableLocationKey} and its corresponding {@link TableLocation} (if it was created). All * subscribers to this TableLocationProvider will be - * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(ImmutableTableLocationKey, Object)} - * notified}. If the TableLocation was created, all of its subscribers will additionally be + * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(ImmutableTableLocationKey) notified}. If the + * TableLocation was created, all of its subscribers will additionally be * {@link TableLocation.Listener#handleUpdate() notified} that it no longer exists. This TableLocationProvider will * continue to update other locations and will no longer provide or request information about the removed location. * @@ -318,6 +407,31 @@ protected void handleTableLocationKeyRemoved( return; } + // When removing a location in a transaction, check for logical consistency. + // 1. If the location was already removed in this transaction, we have a problem. A transaction should not + // remove the same location twice. + // 2. If the location was already added in this transaction, we have a problem. A transaction should not + // add then remove the same location. + if (transactionToken != null) { + synchronized (transactionLock) { + if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { + accumulatedLocationsRemoved.put(transactionToken, new HashSet<>()); + } + final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken); + + if (accumulatedLocationsRemoved.containsKey(locationKey)) { + throw new IllegalStateException("TableLocationKey " + locationKey + + " was removed multiple times in the same transaction."); + } else if (accumulatedLocationsAdded.containsKey(locationKey)) { + throw new IllegalStateException("TableLocationKey " + locationKey + + " was removed after being added in the same transaction."); + } + locationsRemoved.add(locationKey); + return; + } + } + + // If we're not in a transaction, we should push this key immediately. synchronized (subscriptions) { final Object removedLocation = tableLocations.remove(locationKey); if (removedLocation != null) { @@ -325,7 +439,6 @@ protected void handleTableLocationKeyRemoved( if (subscriptions.deliverNotification( Listener::handleTableLocationKeyRemoved, locationKey.makeImmutable(), - transactionToken, true)) { onEmpty(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 6592ea11a12..948fb1fe38b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -196,6 +196,16 @@ public void handleTableLocationKeyRemoved( } } + @Override + public void handleTableLocationKeysUpdate( + @Nullable Collection addedKeys, + @Nullable Collection removedKeys) { + final TableLocationProvider.Listener outputListener = getWrapped(); + if (outputListener != null) { + outputListener.handleTableLocationKeysUpdate(addedKeys, removedKeys); + } + } + @Override public void handleException(@NotNull final TableDataException exception) { final TableLocationProvider.Listener outputListener = getWrapped(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index 44ac8119c11..c73658910d2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -3,7 +3,6 @@ // package io.deephaven.engine.table.impl.locations.impl; -import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableDataException; @@ -30,11 +29,6 @@ public class TableLocationSubscriptionBuffer implements TableLocationProvider.Li private Set pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; - // These sets represent open transactions that are being accumulated. - private final Set transactionTokens = new HashSet<>(); - private final Map> accumulatedLocationsAdded = new HashMap<>(); - private final Map> accumulatedLocationsRemoved = new HashMap<>(); - private TableDataException pendingException = null; public TableLocationSubscriptionBuffer(@NotNull final TableLocationProvider tableLocationProvider) { @@ -77,9 +71,7 @@ public synchronized LocationUpdate processPending() { // NB: Providers that don't support subscriptions don't tick - this single call to run is // sufficient. tableLocationProvider.refresh(); - // TODO: cast this to AbstractTableLocationProvider and call begin/endTransaction? - tableLocationProvider.getTableLocationKeys() - .forEach(tlk -> handleTableLocationKeyAdded(tlk, null)); + handleTableLocationKeysUpdate(tableLocationProvider.getTableLocationKeys(), null); } subscribed = true; } @@ -125,67 +117,12 @@ public synchronized void reset() { @Override public void beginTransaction(@NotNull final Object token) { - synchronized (updateLock) { - // Verify that we can start a new transaction with this token. - if (transactionTokens.contains(token)) { - throw new IllegalStateException("A transaction with token " + token + " is currently open."); - } - Assert.eqFalse(accumulatedLocationsAdded.containsKey(token), - "accumulatedLocationsAdded.containsKey(token)"); - Assert.eqFalse(accumulatedLocationsRemoved.containsKey(token), - "accumulatedLocationsRemoved.containsKey(token)"); - - transactionTokens.add(token); - accumulatedLocationsAdded.put(token, EMPTY_TABLE_LOCATION_KEYS); - accumulatedLocationsRemoved.put(token, EMPTY_TABLE_LOCATION_KEYS); - } + throw new UnsupportedOperationException("Transactions are not supported by this provider."); } @Override public void endTransaction(@NotNull final Object token) { - synchronized (updateLock) { - // Verify that this transaction is open. - if (!transactionTokens.remove(token)) { - throw new IllegalStateException("No transaction with token " + token + " is currently open."); - } - - final Set tokenLocationsAdded = accumulatedLocationsAdded.remove(token); - final Set tokenLocationsRemoved = accumulatedLocationsRemoved.remove(token); - - if (tokenLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { - for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsRemoved) { - // If we have a pending add that is removed by this transaction, we can remove it from the pending - // list because it is cancelled by this remove. This also covers the case where a `replace` - // operation has occurred in a previous transaction. - if (pendingLocationsAdded.remove(tableLocationKey)) { - continue; - } - // Verify that we don't have stacked removes (without intervening adds). - if (pendingLocationsRemoved.contains(tableLocationKey)) { - throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was already removed by a previous transaction."); - } - if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { - pendingLocationsRemoved = new HashSet<>(); - } - pendingLocationsRemoved.add(tableLocationKey); - } - } - - if (tokenLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) { - for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsAdded) { - // Verify that we don't have stacked adds (without intervening removes). - if (pendingLocationsAdded.contains(tableLocationKey)) { - throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was already added by a previous transaction."); - } - if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { - pendingLocationsAdded = new HashSet<>(); - } - pendingLocationsAdded.add(tableLocationKey); - } - } - } + throw new UnsupportedOperationException("Transactions are not supported by this provider."); } @Override @@ -193,36 +130,19 @@ public void handleTableLocationKeyAdded( @NotNull final ImmutableTableLocationKey tableLocationKey, @Nullable Object transactionToken) { synchronized (updateLock) { - // When adding a location in a transaction, check for logical consistency. - // 1. If the location was already added in this transaction, we have a problem. A transaction should not - // add the same location twice. - // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a - // logical error (although it may not be supported by all consumers). - - if (transactionToken == null) { - // If we're not in a transaction, modify the pending locations directly. - // Need to verify that we don't have stacked adds (without intervening removes). - if (pendingLocationsAdded.contains(tableLocationKey)) { - throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was already added by a previous transaction."); - } - if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { - pendingLocationsAdded = new HashSet<>(); - } - pendingLocationsAdded.add(tableLocationKey); - return; + if (transactionToken != null) { + throw new UnsupportedOperationException("Transactions are not supported by this provider."); } - if (accumulatedLocationsAdded.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { - accumulatedLocationsAdded.put(transactionToken, new HashSet<>()); - } - final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken); - - if (accumulatedLocationsAdded.containsKey(tableLocationKey)) { + // Need to verify that we don't have stacked adds (without intervening removes). + if (pendingLocationsAdded.contains(tableLocationKey)) { throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was added multiple times in the same transaction."); + + " was already added by a previous transaction."); } - locationsAdded.add(tableLocationKey); + if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsAdded = new HashSet<>(); + } + pendingLocationsAdded.add(tableLocationKey); } } @@ -231,43 +151,61 @@ public void handleTableLocationKeyRemoved( @NotNull final ImmutableTableLocationKey tableLocationKey, final Object transactionToken) { synchronized (updateLock) { - // When removing a location in a transaction, check for logical consistency. - // 1. If the location was already removed in this transaction, we have a problem. A transaction should not - // remove the same location twice. - // 2. If the location was already added in this transaction, we have a problem. A transaction should not - // add then remove the same location. + if (transactionToken != null) { + throw new UnsupportedOperationException("Transactions are not supported by this provider."); + } - if (transactionToken == null) { - // If we're not in a transaction, modify the pending locations directly. - // If we have a pending add, it is being cancelled by this remove. - if (pendingLocationsAdded.remove(tableLocationKey)) { - return; - } - // Verify that we don't have stacked removes (without intervening adds). - if (pendingLocationsRemoved.contains(tableLocationKey)) { - throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was already removed by a previous transaction."); - } - if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { - pendingLocationsRemoved = new HashSet<>(); - } - pendingLocationsRemoved.add(tableLocationKey); + // If we have a pending add, it is being cancelled by this remove. + if (pendingLocationsAdded.remove(tableLocationKey)) { return; } - - if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { - accumulatedLocationsRemoved.put(transactionToken, new HashSet<>()); + // Verify that we don't have stacked removes (without intervening adds). + if (pendingLocationsRemoved.contains(tableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + tableLocationKey + + " was already removed by a previous transaction."); + } + if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsRemoved = new HashSet<>(); } - final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken); + pendingLocationsRemoved.add(tableLocationKey); + } + } - if (accumulatedLocationsRemoved.containsKey(tableLocationKey)) { - throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was removed multiple times in the same transaction."); - } else if (accumulatedLocationsAdded.containsKey(tableLocationKey)) { - throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was removed after being added in the same transaction."); + @Override + public void handleTableLocationKeysUpdate( + @Nullable Collection addedKeys, + @Nullable Collection removedKeys) { + synchronized (updateLock) { + if (removedKeys != null) { + for (final ImmutableTableLocationKey removedTableLocationKey : removedKeys) { + // If we have a pending add, it is being cancelled by this remove. + if (pendingLocationsAdded.remove(removedTableLocationKey)) { + continue; + } + // Verify that we don't have stacked removes. + if (pendingLocationsRemoved.contains(removedTableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + removedTableLocationKey + + " was already removed by a previous transaction."); + } + if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsRemoved = new HashSet<>(); + } + pendingLocationsRemoved.add(removedTableLocationKey); + } + } + if (addedKeys != null) { + for (final ImmutableTableLocationKey addedTableLocationKey : addedKeys) { + // Need to verify that we don't have stacked adds. + if (pendingLocationsAdded.contains(addedTableLocationKey)) { + throw new IllegalStateException("TableLocationKey " + addedTableLocationKey + + " was already added by a previous transaction."); + } + if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsAdded = new HashSet<>(); + } + pendingLocationsAdded.add(addedTableLocationKey); + } } - locationsRemoved.add(tableLocationKey); } } From d680c0c66b25c053352834de65a9e91a78b7097e Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 26 Aug 2024 12:25:28 -0700 Subject: [PATCH 19/56] Moved transaction accumulation to AbstractTableLocationProvider --- .../impl/locations/TableLocationProvider.java | 12 +- .../impl/AbstractTableLocationProvider.java | 134 ++++++++++-------- .../impl/CompositeTableDataService.java | 4 +- .../impl/PollingTableLocationProvider.java | 15 +- .../impl/TableLocationSubscriptionBuffer.java | 4 +- .../util/ExecutorTableDataRefreshService.java | 36 ++--- .../util/TableDataRefreshService.java | 22 ++- .../regioned/RegionedColumnSourceManager.java | 59 ++++---- 8 files changed, 165 insertions(+), 121 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index 2339ec2efaf..b6205431a59 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -51,7 +51,8 @@ interface Listener extends BasicTableDataListener { *

    * * @param tableLocationKey The new table location key. - * @param transactionToken The token identifying the transaction. + * @param transactionToken The token identifying the transaction, or {@code null} if this addition is not part + * of a transaction. */ void handleTableLocationKeyAdded( @NotNull ImmutableTableLocationKey tableLocationKey, @@ -60,7 +61,8 @@ void handleTableLocationKeyAdded( /** * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location * subscription. This should occur at most once per location, but the order of delivery is not - * guaranteed. Uses {@code this} as the token. + * guaranteed. This addition is not part of any transaction, and is equivalent to + * {@code handleTableLocationKeyAdded(tableLocationKey, null);} by default. * * @param tableLocationKey The new table location key. */ @@ -77,14 +79,16 @@ default void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tabl *

    * * @param tableLocationKey The table location key that was removed. - * @param transactionToken The token identifying the transaction. + * @param transactionToken The token identifying the transaction, or {@code null} if this addition is not part + * of a transaction. */ void handleTableLocationKeyRemoved( @NotNull ImmutableTableLocationKey tableLocationKey, @Nullable Object transactionToken); /** - * Notify the listener of a {@link TableLocationKey} that has been removed. Uses {@code this} as the token. + * Notify the listener of a {@link TableLocationKey} that has been removed. This addition is not part of any + * transaction, and is equivalent to {@code handleTableLocationKeyAdded(tableLocationKey, null);} by default. * * @param tableLocationKey The table location key that was removed. */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 8df894bd253..4afbb770116 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -3,7 +3,6 @@ // package io.deephaven.engine.table.impl.locations.impl; -import io.deephaven.base.verify.Assert; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.*; @@ -30,12 +29,41 @@ public abstract class AbstractTableLocationProvider private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet(); + /** + * Helper class to manage a transaction of added and removed location keys. + */ + private static class Transaction { + Set locationsAdded = EMPTY_TABLE_LOCATION_KEYS; + Set locationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + + void addlocationKey(TableLocationKey locationKey) { + if (locationsAdded == EMPTY_TABLE_LOCATION_KEYS) { + locationsAdded = new HashSet<>(); + } else if (locationsAdded.contains(locationKey)) { + throw new IllegalStateException("TableLocationKey " + locationKey + + " was already added in this transaction."); + } + locationsAdded.add(locationKey); + } + + void removeLocationKey(TableLocationKey locationKey) { + if (locationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + locationsRemoved = new HashSet<>(); + } else if (locationsRemoved.contains(locationKey)) { + throw new IllegalStateException("TableLocationKey " + locationKey + + " was already removed and has not been replaced."); + } else if (locationsAdded.contains(locationKey)) { + throw new IllegalStateException("TableLocationKey " + locationKey + + " was removed after being added in the same transaction."); + } + locationsRemoved.add(locationKey); + } + } + private final ImmutableTableKey tableKey; // These sets represent open transactions that are being accumulated. - private final Set transactionTokens = new HashSet<>(); - private final Map> accumulatedLocationsAdded = new HashMap<>(); - private final Map> accumulatedLocationsRemoved = new HashMap<>(); + private final Map transactions = new HashMap<>(); private final Object transactionLock = new Object(); @@ -114,10 +142,6 @@ protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider listener.handleTableLocationKeysUpdate(unmodifiableTableLocationKeys, null); } - protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey locationKey) { - handleTableLocationKeyAdded(locationKey, null); - } - /** * Internal method to begin an atomic transaction of location adds and removes. * @@ -126,17 +150,10 @@ protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey protected void beginTransaction(@NotNull final Object token) { synchronized (transactionLock) { // Verify that we can start a new transaction with this token. - if (transactionTokens.contains(token)) { + if (transactions.containsKey(token)) { throw new IllegalStateException("A transaction with token " + token + " is currently open."); } - Assert.eqFalse(accumulatedLocationsAdded.containsKey(token), - "accumulatedLocationsAdded.containsKey(token)"); - Assert.eqFalse(accumulatedLocationsRemoved.containsKey(token), - "accumulatedLocationsRemoved.containsKey(token)"); - - transactionTokens.add(token); - accumulatedLocationsAdded.put(token, EMPTY_TABLE_LOCATION_KEYS); - accumulatedLocationsRemoved.put(token, EMPTY_TABLE_LOCATION_KEYS); + transactions.put(token, new Transaction()); } } @@ -146,26 +163,26 @@ protected void beginTransaction(@NotNull final Object token) { * @param token A token to identify the transaction */ protected void endTransaction(@NotNull final Object token) { - final Set locationsAdded; - final Set locationsRemoved; + final Transaction transaction; synchronized (transactionLock) { // Verify that this transaction is open. - if (!transactionTokens.remove(token)) { + transaction = transactions.remove(token); + if (transaction == null) { throw new IllegalStateException("No transaction with token " + token + " is currently open."); } - - locationsAdded = accumulatedLocationsAdded.remove(token); - locationsRemoved = accumulatedLocationsRemoved.remove(token); } - final Collection addedImmutableKeys = new ArrayList<>(locationsAdded.size()); - final Collection removedImmutableKeys = new ArrayList<>(locationsRemoved.size()); + final Collection addedImmutableKeys = + new ArrayList<>(transaction.locationsAdded.size()); + final Collection removedImmutableKeys = + new ArrayList<>(transaction.locationsRemoved.size()); // Process the accumulated adds and removes under a lock on `tableLocations` to keep modifications atomic to // other holders of this lock. synchronized (tableLocations) { - if (locationsAdded != EMPTY_TABLE_LOCATION_KEYS || locationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { - for (TableLocationKey locationKey : locationsAdded) { + if (transaction.locationsAdded != EMPTY_TABLE_LOCATION_KEYS + || transaction.locationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { + for (TableLocationKey locationKey : transaction.locationsAdded) { locationCreatedRecorder = false; final Object result = tableLocations.putIfAbsent(locationKey, this::observeInsert); visitLocationKey(locationKey); @@ -175,7 +192,7 @@ protected void endTransaction(@NotNull final Object token) { } } - for (TableLocationKey locationKey : locationsRemoved) { + for (TableLocationKey locationKey : transaction.locationsRemoved) { final Object removedLocation = tableLocations.remove(locationKey); if (removedLocation != null) { maybeClearLocationForRemoval(removedLocation); @@ -190,10 +207,10 @@ protected void endTransaction(@NotNull final Object token) { // Push the notifications to the subscribers. if ((!addedImmutableKeys.isEmpty() || !removedImmutableKeys.isEmpty()) && subscriptions.deliverNotification( - Listener::handleTableLocationKeysUpdate, - addedImmutableKeys, - removedImmutableKeys, - true)) { + Listener::handleTableLocationKeysUpdate, + addedImmutableKeys, + removedImmutableKeys, + true)) { onEmpty(); } } @@ -204,7 +221,17 @@ protected void endTransaction(@NotNull final Object token) { * Deliver a possibly-new key. * * @param locationKey The new key - * @param transactionToken The token identifying the transaction + * @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools. + */ + protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey locationKey) { + handleTableLocationKeyAdded(locationKey, null); + } + + /** + * Deliver a possibly-new key, optionally as part of a transaction. + * + * @param locationKey The new key + * @param transactionToken The token identifying the transaction (or null if not part of a transaction) * @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools. */ protected final void handleTableLocationKeyAdded( @@ -213,7 +240,7 @@ protected final void handleTableLocationKeyAdded( if (!supportsSubscriptions()) { tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable); - visitLocationKey(toKeyImmutable(locationKey)); + visitLocationKey(locationKey); return; } @@ -224,16 +251,12 @@ protected final void handleTableLocationKeyAdded( // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a // logical error (although it may not be supported by all consumers). synchronized (transactionLock) { - if (accumulatedLocationsAdded.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { - accumulatedLocationsAdded.put(transactionToken, new HashSet<>()); - } - final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken); - - if (accumulatedLocationsAdded.containsKey(locationKey)) { - throw new IllegalStateException("TableLocationKey " + locationKey - + " was added multiple times in the same transaction."); + final Transaction transaction = transactions.get(transactionToken); + if (transaction == null) { + throw new IllegalStateException( + "No transaction with token " + transactionToken + " is currently open."); } - locationsAdded.add(locationKey); + transaction.addlocationKey(locationKey); } return; } @@ -388,6 +411,12 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey) handleTableLocationKeyRemoved(locationKey, null); } + /** + * Handle a removal. Notify subscribers that {@code locationKey} was removed if necessary. See + * {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics. + * + * @param locationKey the TableLocation that was removed + */ protected final void handleTableLocationKeyRemoved(@NotNull final TableLocationKey locationKey) { handleTableLocationKeyRemoved(locationKey, null); } @@ -397,7 +426,7 @@ protected final void handleTableLocationKeyRemoved(@NotNull final TableLocationK * necessary. See {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics. * * @param locationKey the TableLocation that was removed - * @param transactionToken The token identifying the transaction + * @param transactionToken The token identifying the transaction (or null if not part of a transaction) */ protected void handleTableLocationKeyRemoved( @NotNull final TableLocationKey locationKey, @@ -414,19 +443,12 @@ protected void handleTableLocationKeyRemoved( // add then remove the same location. if (transactionToken != null) { synchronized (transactionLock) { - if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) { - accumulatedLocationsRemoved.put(transactionToken, new HashSet<>()); - } - final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken); - - if (accumulatedLocationsRemoved.containsKey(locationKey)) { - throw new IllegalStateException("TableLocationKey " + locationKey - + " was removed multiple times in the same transaction."); - } else if (accumulatedLocationsAdded.containsKey(locationKey)) { - throw new IllegalStateException("TableLocationKey " + locationKey - + " was removed after being added in the same transaction."); + final Transaction transaction = transactions.get(transactionToken); + if (transaction == null) { + throw new IllegalStateException( + "No transaction with token " + transactionToken + " is currently open."); } - locationsRemoved.add(locationKey); + transaction.removeLocationKey(locationKey); return; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index 3abcbf623e3..a9157da4b0f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -115,9 +115,7 @@ public void subscribe(@NotNull final Listener listener) { p.subscribe(listener); } else { p.refresh(); - listener.beginTransaction(p); - p.getTableLocationKeys().forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, p)); - listener.endTransaction(p); + p.getTableLocationKeys().forEach(listener::handleTableLocationKeyAdded); } }); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java index c9d213713d5..a350b93096e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java @@ -8,9 +8,6 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.HashSet; -import java.util.Set; - /** * Polling-driven {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} * discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a @@ -46,19 +43,9 @@ public String getImplementationName() { return IMPLEMENTATION_NAME; } - // The simplest way to support "push" of new data availability is to provide a callback to the user that just calls - // `refresh`, which would need to become synchronized. Alternatively, we could make an Iceberg-specific aTLP - // implementation that exposes a more specific callback, e.g. with a snapshot ID, as well as the option to disable - // polling. We do need a mechanism to avoid going backwards, probably. @Override public void refresh() { - final Set missedKeys = new HashSet<>(getTableLocationKeys()); - locationKeyFinder.findKeys(tableLocationKey -> { - // noinspection SuspiciousMethodCalls - missedKeys.remove(tableLocationKey); - handleTableLocationKeyAdded(tableLocationKey); - }); - missedKeys.forEach(this::handleTableLocationKeyRemoved); + locationKeyFinder.findKeys(this::handleTableLocationKeyAdded); setInitialized(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index c73658910d2..33a121c95ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -162,7 +162,7 @@ public void handleTableLocationKeyRemoved( // Verify that we don't have stacked removes (without intervening adds). if (pendingLocationsRemoved.contains(tableLocationKey)) { throw new IllegalStateException("TableLocationKey " + tableLocationKey - + " was already removed by a previous transaction."); + + " was already removed and has not been replaced."); } if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { pendingLocationsRemoved = new HashSet<>(); @@ -185,7 +185,7 @@ public void handleTableLocationKeysUpdate( // Verify that we don't have stacked removes. if (pendingLocationsRemoved.contains(removedTableLocationKey)) { throw new IllegalStateException("TableLocationKey " + removedTableLocationKey - + " was already removed by a previous transaction."); + + " was already removed and has not been replaced."); } if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { pendingLocationsRemoved = new HashSet<>(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java index 8db2b625354..3f0a4a862fb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java @@ -28,8 +28,8 @@ public class ExecutorTableDataRefreshService implements TableDataRefreshService private static final String NAME_PREFIX = "TableDataRefreshService-"; private final String name; - private final long tableLocationProviderRefreshIntervalMillis; - private final long tableLocationRefreshIntervalMillis; + private final long tableLocationProviderDefaultRefreshIntervalMillis; + private final long tableLocationDefaultRefreshIntervalMillis; private final ScheduledThreadPoolExecutor scheduler; @@ -43,9 +43,10 @@ public ExecutorTableDataRefreshService(@NotNull final String name, final long tableLocationRefreshIntervalMillis, final int threadPoolSize) { this.name = Require.neqNull(name, "name"); - this.tableLocationProviderRefreshIntervalMillis = Require.gtZero(tableLocationProviderRefreshIntervalMillis, - "tableLocationProviderRefreshIntervalMillis"); - this.tableLocationRefreshIntervalMillis = + this.tableLocationProviderDefaultRefreshIntervalMillis = + Require.gtZero(tableLocationProviderRefreshIntervalMillis, + "tableLocationProviderRefreshIntervalMillis"); + this.tableLocationDefaultRefreshIntervalMillis = Require.gtZero(tableLocationRefreshIntervalMillis, "tableLocationRefreshIntervalMillis"); NamingThreadFactory threadFactory = new NamingThreadFactory(TableDataRefreshService.class, "refreshThread"); @@ -110,12 +111,6 @@ public void cancel() { private class ScheduledTableLocationProviderRefresh extends ScheduledSubscriptionTask { - private ScheduledTableLocationProviderRefresh( - @NotNull final AbstractTableLocationProvider tableLocationProvider) { - super(tableLocationProvider, tableLocationProviderRefreshIntervalMillis); - providerSubscriptions.increment(1); - } - private ScheduledTableLocationProviderRefresh( @NotNull final AbstractTableLocationProvider tableLocationProvider, final long refreshIntervalMillis) { @@ -123,7 +118,6 @@ private ScheduledTableLocationProviderRefresh( providerSubscriptions.increment(1); } - @Override protected void refresh() { final long startTimeNanos = System.nanoTime(); @@ -140,8 +134,10 @@ public void cancel() { private class ScheduledTableLocationRefresh extends ScheduledSubscriptionTask { - private ScheduledTableLocationRefresh(@NotNull AbstractTableLocation tableLocation) { - super(tableLocation, tableLocationRefreshIntervalMillis); + private ScheduledTableLocationRefresh( + @NotNull final AbstractTableLocation tableLocation, + final long refreshIntervalMillis) { + super(tableLocation, refreshIntervalMillis); locationSubscriptions.increment(1); } @@ -162,7 +158,8 @@ public void cancel() { @Override public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( @NotNull final AbstractTableLocationProvider tableLocationProvider) { - return new ScheduledTableLocationProviderRefresh(tableLocationProvider); + return new ScheduledTableLocationProviderRefresh(tableLocationProvider, + tableLocationProviderDefaultRefreshIntervalMillis); } @Override @@ -175,6 +172,13 @@ public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( @Override public CancellableSubscriptionToken scheduleTableLocationRefresh( @NotNull final AbstractTableLocation tableLocation) { - return new ScheduledTableLocationRefresh(tableLocation); + return new ScheduledTableLocationRefresh(tableLocation, tableLocationDefaultRefreshIntervalMillis); + } + + @Override + public CancellableSubscriptionToken scheduleTableLocationRefresh( + @NotNull final AbstractTableLocation tableLocation, + final long refreshIntervalMillis) { + return new ScheduledTableLocationRefresh(tableLocation, refreshIntervalMillis); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java index 093426d0bb6..dbdb942282d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java @@ -32,7 +32,7 @@ interface CancellableSubscriptionToken { void submitOneTimeAsyncTask(@NotNull Runnable task); /** - * Schedule run for an AbstractTableLocationProvider. + * Schedule run for an AbstractTableLocationProvider using an implementation-defined default refresh interval * * @param tableLocationProvider The table location provider * @return A subscription token to be used for matching, which also supports cancellation @@ -52,13 +52,24 @@ CancellableSubscriptionToken scheduleTableLocationProviderRefresh( long refreshIntervalMs); /** - * Schedule run for an AbstractTableLocation. + * Schedule run for an AbstractTableLocation using an implementation-defined default refresh interval * * @param tableLocation The table location * @return A subscription token to be used for matching, which also supports cancellation */ CancellableSubscriptionToken scheduleTableLocationRefresh(@NotNull AbstractTableLocation tableLocation); + /** + * Schedule run for an AbstractTableLocation. + * + * @param tableLocation The table location + * @param refreshIntervalMs The interval in milliseconds between refreshes + * @return A subscription token to be used for matching, which also supports cancellation + */ + CancellableSubscriptionToken scheduleTableLocationRefresh( + @NotNull AbstractTableLocation tableLocation, + long refreshIntervalMs); + /** * Get (and possibly construct) a shared instance. * @@ -149,5 +160,12 @@ public CancellableSubscriptionToken scheduleTableLocationRefresh( @NotNull final AbstractTableLocation tableLocation) { throw new UnsupportedOperationException(); } + + @Override + public CancellableSubscriptionToken scheduleTableLocationRefresh( + @NotNull AbstractTableLocation tableLocation, + final long refreshIntervalMs) { + throw new UnsupportedOperationException(); + } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 3383d7c78a1..710dcc663a7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -26,7 +26,6 @@ import io.deephaven.io.logger.Logger; import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.ReferentialIntegrity; -import io.deephaven.util.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -75,17 +74,6 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col private final KeyedObjectHashMap includedTableLocations = new KeyedObjectHashMap<>(INCLUDED_TABLE_LOCATION_ENTRY_KEY); - /** - * List of locations that were removed this cycle. Will be cleared after each update. - */ - private final List removedTableLocations = new ArrayList<>(); - - /** - * The next region index to assign to a location. We increment for each new location and will not reuse indices from - * regions that were removed. - */ - private final MutableInt nextRegionIndex = new MutableInt(0); - /** * Table locations that provide the regions backing our column sources, in insertion order. */ @@ -122,7 +110,27 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col @ReferentialIntegrity private Runnable delayedErrorReference; - final List invalidatedLocations; + /** + * The next region index to assign to a location. We increment for each new location and will not reuse indices from + * regions that were removed. + */ + private int nextRegionIndex = 0; + + /** + * List of locations that were removed this cycle. Will be swapped each cycle with {@code invalidatedLocations} and + * cleared. + */ + private List removedTableLocations = new ArrayList<>(); + + /** + * List of locations to invalidate at the end of the cycle. Swapped with {@code removedTableLocations} each cycle to + * avoid reallocating. + */ + private List invalidatedLocations = new ArrayList<>(); + + /** + * Will invalidate the locations at the end of the cycle after all downstream updates are complete. + */ final UpdateCommitter invalidateCommitter; /** @@ -184,12 +192,13 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col } } - invalidatedLocations = new ArrayList<>(); invalidateCommitter = new UpdateCommitter<>(this, ExecutionContext.getContext().getUpdateGraph(), (ignored) -> { - invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate); - invalidatedLocations.clear(); + synchronized (this) { + invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate); + invalidatedLocations.clear(); + } }); } @@ -223,7 +232,7 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation) } @Override - public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locationKey) { + public synchronized boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locationKey) { final IncludedTableLocationEntry includedLocation = includedTableLocations.remove(locationKey); final EmptyTableLocationEntry emptyLocation = emptyTableLocations.remove(locationKey); @@ -234,9 +243,6 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati } else if (includedLocation != null) { orderedIncludedTableLocations.remove(includedLocation); removedTableLocations.add(includedLocation); - - // Mark this location for invalidation. - invalidatedLocations.add(includedLocation); invalidateCommitter.maybeActivate(); return true; } @@ -327,7 +333,12 @@ private TableUpdateImpl update(final boolean initializing) { removedRowSetBuilder.appendRowSequenceWithOffset(removedLocation.location.getRowSet(), regionFirstKey); removedRegionBuilder.appendKey(removedLocation.regionIndex); } - removedTableLocations.clear(); + + // Swap invalidatedLocations with removedTableLocations. + final List tmpTableLocations = removedTableLocations; + removedTableLocations = invalidatedLocations; + invalidatedLocations = tmpTableLocations; + Assert.eqTrue(removedTableLocations.isEmpty(), "removedTableLocations.isEmpty()"); final RowSetBuilderSequential modifiedRegionBuilder = initializing ? null : RowSetFactory.builderSequential(); @@ -370,8 +381,8 @@ private TableUpdateImpl update(final boolean initializing) { final RowSetBuilderSequential addedRegionBuilder = RowSetFactory.builderSequential(); - final int prevMaxIndex = nextRegionIndex.get(); - final int maxIndex = nextRegionIndex.get() + (entriesToInclude.isEmpty() ? 0 : entriesToInclude.size()); + final int prevMaxIndex = nextRegionIndex; + final int maxIndex = nextRegionIndex + (entriesToInclude.isEmpty() ? 0 : entriesToInclude.size()); if (!entriesToInclude.isEmpty()) { partitioningColumnValueSources.values().forEach( (final WritableColumnSource wcs) -> wcs.ensureCapacity(maxIndex)); @@ -528,7 +539,7 @@ private class IncludedTableLocationEntry implements Comparable> columnLocationStates = new ArrayList<>(); /** From 6607fc3ccb130d1bdb2a5dabeaf3747174686ea7 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 28 Aug 2024 14:30:57 -0700 Subject: [PATCH 20/56] PR comments addressed. --- .../impl/AbstractTableLocationProvider.java | 2 +- .../regioned/RegionedColumnSourceManager.java | 24 +++++++---- .../iceberg/util/IcebergToolsS3.java | 2 + ...ebergAutoRefreshTableLocationProvider.java | 4 +- ...ergManualRefreshTableLocationProvider.java | 11 +++-- .../IcebergStaticTableLocationProvider.java | 2 +- .../IcebergTableLocationProviderBase.java | 6 +-- .../iceberg/util/IcebergCatalogAdapter.java | 41 +++++++++++++------ .../iceberg/util/IcebergInstructions.java | 6 +-- .../iceberg/util/IcebergTableImpl.java | 11 +++++ .../iceberg/util/IcebergUpdateMode.java | 23 +++++++---- py/server/deephaven/experimental/iceberg.py | 32 ++++++++++----- 12 files changed, 112 insertions(+), 52 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 4afbb770116..b4afbe12cd1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -468,7 +468,7 @@ protected void handleTableLocationKeyRemoved( } } - private void maybeClearLocationForRemoval(@Nullable final Object removedLocation) { + private synchronized void maybeClearLocationForRemoval(@Nullable final Object removedLocation) { if (removedLocation instanceof AbstractTableLocation) { locationsToClear.add((AbstractTableLocation) removedLocation); locationClearCommitter.maybeActivate(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 710dcc663a7..e2f2714e688 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -322,9 +322,11 @@ public void deliverError(@NotNull final Throwable error, @Nullable final TableLi private TableUpdateImpl update(final boolean initializing) { final RowSetBuilderSequential addedRowSetBuilder = RowSetFactory.builderSequential(); - final RowSetBuilderSequential removedRowSetBuilder = RowSetFactory.builderSequential(); - final RowSetBuilderSequential removedRegionBuilder = RowSetFactory.builderSequential(); + final RowSetBuilderSequential removedRowSetBuilder = + removedTableLocations.isEmpty() ? null : RowSetFactory.builderSequential(); + final RowSetBuilderSequential removedRegionBuilder = + removedTableLocations.isEmpty() ? null : RowSetFactory.builderSequential(); // Sort the removed locations by region index, so that we can process them in order. removedTableLocations.sort(Comparator.comparingInt(e -> e.regionIndex)); @@ -379,7 +381,8 @@ private TableUpdateImpl update(final boolean initializing) { emptyTableLocations.removeAll(entriesToInclude); - final RowSetBuilderSequential addedRegionBuilder = RowSetFactory.builderSequential(); + final RowSetBuilderSequential addedRegionBuilder = + entriesToInclude.isEmpty() ? null : RowSetFactory.builderSequential(); final int prevMaxIndex = nextRegionIndex; final int maxIndex = nextRegionIndex + (entriesToInclude.isEmpty() ? 0 : entriesToInclude.size()); @@ -407,14 +410,14 @@ private TableUpdateImpl update(final boolean initializing) { addedRegionBuilder.appendKey(entry.regionIndex); }); } - final RowSet addedRegions = addedRegionBuilder.build(); + final RowSet addedRegions = addedRegionBuilder == null ? RowSetFactory.empty() : addedRegionBuilder.build(); if (addedRegions.isNonempty()) { includedLocationsTable.getRowSet().writableCast().insert(addedRegions); } if (initializing) { - Assert.eqZero(prevMaxIndex, "previousNumRegions"); + Assert.eqZero(prevMaxIndex, "prevMaxIndex"); if (isRefreshing) { rowSetSource.startTrackingPrevValues(); includedLocationsTable.getRowSet().writableCast().initializePreviousValue(); @@ -424,7 +427,8 @@ private TableUpdateImpl update(final boolean initializing) { } } else { final RowSet modifiedRegions = modifiedRegionBuilder.build(); - final RowSet removedRegions = removedRegionBuilder.build(); + final RowSet removedRegions = + removedRegionBuilder == null ? RowSetFactory.empty() : removedRegionBuilder.build(); if (addedRegions.isEmpty() && modifiedRegions.isEmpty() && removedRegions.isEmpty()) { SafeCloseable.closeAll(addedRegions, modifiedRegions, removedRegions); } else { @@ -438,8 +442,12 @@ private TableUpdateImpl update(final boolean initializing) { includedLocationsTable.notifyListeners(update); } } - return new TableUpdateImpl(addedRowSetBuilder.build(), removedRowSetBuilder.build(), RowSetFactory.empty(), - RowSetShiftData.EMPTY, ModifiedColumnSet.EMPTY); + return new TableUpdateImpl( + addedRowSetBuilder.build(), + removedRowSetBuilder == null ? RowSetFactory.empty() : removedRowSetBuilder.build(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.EMPTY); } @Override diff --git a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java index 166b47e5d28..a297bfb0964 100644 --- a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java +++ b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java @@ -68,6 +68,8 @@ public static IcebergCatalogAdapter createS3Rest( properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); } + // final org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index 4a826d460f5..81b38edd70c 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -28,6 +28,7 @@ public class IcebergAutoRefreshTableLocationProvider locationFactory, @NotNull final IcebergCatalogAdapter adapter, @NotNull final TableIdentifier tableIdentifier) { - super(tableKey, locationKeyFinder, locationFactory, null, true, adapter, tableIdentifier); + super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier); } // ------------------------------------------------------------------------------------------------------------------ @@ -108,8 +110,11 @@ private void refreshSnapshot() { @Override protected void activateUnderlyingDataSource() { - refreshSnapshot(); - activationSuccessful(this); + if (!initialized) { + refreshSnapshot(); + activationSuccessful(this); + initialized = true; + } } @Override diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index a9f52487681..53a5e9fc172 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -26,7 +26,7 @@ public IcebergStaticTableLocationProvider( @NotNull final IcebergBaseLayout locationKeyFinder, @NotNull final TableLocationFactory locationFactory, @NotNull final TableIdentifier tableIdentifier) { - super(tableKey, locationKeyFinder, locationFactory, null, false, null, tableIdentifier); + super(tableKey, locationKeyFinder, locationFactory, false, null, tableIdentifier); } // ------------------------------------------------------------------------------------------------------------------ diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java index 9e6577d500b..3e9fa7d5fa5 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java @@ -6,7 +6,6 @@ import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; -import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.iceberg.util.IcebergCatalogAdapter; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.TableIdentifier; @@ -18,7 +17,6 @@ public abstract class IcebergTableLocationProviderBase locationFactory; - final TableDataRefreshService refreshService; final IcebergCatalogAdapter adapter; final TableIdentifier tableIdentifier; @@ -26,14 +24,12 @@ public IcebergTableLocationProviderBase( @NotNull final TK tableKey, @NotNull final IcebergBaseLayout locationKeyFinder, @NotNull final TableLocationFactory locationFactory, - @Nullable final TableDataRefreshService refreshService, final boolean isRefreshing, @Nullable final IcebergCatalogAdapter adapter, @NotNull final TableIdentifier tableIdentifier) { super(tableKey, isRefreshing); this.locationKeyFinder = locationKeyFinder; this.locationFactory = locationFactory; - this.refreshService = refreshService; this.adapter = adapter; this.tableIdentifier = tableIdentifier; } @@ -65,6 +61,6 @@ public IcebergTableLocationProviderBase( @NotNull protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) { // noinspection unchecked - return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService); + return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, null); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index c4ed048c82e..6c0ecaee6cf 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -10,7 +10,6 @@ import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableKey; -import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; @@ -342,13 +341,34 @@ public List listSnapshots(@NotNull final TableIdentifier tableIdentifi /** * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting * table will be static and contain the following columns: - *
      - *
    • Id: the snapshot identifier (can be used for updating the table or loading a specific snapshot).
    • - *
    • TimestampMs: the timestamp of the snapshot.
    • - *
    • Operation: the data operation that created this snapshot.
    • - *
    • Summary: additional information about the snapshot from the Iceberg metadata.
    • - *
    • SnapshotObject: a Java object containing the Iceberg API snapshot.
    • - *
    + *
+ * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
Column NameDescription
IdThe snapshot identifier (can be used for updating the table or loading a specific snapshot)
TimestampMsThe timestamp of the snapshot
OperationThe data operation that created this snapshot
SummaryAdditional information about the snapshot from the Iceberg metadata
SnapshotObjectA Java object containing the Iceberg API snapshot
* * @param tableIdentifier The identifier of the table from which to gather snapshots. * @return A list of all tables in the given namespace. @@ -808,11 +828,6 @@ private IcebergTable readTableInternal( updateSourceRegistrar); } - private static KnownLocationKeyFinder toKnownKeys( - final IcebergBaseLayout keyFinder) { - return KnownLocationKeyFinder.copyFrom(keyFinder, Comparator.naturalOrder()); - } - /** * Returns the underlying Iceberg {@link Catalog catalog} used by this adapter. */ diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 5b6438475b2..9bc776c43b1 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -48,11 +48,11 @@ public static Builder builder() { /** * The {@link IcebergUpdateMode} mode to use when reading the Iceberg data files. Default is - * {@link IcebergUpdateMode#STATIC}. + * {@link IcebergUpdateMode#staticMode()}. */ @Value.Default public IcebergUpdateMode updateMode() { - return IcebergUpdateMode.STATIC; + return IcebergUpdateMode.staticMode(); } public interface Builder { @@ -69,7 +69,7 @@ public interface Builder { Builder putAllColumnRenames(Map entries); @SuppressWarnings("unused") - Builder updateMode(IcebergUpdateMode refreshing); + Builder updateMode(IcebergUpdateMode updateMode); IcebergInstructions build(); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java index a73a55d40be..02e3db6e115 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java @@ -57,4 +57,15 @@ public void update(final long snapshotId) { public void update(final @NotNull Snapshot snapshot) { locationProvider.update(snapshot); } + + @Override + public int currentSnapshotId() { + // TODO: implement the TableAdapter + return 0; + } + + @Override + public Snapshot currentSnapshot() { + return null; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java index 246e81ac65e..43da976fce9 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java @@ -11,12 +11,10 @@ public abstract class IcebergUpdateMode { private static final long REFRESH_INTERVAL_MS = 60_000L; - public static final IcebergUpdateMode STATIC = builder().updateType(IcebergUpdateType.STATIC).build(); - @SuppressWarnings("unused") - public static final IcebergUpdateMode AUTO_REFRESHING = + private static final IcebergUpdateMode STATIC = builder().updateType(IcebergUpdateType.STATIC).build(); + private static final IcebergUpdateMode AUTO_REFRESHING = builder().updateType(IcebergUpdateType.AUTO_REFRESHING).build(); - @SuppressWarnings("unused") - public static final IcebergUpdateMode MANUAL_REFRESHING = + private static final IcebergUpdateMode MANUAL_REFRESHING = builder().updateType(IcebergUpdateType.MANUAL_REFRESHING).build(); public enum IcebergUpdateType { @@ -27,8 +25,19 @@ public static Builder builder() { return ImmutableIcebergUpdateMode.builder(); } - @SuppressWarnings("unused") - public static IcebergUpdateMode autoRefreshing(final long refreshMs) { + public static IcebergUpdateMode staticMode() { + return STATIC; + } + + public static IcebergUpdateMode manualRefreshingMode() { + return MANUAL_REFRESHING; + } + + public static IcebergUpdateMode autoRefreshingMode() { + return AUTO_REFRESHING; + } + + public static IcebergUpdateMode autoRefreshingMode(final long refreshMs) { return ImmutableIcebergUpdateMode.builder() .updateType(IcebergUpdateType.AUTO_REFRESHING) .autoRefreshMs(refreshMs) diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 1080d2ccad9..623cccc6c16 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -32,9 +32,9 @@ class IcebergUpdateMode(JObjectWrapper): """ This class specifies the update mode for the Iceberg table to be loaded into Deephaven. The modes are: - - `STATIC`: The table is loaded once and does not change - - `MANUAL_REFRESHING`: The table can be manually refreshed by the user. - - `AUTO_REFRESHING`: The table will be automatically refreshed at a specified interval (use + - `static()`: The table is loaded once and does not change + - `manual_refresh()`: The table can be manually refreshed by the user. + - `auto_refresh()`: The table will be automatically refreshed at a system-defined interval (also can call `auto_refreshing(auto_refresh_ms: int)` to specify an interval rather than use the system default of 60 seconds). """ @@ -44,22 +44,34 @@ def __init__(self, mode: _JIcebergUpdateMode): self._j_object = mode @classmethod - def auto_refreshing(cls, auto_refresh_ms: int) -> IcebergUpdateMode: + def static(cls) -> IcebergUpdateMode: + """ + Creates an IcebergUpdateMode with no refreshing supported. + """ + return IcebergUpdateMode(_JIcebergUpdateMode.staticMode()) + + @classmethod + def manual_refresh(cls) -> IcebergUpdateMode: + """ + Creates an IcebergUpdateMode with manual refreshing enabled. + """ + return IcebergUpdateMode(_JIcebergUpdateMode.manualRefreshingMode()) + + @classmethod + def auto_refresh(cls, auto_refresh_ms:Optional[int] = None) -> IcebergUpdateMode: """ Creates an IcebergUpdateMode with auto-refreshing mode enabled using the provided refresh interval. :param auto_refresh_ms (int): the refresh interval in milliseconds. """ - return IcebergUpdateMode(_JIcebergUpdateMode.autoRefreshing(auto_refresh_ms)) + if auto_refresh_ms is None: + return IcebergUpdateMode(_JIcebergUpdateMode.autoRefreshingMode()) + return IcebergUpdateMode(_JIcebergUpdateMode.autoRefreshingMode(auto_refresh_ms)) @property def j_object(self) -> jpy.JType: return self._j_object -IcebergUpdateMode.STATIC=IcebergUpdateMode(_JIcebergUpdateMode.STATIC) -IcebergUpdateMode.MANUAL_REFRESHING=IcebergUpdateMode(_JIcebergUpdateMode.MANUAL_REFRESHING) -IcebergUpdateMode.AUTO_REFRESHING=IcebergUpdateMode(_JIcebergUpdateMode.AUTO_REFRESHING) - class IcebergInstructions(JObjectWrapper): """ @@ -119,7 +131,7 @@ def j_object(self) -> jpy.JType: class IcebergTable(Table): """ - IcebergTable is a subclass of Table that allows the users to dynamically update the table with new snapshots from + IcebergTable is a subclass of Table that allows users to dynamically update the table with new snapshots from the Iceberg catalog. """ j_object_type = _JIcebergTable From 893336f0ed9f4d80044be882849df12f849b9282 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 30 Aug 2024 14:31:02 -0700 Subject: [PATCH 21/56] Updated to use IcebergTableAdapter and exposed in python. Addressed PR comments. --- .../locations/impl/AbstractTableLocation.java | 44 +- .../impl/AbstractTableLocationProvider.java | 27 - .../regioned/RegionedColumnSourceManager.java | 23 + extensions/iceberg/s3/build.gradle | 1 + .../iceberg/util/IcebergToolsS3.java | 13 +- .../iceberg/util/IcebergToolsTest.java | 64 +- ...ebergAutoRefreshTableLocationProvider.java | 7 +- .../iceberg/layout/IcebergBaseLayout.java | 39 +- .../iceberg/layout/IcebergFlatLayout.java | 15 +- .../IcebergKeyValuePartitionedLayout.java | 17 +- ...ergManualRefreshTableLocationProvider.java | 10 +- .../IcebergTableLocationProviderBase.java | 6 +- .../iceberg/util/IcebergCatalogAdapter.java | 516 +++------------ .../deephaven/iceberg/util/IcebergTable.java | 3 - .../iceberg/util/IcebergTableAdapter.java | 587 ++++++++++++++++++ .../iceberg/util/IcebergTableImpl.java | 11 - .../deephaven/iceberg/util/IcebergTools.java | 9 +- py/server/deephaven/experimental/iceberg.py | 103 ++- 18 files changed, 925 insertions(+), 570 deletions(-) create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index 8391c0a774f..1c3f429fad7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java @@ -12,6 +12,7 @@ import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; import io.deephaven.util.annotations.InternalUseOnly; +import io.deephaven.util.referencecounting.ReferenceCounted; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -35,6 +36,8 @@ public abstract class AbstractTableLocation private final KeyedObjectHashMap columnLocations = new KeyedObjectHashMap<>(StringUtils.charSequenceKey()); + private final ReferenceCounted referenceCounted; + @SuppressWarnings("rawtypes") private static final AtomicReferenceFieldUpdater CACHED_DATA_INDEXES_UPDATER = AtomicReferenceFieldUpdater.newUpdater( @@ -58,6 +61,15 @@ protected AbstractTableLocation(@NotNull final TableKey tableKey, super(supportsSubscriptions); this.tableKey = Require.neqNull(tableKey, "tableKey").makeImmutable(); this.tableLocationKey = Require.neqNull(tableLocationKey, "tableLocationKey").makeImmutable(); + + referenceCounted = new ReferenceCounted() { + @Override + protected void onReferenceCountAtZero() { + // Call the location's onReferenceCountAtZero method + AbstractTableLocation.this.onReferenceCountAtZero(); + } + }; + } @Override @@ -158,7 +170,7 @@ public final ColumnLocation getColumnLocation(@NotNull final CharSequence name) * Clear all column locations (usually because a truncated location was observed). */ @SuppressWarnings("unused") - protected final void clearColumnLocations() { + public final void clearColumnLocations() { columnLocations.clear(); } @@ -229,4 +241,34 @@ public final BasicDataIndex getDataIndex(@NotNull final String... columns) { @InternalUseOnly @Nullable public abstract BasicDataIndex loadDataIndex(@NotNull String... columns); + + // ------------------------------------------------------------------------------------------------------------------ + // Reference counting implementation + // ------------------------------------------------------------------------------------------------------------------ + + /** + * Increment the reference count by one. + * + * @throws IllegalStateException If the reference count was not successfully incremented + */ + public final void incrementReferenceCount() { + referenceCounted.incrementReferenceCount(); + } + + /** + * Decrement the reference count by one, when the reference count reaches zero this location will be cleared. + * + * @throws IllegalStateException If the reference count was not successfully incremented + */ + public void decrementReferenceCount() { + referenceCounted.decrementReferenceCount(); + } + + /** + * The reference count has reached zero, we can clear this location and release any resources. + */ + private void onReferenceCountAtZero() { + handleUpdate(null, System.currentTimeMillis()); + clearColumnLocations(); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index b4afbe12cd1..3b49fa56634 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -3,10 +3,8 @@ // package io.deephaven.engine.table.impl.locations.impl; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.*; -import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; import org.jetbrains.annotations.NotNull; @@ -83,9 +81,6 @@ void removeLocationKey(TableLocationKey locationKey) { (Collection) (Collection) Collections .unmodifiableCollection(tableLocations.keySet()); - final List locationsToClear; - final UpdateCommitter locationClearCommitter; - private volatile boolean initialized; private List partitionKeys; @@ -101,18 +96,6 @@ protected AbstractTableLocationProvider(@NotNull final TableKey tableKey, final super(supportsSubscriptions); this.tableKey = tableKey.makeImmutable(); this.partitionKeys = null; - - locationsToClear = new ArrayList<>(); - locationClearCommitter = new UpdateCommitter<>(this, - ExecutionContext.getContext().getUpdateGraph(), - (ignored) -> { - locationsToClear.forEach(location -> { - location.handleUpdate(null, System.currentTimeMillis()); - location.clearColumnLocations(); - - }); - locationsToClear.clear(); - }); } /** @@ -195,7 +178,6 @@ protected void endTransaction(@NotNull final Object token) { for (TableLocationKey locationKey : transaction.locationsRemoved) { final Object removedLocation = tableLocations.remove(locationKey); if (removedLocation != null) { - maybeClearLocationForRemoval(removedLocation); removedImmutableKeys.add(toKeyImmutable(locationKey)); } } @@ -432,7 +414,6 @@ protected void handleTableLocationKeyRemoved( @NotNull final TableLocationKey locationKey, @Nullable final Object transactionToken) { if (!supportsSubscriptions()) { - maybeClearLocationForRemoval(tableLocations.remove(locationKey)); return; } @@ -457,7 +438,6 @@ protected void handleTableLocationKeyRemoved( synchronized (subscriptions) { final Object removedLocation = tableLocations.remove(locationKey); if (removedLocation != null) { - maybeClearLocationForRemoval(removedLocation); if (subscriptions.deliverNotification( Listener::handleTableLocationKeyRemoved, locationKey.makeImmutable(), @@ -468,13 +448,6 @@ protected void handleTableLocationKeyRemoved( } } - private synchronized void maybeClearLocationForRemoval(@Nullable final Object removedLocation) { - if (removedLocation instanceof AbstractTableLocation) { - locationsToClear.add((AbstractTableLocation) removedLocation); - locationClearCommitter.maybeActivate(); - } - } - private void verifyPartitionKeys(@NotNull final TableLocationKey locationKey) { if (partitionKeys == null) { partitionKeys = new ArrayList<>(locationKey.getPartitionKeys()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index e2f2714e688..274f603835a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; import io.deephaven.engine.table.impl.locations.impl.TableLocationUpdateSubscriptionBuffer; import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; import io.deephaven.engine.table.impl.sources.ObjectArraySource; @@ -103,6 +104,9 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col @ReferentialIntegrity private final Collection retainedDataIndexes = new ArrayList<>(); + private final List locationsToClear; + private final UpdateCommitter locationClearCommitter; + /** * A reference to a delayed error notifier for the {@link #includedLocationsTable}, if one is pending. */ @@ -192,6 +196,19 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col } } + + locationsToClear = new ArrayList<>(); + locationClearCommitter = new UpdateCommitter<>(this, + ExecutionContext.getContext().getUpdateGraph(), + (ignored) -> { + locationsToClear.forEach(location -> { + location.handleUpdate(null, System.currentTimeMillis()); + location.clearColumnLocations(); + + }); + locationsToClear.clear(); + }); + invalidateCommitter = new UpdateCommitter<>(this, ExecutionContext.getContext().getUpdateGraph(), (ignored) -> { @@ -212,6 +229,9 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation) log.debug().append("LOCATION_ADDED:").append(tableLocation.toString()).endl(); } emptyTableLocations.add(new EmptyTableLocationEntry(tableLocation)); + if (tableLocation instanceof AbstractTableLocation) { + ((AbstractTableLocation) tableLocation).incrementReferenceCount(); + } } else { // Duplicate location - not allowed final TableLocation duplicateLocation = @@ -656,6 +676,9 @@ private boolean pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) { private void invalidate() { columnLocationStates.forEach(cls -> cls.source.invalidateRegion(regionIndex)); + if (location instanceof AbstractTableLocation) { + ((AbstractTableLocation) location).decrementReferenceCount(); + } } @Override diff --git a/extensions/iceberg/s3/build.gradle b/extensions/iceberg/s3/build.gradle index bde1e84bc7f..019906dd767 100644 --- a/extensions/iceberg/s3/build.gradle +++ b/extensions/iceberg/s3/build.gradle @@ -1,6 +1,7 @@ plugins { id 'java-library' id 'io.deephaven.project.register' + id 'io.deephaven.hadoop-common-dependencies' } evaluationDependsOn Docker.registryProject('localstack') diff --git a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java index a297bfb0964..a552a7f3a05 100644 --- a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java +++ b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java @@ -4,6 +4,7 @@ package io.deephaven.iceberg.util; import com.google.common.base.Strings; +import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.aws.AwsClientProperties; @@ -68,14 +69,11 @@ public static IcebergCatalogAdapter createS3Rest( properties.put(S3FileIOProperties.ENDPOINT, endpointOverride); } - // final org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); - - final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); - final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; + catalog.setConf(new Configuration()); catalog.initialize(catalogName, properties); - return new IcebergCatalogAdapter(catalog, fileIO); + return new IcebergCatalogAdapter(catalog); } /** @@ -103,11 +101,10 @@ public static IcebergCatalogAdapter createGlue( properties.put(CatalogProperties.URI, catalogURI); properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); - final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null); - final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; + catalog.setConf(new Configuration()); catalog.initialize(catalogName, properties); - return new IcebergCatalogAdapter(catalog, fileIO); + return new IcebergCatalogAdapter(catalog); } } diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 44a17942cdf..07b6b127cdb 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -44,8 +44,8 @@ import java.util.stream.Collectors; import static io.deephaven.iceberg.util.IcebergCatalogAdapter.NAMESPACE_DEFINITION; -import static io.deephaven.iceberg.util.IcebergCatalogAdapter.SNAPSHOT_DEFINITION; import static io.deephaven.iceberg.util.IcebergCatalogAdapter.TABLES_DEFINITION; +import static io.deephaven.iceberg.util.IcebergTableAdapter.SNAPSHOT_DEFINITION; public abstract class IcebergToolsTest { @@ -187,7 +187,7 @@ public void tearDown() throws ExecutionException, InterruptedException { @Test public void testListNamespaces() { - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Collection namespaces = adapter.listNamespaces(); final Collection namespaceNames = @@ -197,14 +197,14 @@ public void testListNamespaces() { Assert.eqTrue(namespaceNames.contains("sales"), "namespaceNames.contains(sales)"); Assert.eqTrue(namespaceNames.contains("sample"), "namespaceNames.contains(sample)"); - final Table table = adapter.listNamespacesAsTable(); + final Table table = adapter.namespaces(); Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog"); Assert.equals(table.getDefinition(), "table.getDefinition()", NAMESPACE_DEFINITION); } @Test public void testListTables() { - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); @@ -216,19 +216,19 @@ public void testListTables() { Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_single")), "tables.contains(sales_single)"); Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_renamed")), "tables.contains(sales_renamed)"); - Table table = adapter.listTablesAsTable(ns); + Table table = adapter.tables(ns); Assert.eq(table.size(), "table.size()", 4, "4 tables in the namespace"); Assert.equals(table.getDefinition(), "table.getDefinition()", TABLES_DEFINITION); // Test the string versions of the methods - table = adapter.listTablesAsTable("sales"); + table = adapter.tables("sales"); Assert.eq(table.size(), "table.size()", 4, "4 tables in the namespace"); Assert.equals(table.getDefinition(), "table.getDefinition()", TABLES_DEFINITION); } @Test public void testListSnapshots() { - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final TLongArrayList snapshotIds = new TLongArrayList(); final TableIdentifier tableIdentifier = TableIdentifier.of("sales", "sales_multi"); @@ -242,12 +242,12 @@ public void testListSnapshots() { Assert.eqTrue(snapshotIds.contains(3247344357341484163L), "snapshots.contains(3247344357341484163L)"); Assert.eqTrue(snapshotIds.contains(1792185872197984875L), "snapshots.contains(1792185872197984875L)"); - Table table = adapter.listSnapshotsAsTable(tableIdentifier); + Table table = adapter.snapshots(tableIdentifier); Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION); // Test the string versions of the methods - table = adapter.listSnapshotsAsTable("sales.sales_multi"); + table = adapter.snapshots("sales.sales_multi"); Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION); } @@ -257,7 +257,7 @@ public void testOpenTableA() throws ExecutionException, InterruptedException, Ti uploadSalesPartitioned(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -280,7 +280,7 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti uploadSalesMulti(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); @@ -302,7 +302,7 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti uploadSalesSingle(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_single"); @@ -324,7 +324,7 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti public void testOpenTableS3Only() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesPartitioned(); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -345,7 +345,7 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -373,7 +373,7 @@ public void testOpenTablePartitionTypeException() { .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -417,7 +417,7 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -447,7 +447,7 @@ public void testSkippedPartitioningColumn() throws ExecutionException, Interrupt .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -477,7 +477,7 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -498,7 +498,7 @@ public void testZeroPartitioningColumns() throws ExecutionException, Interrupted .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -526,7 +526,7 @@ public void testIncorrectPartitioningColumns() throws ExecutionException, Interr .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -562,7 +562,7 @@ public void testMissingPartitioningColumns() { .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -591,7 +591,7 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -610,7 +610,7 @@ public void testOpenTableColumnLegalization() throws ExecutionException, Interru .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_renamed"); @@ -633,7 +633,7 @@ public void testOpenTableColumnLegalizationRename() .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_renamed"); @@ -665,7 +665,7 @@ public void testOpenTableColumnLegalizationPartitionException() { .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -696,7 +696,7 @@ public void testOpenTableColumnRenamePartitioningColumns() .build(); final IcebergCatalogAdapter adapter = - IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); @@ -720,7 +720,7 @@ public void testOpenTableColumnRenamePartitioningColumns() public void testOpenTableSnapshot() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); @@ -752,7 +752,7 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); @@ -799,7 +799,7 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx public void testOpenAllTypesTable() throws ExecutionException, InterruptedException, TimeoutException { uploadAllTypes(); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sample"); final TableIdentifier tableId = TableIdentifier.of(ns, "all_types"); @@ -812,7 +812,7 @@ public void testOpenAllTypesTable() throws ExecutionException, InterruptedExcept @Test public void testTableDefinition() { - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); @@ -837,7 +837,7 @@ public void testTableDefinition() { @Test public void testTableDefinitionTable() { - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final Namespace ns = Namespace.of("sales"); final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); @@ -870,7 +870,7 @@ public void testTableDefinitionTable() { @Test public void testTableDefinitionWithInstructions() { - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); IcebergInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index 81b38edd70c..bafbcdff872 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -8,7 +8,7 @@ import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; -import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import io.deephaven.iceberg.util.IcebergTableAdapter; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; @@ -39,7 +39,7 @@ public IcebergAutoRefreshTableLocationProvider( @NotNull final TableLocationFactory locationFactory, @NotNull final TableDataRefreshService refreshService, final long refreshIntervalMs, - @NotNull final IcebergCatalogAdapter adapter, + @NotNull final IcebergTableAdapter adapter, @NotNull final TableIdentifier tableIdentifier) { super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier); @@ -59,7 +59,8 @@ public String getImplementationName() { @Override public synchronized void refresh() { - final Snapshot latestSnapshot = adapter.getCurrentSnapshot(tableIdentifier); + adapter.refresh(); + final Snapshot latestSnapshot = adapter.currentSnapshot(); if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) { locationKeyFinder.snapshot = latestSnapshot; refreshSnapshot(); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index 70a65230f29..1f1d9ba0546 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -10,9 +10,9 @@ import io.deephaven.iceberg.location.IcebergTableLocationKey; import io.deephaven.iceberg.location.IcebergTableParquetLocationKey; import io.deephaven.iceberg.util.IcebergInstructions; +import io.deephaven.iceberg.util.IcebergTableAdapter; import io.deephaven.parquet.table.ParquetInstructions; import org.apache.iceberg.*; -import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -24,19 +24,15 @@ public abstract class IcebergBaseLayout implements TableLocationKeyFinder { /** - * The {@link TableDefinition} that will be used for the table. + * The {@link IcebergTableAdapter} that will be used to access the table. */ - final TableDefinition tableDef; - - /** - * The Iceberg {@link Table} to discover locations for. - */ - final Table table; + final IcebergTableAdapter tableAdapter; /** - * The {@link FileIO} to use for passing to the catalog reading manifest data files. + * The {@link TableDefinition} that will be used for life of this table. Although Iceberg table schema may change, + * schema changes are not supported in Deephaven. */ - final FileIO fileIO; + final TableDefinition tableDef; /** * The instructions for customizations while reading. @@ -87,28 +83,24 @@ protected IcebergTableLocationKey locationKey( return new IcebergTableParquetLocationKey(fileUri, 0, partitions, parquetInstructions); } throw new UnsupportedOperationException(String.format("%s:%d - an unsupported file format %s for URI '%s'", - table, snapshot.snapshotId(), format, fileUri)); + tableAdapter, snapshot.snapshotId(), format, fileUri)); } /** - * @param tableDef The {@link TableDefinition} that will be used for the table. - * @param table The {@link Table} to discover locations for. + * @param tableAdapter The {@link IcebergTableAdapter} that will be used to access the table. * @param tableSnapshot The {@link Snapshot} from which to discover data files. - * @param fileIO The file IO to use for reading manifest data files. * @param instructions The instructions for customizations while reading. */ public IcebergBaseLayout( - @NotNull final TableDefinition tableDef, - @NotNull final Table table, + @NotNull final IcebergTableAdapter tableAdapter, @NotNull final Snapshot tableSnapshot, - @NotNull final FileIO fileIO, @NotNull final IcebergInstructions instructions) { - this.tableDef = tableDef; - this.table = table; + this.tableAdapter = tableAdapter; this.snapshot = tableSnapshot; - this.fileIO = fileIO; this.instructions = instructions; + this.tableDef = tableAdapter.definition(tableSnapshot, instructions); + this.cache = new HashMap<>(); } @@ -116,9 +108,10 @@ public IcebergBaseLayout( @Override public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { + final Table table = tableAdapter.icebergTable(); try { // Retrieve the manifest files from the snapshot - final List manifestFiles = snapshot.allManifests(fileIO); + final List manifestFiles = snapshot.allManifests(table.io()); for (final ManifestFile manifestFile : manifestFiles) { // Currently only can process manifest files with DATA content type. if (manifestFile.content() != ManifestContent.DATA) { @@ -126,7 +119,7 @@ public synchronized void findKeys(@NotNull final Consumer reader = ManifestFiles.read(manifestFile, fileIO)) { + try (final ManifestReader reader = ManifestFiles.read(manifestFile, table.io())) { for (DataFile df : reader) { final URI fileUri = FileUtils.convertToURI(df.path().toString(), false); final IcebergTableLocationKey locationKey = @@ -139,7 +132,7 @@ public synchronized void findKeys(@NotNull final Consumer type; final int index; @@ -39,21 +38,17 @@ public ColumnData(String name, Class type, int index) { private final List outputPartitioningColumns; /** - * @param tableDef The {@link TableDefinition} that will be used for the table. - * @param table The {@link Table} to discover locations for. + * @param tableAdapter The {@link IcebergTableAdapter} that will be used to access the table. * @param tableSnapshot The {@link Snapshot} from which to discover data files. - * @param fileIO The file IO to use for reading manifest data files. * @param partitionSpec The Iceberg {@link PartitionSpec partition spec} for the table. * @param instructions The instructions for customizations while reading. */ public IcebergKeyValuePartitionedLayout( - @NotNull final TableDefinition tableDef, - @NotNull final org.apache.iceberg.Table table, + @NotNull final IcebergTableAdapter tableAdapter, @NotNull final org.apache.iceberg.Snapshot tableSnapshot, - @NotNull final FileIO fileIO, @NotNull final PartitionSpec partitionSpec, @NotNull final IcebergInstructions instructions) { - super(tableDef, table, tableSnapshot, fileIO, instructions); + super(tableAdapter, tableSnapshot, instructions); // We can assume due to upstream validation that there are no duplicate names (after renaming) that are included // in the output definition, so we can ignore duplicates. @@ -81,7 +76,7 @@ public IcebergKeyValuePartitionedLayout( @Override public String toString() { - return IcebergKeyValuePartitionedLayout.class.getSimpleName() + '[' + table.name() + ']'; + return IcebergKeyValuePartitionedLayout.class.getSimpleName() + '[' + tableAdapter + ']'; } @Override diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index bb1f4b50fdc..f2e5e8fb7e2 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -6,7 +6,7 @@ import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; -import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import io.deephaven.iceberg.util.IcebergTableAdapter; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; @@ -33,7 +33,7 @@ public IcebergManualRefreshTableLocationProvider( @NotNull final TK tableKey, @NotNull final IcebergBaseLayout locationKeyFinder, @NotNull final TableLocationFactory locationFactory, - @NotNull final IcebergCatalogAdapter adapter, + @NotNull final IcebergTableAdapter adapter, @NotNull final TableIdentifier tableIdentifier) { super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier); } @@ -55,12 +55,14 @@ public synchronized void refresh() { @Override public synchronized void update() { - update(adapter.getCurrentSnapshot(tableIdentifier)); + adapter.refresh(); + update(adapter.currentSnapshot()); } @Override public synchronized void update(final long snapshotId) { - final List snapshots = adapter.listSnapshots(tableIdentifier); + adapter.refresh(); + final List snapshots = adapter.listSnapshots(); final Snapshot snapshot = snapshots.stream() .filter(s -> s.snapshotId() == snapshotId).findFirst() diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java index 3e9fa7d5fa5..9ec4eeb18b0 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java @@ -6,7 +6,7 @@ import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; -import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import io.deephaven.iceberg.util.IcebergTableAdapter; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; @@ -17,7 +17,7 @@ public abstract class IcebergTableLocationProviderBase locationFactory; - final IcebergCatalogAdapter adapter; + final IcebergTableAdapter adapter; final TableIdentifier tableIdentifier; public IcebergTableLocationProviderBase( @@ -25,7 +25,7 @@ public IcebergTableLocationProviderBase( @NotNull final IcebergBaseLayout locationKeyFinder, @NotNull final TableLocationFactory locationFactory, final boolean isRefreshing, - @Nullable final IcebergCatalogAdapter adapter, + @Nullable final IcebergTableAdapter adapter, @NotNull final TableIdentifier tableIdentifier) { super(tableKey, isRefreshing); this.locationKeyFinder = locationKeyFinder; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index 6c0ecaee6cf..6a396a52bd8 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -3,44 +3,20 @@ // package io.deephaven.iceberg.util; -import io.deephaven.api.util.NameValidator; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.locations.TableDataException; -import io.deephaven.engine.table.impl.locations.TableKey; -import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; -import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; -import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; -import io.deephaven.engine.updategraph.UpdateSourceRegistrar; -import io.deephaven.iceberg.layout.*; -import io.deephaven.engine.util.TableTools; -import io.deephaven.iceberg.layout.IcebergFlatLayout; -import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout; -import io.deephaven.iceberg.location.IcebergTableLocationFactory; -import io.deephaven.iceberg.location.IcebergTableLocationKey; -import io.deephaven.time.DateTimeUtils; import io.deephaven.util.annotations.VisibleForTesting; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.time.Instant; -import java.time.LocalDateTime; import java.util.*; -import java.util.stream.Collectors; public class IcebergCatalogAdapter { @@ -55,142 +31,15 @@ public class IcebergCatalogAdapter { ColumnDefinition.ofString("TableName"), ColumnDefinition.fromGenericType("TableIdentifierObject", TableIdentifier.class)); - @VisibleForTesting - static final TableDefinition SNAPSHOT_DEFINITION = TableDefinition.of( - ColumnDefinition.ofLong("Id"), - ColumnDefinition.ofTime("Timestamp"), - ColumnDefinition.ofString("Operation"), - ColumnDefinition.fromGenericType("Summary", Map.class), - ColumnDefinition.fromGenericType("SnapshotObject", Snapshot.class)); - private final Catalog catalog; - private final FileIO fileIO; /** * Construct an IcebergCatalogAdapter from a catalog and file IO. */ - IcebergCatalogAdapter( - @NotNull final Catalog catalog, - @NotNull final FileIO fileIO) { + IcebergCatalogAdapter(@NotNull final Catalog catalog) { this.catalog = catalog; - this.fileIO = fileIO; - } - - /** - * Create a single {@link TableDefinition} from a given Schema, PartitionSpec, and TableDefinition. Takes into - * account {@link Map<> column rename instructions} - * - * @param schema The schema of the table. - * @param partitionSpec The partition specification of the table. - * @param userTableDef The table definition. - * @param columnRename The map for renaming columns. - * @return The generated TableDefinition. - */ - private static TableDefinition fromSchema( - @NotNull final Schema schema, - @NotNull final PartitionSpec partitionSpec, - @Nullable final TableDefinition userTableDef, - @NotNull final Map columnRename) { - - final Set columnNames = userTableDef != null - ? userTableDef.getColumnNameSet() - : null; - - final Set partitionNames = - partitionSpec.fields().stream() - .map(PartitionField::name) - .map(colName -> columnRename.getOrDefault(colName, colName)) - .collect(Collectors.toSet()); - - final List> columns = new ArrayList<>(); - - for (final Types.NestedField field : schema.columns()) { - final String name = columnRename.getOrDefault(field.name(), field.name()); - // Skip columns that are not in the provided table definition. - if (columnNames != null && !columnNames.contains(name)) { - continue; - } - final Type type = field.type(); - final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); - final ColumnDefinition column; - if (partitionNames.contains(name)) { - column = ColumnDefinition.of(name, qstType).withPartitioning(); - } else { - column = ColumnDefinition.of(name, qstType); - } - columns.add(column); - } - - final TableDefinition icebergTableDef = TableDefinition.of(columns); - if (userTableDef == null) { - return icebergTableDef; - } - - // If the user supplied a table definition, make sure it's fully compatible. - final TableDefinition tableDef = icebergTableDef.checkCompatibility(userTableDef); - - // Ensure that the user has not marked non-partitioned columns as partitioned. - final Set userPartitionColumns = userTableDef.getPartitioningColumns().stream() - .map(ColumnDefinition::getName) - .collect(Collectors.toSet()); - final Set partitionColumns = tableDef.getPartitioningColumns().stream() - .map(ColumnDefinition::getName) - .collect(Collectors.toSet()); - - // The working partitioning column set must be a super-set of the user-supplied set. - if (!partitionColumns.containsAll(userPartitionColumns)) { - final Set invalidColumns = new HashSet<>(userPartitionColumns); - invalidColumns.removeAll(partitionColumns); - - throw new TableDataException("The following columns are not partitioned in the Iceberg table: " + - invalidColumns); - } - return tableDef; } - /** - * Convert an Iceberg data type to a Deephaven type. - * - * @param icebergType The Iceberg data type to be converted. - * @return The converted Deephaven type. - */ - static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type icebergType) { - final Type.TypeID typeId = icebergType.typeId(); - switch (typeId) { - case BOOLEAN: - return io.deephaven.qst.type.Type.booleanType().boxedType(); - case DOUBLE: - return io.deephaven.qst.type.Type.doubleType(); - case FLOAT: - return io.deephaven.qst.type.Type.floatType(); - case INTEGER: - return io.deephaven.qst.type.Type.intType(); - case LONG: - return io.deephaven.qst.type.Type.longType(); - case STRING: - return io.deephaven.qst.type.Type.stringType(); - case TIMESTAMP: - final Types.TimestampType timestampType = (Types.TimestampType) icebergType; - return timestampType.shouldAdjustToUTC() - ? io.deephaven.qst.type.Type.find(Instant.class) - : io.deephaven.qst.type.Type.find(LocalDateTime.class); - case DATE: - return io.deephaven.qst.type.Type.find(java.time.LocalDate.class); - case TIME: - return io.deephaven.qst.type.Type.find(java.time.LocalTime.class); - case DECIMAL: - return io.deephaven.qst.type.Type.find(java.math.BigDecimal.class); - case FIXED: // Fall through - case BINARY: - return io.deephaven.qst.type.Type.find(byte[].class); - case UUID: // Fall through - case STRUCT: // Fall through - case LIST: // Fall through - case MAP: // Fall through - default: - throw new TableDataException("Unsupported iceberg column type " + typeId.name()); - } - } /** * List all {@link Namespace namespaces} in the catalog. This method is only supported if the catalog implements @@ -225,8 +74,8 @@ public List listNamespaces(@NotNull final Namespace namespace) { * * @return A {@link Table table} of all namespaces. */ - public Table listNamespacesAsTable() { - return listNamespacesAsTable(Namespace.empty()); + public Table namespaces() { + return namespaces(Namespace.empty()); } /** @@ -235,7 +84,7 @@ public Table listNamespacesAsTable() { * * @return A {@link Table table} of all namespaces. */ - public Table listNamespacesAsTable(@NotNull final Namespace namespace) { + public Table namespaces(@NotNull final Namespace namespace) { final List namespaces = listNamespaces(namespace); final long size = namespaces.size(); @@ -269,8 +118,8 @@ public Table listNamespacesAsTable(@NotNull final Namespace namespace) { * @return A {@link Table table} of all namespaces. */ @SuppressWarnings("unused") - public Table listNamespacesAsTable(@NotNull final String... namespace) { - return listNamespacesAsTable(Namespace.of(namespace)); + public Table namespaces(@NotNull final String... namespace) { + return namespaces(Namespace.of(namespace)); } /** @@ -290,7 +139,7 @@ public List listTables(@NotNull final Namespace namespace) { * @param namespace The namespace from which to gather the tables * @return A list of all tables in the given namespace. */ - public Table listTablesAsTable(@NotNull final Namespace namespace) { + public Table tables(@NotNull final Namespace namespace) { final List tableIdentifiers = listTables(namespace); final long size = tableIdentifiers.size(); @@ -322,8 +171,35 @@ public Table listTablesAsTable(@NotNull final Namespace namespace) { return new QueryTable(TABLES_DEFINITION, RowSetFactory.flat(size).toTracking(), columnSourceMap); } - public Table listTablesAsTable(@NotNull final String... namespace) { - return listTablesAsTable(Namespace.of(namespace)); + public Table tables(@NotNull final String... namespace) { + return tables(Namespace.of(namespace)); + } + + /** + * Load an Iceberg table from the catalog. + * + * @param tableIdentifier The identifier of the table to load. + * @return The {@link IcebergTableAdapter table adapter} for the Iceberg table. + */ + public IcebergTableAdapter loadTable(@NotNull final TableIdentifier tableIdentifier) { + // Load the table from the catalog. + final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); + if (table == null) { + throw new IllegalArgumentException("Table not found: " + tableIdentifier); + } + return new IcebergTableAdapter(tableIdentifier, table); + } + + /** + * List all {@link Snapshot snapshots} of a given Iceberg table. + * + * @param tableIdentifier The identifier of the table from which to gather snapshots. + * @return A list of all snapshots of the given table. + */ + @Deprecated(forRemoval = true) + public List listSnapshots(@NotNull final String tableIdentifier) { + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.listSnapshots(); } /** @@ -332,10 +208,10 @@ public Table listTablesAsTable(@NotNull final String... namespace) { * @param tableIdentifier The identifier of the table from which to gather snapshots. * @return A list of all snapshots of the given table. */ + @Deprecated(forRemoval = true) public List listSnapshots(@NotNull final TableIdentifier tableIdentifier) { - final List snapshots = new ArrayList<>(); - catalog.loadTable(tableIdentifier).snapshots().forEach(snapshots::add); - return snapshots; + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.listSnapshots(); } /** @@ -373,46 +249,10 @@ public List listSnapshots(@NotNull final TableIdentifier tableIdentifi * @param tableIdentifier The identifier of the table from which to gather snapshots. * @return A list of all tables in the given namespace. */ - public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier) { - final List snapshots = listSnapshots(tableIdentifier); - final long size = snapshots.size(); - - // Create and return a table containing the namespaces as strings - final Map> columnSourceMap = new LinkedHashMap<>(); - - // Create the column source(s) - final long[] idArr = new long[(int) size]; - columnSourceMap.put("Id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr, long.class, null)); - - final long[] timestampArr = new long[(int) size]; - columnSourceMap.put("Timestamp", - InMemoryColumnSource.getImmutableMemoryColumnSource(timestampArr, Instant.class, null)); - - final String[] operatorArr = new String[(int) size]; - columnSourceMap.put("Operation", - InMemoryColumnSource.getImmutableMemoryColumnSource(operatorArr, String.class, null)); - - final Map[] summaryArr = new Map[(int) size]; - columnSourceMap.put("Summary", - InMemoryColumnSource.getImmutableMemoryColumnSource(summaryArr, Map.class, null)); - - final Snapshot[] snapshotArr = new Snapshot[(int) size]; - columnSourceMap.put("SnapshotObject", - InMemoryColumnSource.getImmutableMemoryColumnSource(snapshotArr, Snapshot.class, null)); - - // Populate the column source(s) - for (int i = 0; i < size; i++) { - final Snapshot snapshot = snapshots.get(i); - idArr[i] = snapshot.snapshotId(); - // Provided as millis from epoch, convert to nanos - timestampArr[i] = DateTimeUtils.millisToNanos(snapshot.timestampMillis()); - operatorArr[i] = snapshot.operation(); - summaryArr[i] = snapshot.summary(); - snapshotArr[i] = snapshot; - } - - // Create and return the table - return new QueryTable(SNAPSHOT_DEFINITION, RowSetFactory.flat(size).toTracking(), columnSourceMap); + @Deprecated(forRemoval = true) + public Table snapshots(@NotNull final TableIdentifier tableIdentifier) { + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.snapshots(); } /** @@ -422,77 +262,9 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier * @param tableIdentifier The identifier of the table from which to gather snapshots. * @return A list of all tables in the given namespace. */ - public Table listSnapshotsAsTable(@NotNull final String tableIdentifier) { - return listSnapshotsAsTable(TableIdentifier.parse(tableIdentifier)); - } - - /** - * Retrieve a specific {@link Snapshot snapshot} of an Iceberg table. - * - * @param tableIdentifier The identifier of the table from which to load the snapshot. - * @param snapshotId The identifier of the snapshot to load. - * - * @return An Optional containing the requested snapshot if it exists. - */ - private Optional getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long snapshotId) { - return listSnapshots(tableIdentifier).stream() - .filter(snapshot -> snapshot.snapshotId() == snapshotId) - .findFirst(); - } - - /** - * Get the current {@link Snapshot snapshot} of a given Iceberg table. - * - * @param tableIdentifier The identifier of the table. - * @return The current snapshot of the table. - */ - public Snapshot getCurrentSnapshot(@NotNull final TableIdentifier tableIdentifier) { - final List snapshots = listSnapshots(tableIdentifier); - if (snapshots.isEmpty()) { - throw new IllegalStateException("No snapshots found for table " + tableIdentifier); - } - return snapshots.get(snapshots.size() - 1); - } - - /** - * Get a legalized column rename map from a table schema and user instructions. - */ - private Map getRenameColumnMap( - @NotNull final org.apache.iceberg.Table table, - @NotNull final Schema schema, - @NotNull final IcebergInstructions instructions) { - - final Set takenNames = new HashSet<>(); - - // Map all the column names in the schema to their legalized names. - final Map legalizedColumnRenames = new HashMap<>(); - - // Validate user-supplied names meet legalization instructions - for (final Map.Entry entry : instructions.columnRenames().entrySet()) { - final String destinationName = entry.getValue(); - if (!NameValidator.isValidColumnName(destinationName)) { - throw new TableDataException( - String.format("%s - invalid column name provided (%s)", table, destinationName)); - } - // Add these renames to the legalized list. - legalizedColumnRenames.put(entry.getKey(), destinationName); - takenNames.add(destinationName); - } - - for (final Types.NestedField field : schema.columns()) { - final String name = field.name(); - // Do we already have a valid rename for this column from the user or a partitioned column? - if (!legalizedColumnRenames.containsKey(name)) { - final String legalizedName = - NameValidator.legalizeColumnName(name, s -> s.replace(" ", "_"), takenNames); - if (!legalizedName.equals(name)) { - legalizedColumnRenames.put(name, legalizedName); - takenNames.add(legalizedName); - } - } - } - - return legalizedColumnRenames; + @Deprecated(forRemoval = true) + public Table snapshots(@NotNull final String tableIdentifier) { + return snapshots(TableIdentifier.parse(tableIdentifier)); } /** @@ -503,12 +275,12 @@ private Map getRenameColumnMap( * @param instructions The instructions for customizations while reading * @return The table definition */ + @Deprecated(forRemoval = true) public TableDefinition getTableDefinition( @NotNull final String tableIdentifier, @Nullable final IcebergInstructions instructions) { - final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); - // Load the table from the catalog. - return getTableDefinition(tableId, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definition(instructions); } /** @@ -519,11 +291,12 @@ public TableDefinition getTableDefinition( * @param instructions The instructions for customizations while reading * @return The table definition */ + @Deprecated(forRemoval = true) public TableDefinition getTableDefinition( @NotNull final TableIdentifier tableIdentifier, @Nullable final IcebergInstructions instructions) { - // Load the table from the catalog. - return getTableDefinitionInternal(tableIdentifier, null, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definition(instructions); } /** @@ -535,19 +308,13 @@ public TableDefinition getTableDefinition( * @param instructions The instructions for customizations while reading * @return The table definition */ + @Deprecated(forRemoval = true) public TableDefinition getTableDefinition( @NotNull final String tableIdentifier, final long snapshotId, @Nullable final IcebergInstructions instructions) { - final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); - - // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = - getSnapshot(tableId, snapshotId).orElseThrow(() -> new IllegalArgumentException( - "Snapshot with id " + snapshotId + " not found for table " + tableId)); - - // Load the table from the catalog. - return getTableDefinition(tableId, tableSnapshot, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definition(snapshotId, instructions); } /** @@ -559,12 +326,13 @@ public TableDefinition getTableDefinition( * @param instructions The instructions for customizations while reading * @return The table definition */ + @Deprecated(forRemoval = true) public TableDefinition getTableDefinition( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { - // Load the table from the catalog. - return getTableDefinitionInternal(tableIdentifier, tableSnapshot, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definition(tableSnapshot, instructions); } /** @@ -575,11 +343,12 @@ public TableDefinition getTableDefinition( * @param instructions The instructions for customizations while reading * @return The table definition as a Deephaven table */ + @Deprecated(forRemoval = true) public Table getTableDefinitionTable( @NotNull final String tableIdentifier, @Nullable final IcebergInstructions instructions) { - final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); - return getTableDefinitionTable(tableId, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definitionTable(instructions); } /** @@ -590,11 +359,12 @@ public Table getTableDefinitionTable( * @param instructions The instructions for customizations while reading * @return The table definition as a Deephaven table */ + @Deprecated(forRemoval = true) public Table getTableDefinitionTable( @NotNull final TableIdentifier tableIdentifier, @Nullable final IcebergInstructions instructions) { - final TableDefinition definition = getTableDefinition(tableIdentifier, instructions); - return TableTools.metaTable(definition); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definitionTable(instructions); } /** @@ -606,18 +376,13 @@ public Table getTableDefinitionTable( * @param instructions The instructions for customizations while reading * @return The table definition as a Deephaven table */ + @Deprecated(forRemoval = true) public Table getTableDefinitionTable( @NotNull final String tableIdentifier, final long snapshotId, @Nullable final IcebergInstructions instructions) { - final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); - - // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = - getSnapshot(tableId, snapshotId).orElseThrow(() -> new IllegalArgumentException( - "Snapshot with id " + snapshotId + " not found for table " + tableId)); - - return getTableDefinitionTable(tableId, tableSnapshot, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definitionTable(snapshotId, instructions); } /** @@ -629,35 +394,13 @@ public Table getTableDefinitionTable( * @param instructions The instructions for customizations while reading * @return The table definition as a Deephaven table */ + @Deprecated(forRemoval = true) public Table getTableDefinitionTable( @NotNull final TableIdentifier tableIdentifier, @Nullable final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { - final TableDefinition definition = getTableDefinition(tableIdentifier, tableSnapshot, instructions); - return TableTools.metaTable(definition); - } - - /** - * Internal method to create a {@link TableDefinition} from the table schema, snapshot and user instructions. - */ - private TableDefinition getTableDefinitionInternal( - @NotNull final TableIdentifier tableIdentifier, - @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { - final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); - if (table == null) { - throw new IllegalArgumentException("Table not found: " + tableIdentifier); - } - - final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); - final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema(); - - final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; - - return fromSchema(schema, - table.spec(), - userInstructions.tableDefinition().orElse(null), - getRenameColumnMap(table, schema, userInstructions)); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definitionTable(tableSnapshot, instructions); } /** @@ -668,10 +411,12 @@ private TableDefinition getTableDefinitionInternal( * @return The loaded table */ @SuppressWarnings("unused") + @Deprecated(forRemoval = true) public IcebergTable readTable( @NotNull final TableIdentifier tableIdentifier, @Nullable final IcebergInstructions instructions) { - return readTableInternal(tableIdentifier, null, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.table(instructions); } /** @@ -682,10 +427,12 @@ public IcebergTable readTable( * @return The loaded table */ @SuppressWarnings("unused") + @Deprecated(forRemoval = true) public IcebergTable readTable( @NotNull final String tableIdentifier, @Nullable final IcebergInstructions instructions) { - return readTable(TableIdentifier.parse(tableIdentifier), instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.table(instructions); } /** @@ -697,16 +444,13 @@ public IcebergTable readTable( * @return The loaded table */ @SuppressWarnings("unused") + @Deprecated(forRemoval = true) public IcebergTable readTable( @NotNull final TableIdentifier tableIdentifier, final long tableSnapshotId, @Nullable final IcebergInstructions instructions) { - // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = - getSnapshot(tableIdentifier, tableSnapshotId).orElseThrow(() -> new IllegalArgumentException( - "Snapshot with id " + tableSnapshotId + " not found for table " + tableIdentifier)); - - return readTableInternal(tableIdentifier, tableSnapshot, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.table(tableSnapshotId, instructions); } /** @@ -718,11 +462,13 @@ public IcebergTable readTable( * @return The loaded table */ @SuppressWarnings("unused") + @Deprecated(forRemoval = true) public IcebergTable readTable( @NotNull final String tableIdentifier, final long tableSnapshotId, @Nullable final IcebergInstructions instructions) { - return readTable(TableIdentifier.parse(tableIdentifier), tableSnapshotId, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.table(tableSnapshotId, instructions); } /** @@ -734,98 +480,24 @@ public IcebergTable readTable( * @return The loaded table */ @SuppressWarnings("unused") + @Deprecated(forRemoval = true) public IcebergTable readTable( @NotNull final TableIdentifier tableIdentifier, @NotNull final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { - return readTableInternal(tableIdentifier, tableSnapshot, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.table(tableSnapshot, instructions); } - private IcebergTable readTableInternal( - @NotNull final TableIdentifier tableIdentifier, - @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { - // Load the table from the catalog. - final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier); - if (table == null) { - throw new IllegalArgumentException("Table not found: " + tableIdentifier); - } - - // Do we want the latest or a specific snapshot? - final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); - final Schema schema = table.schemas().get(snapshot.schemaId()); - - // Load the partitioning schema. - final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); - - // Get default instructions if none are provided - final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; - - // Get the user supplied table definition. - final TableDefinition userTableDef = userInstructions.tableDefinition().orElse(null); - - // Map all the column names in the schema to their legalized names. - final Map legalizedColumnRenames = getRenameColumnMap(table, schema, userInstructions); - - // Get the table definition from the schema (potentially limited by the user supplied table definition and - // applying column renames). - final TableDefinition tableDef = fromSchema(schema, partitionSpec, userTableDef, legalizedColumnRenames); - - final IcebergBaseLayout keyFinder; - - if (partitionSpec.isUnpartitioned()) { - // Create the flat layout location key finder - keyFinder = new IcebergFlatLayout(tableDef, table, snapshot, fileIO, userInstructions); - } else { - // Create the partitioning column location key finder - keyFinder = new IcebergKeyValuePartitionedLayout(tableDef, table, snapshot, fileIO, partitionSpec, - userInstructions); - } - - if (instructions == null - || instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) { - final IcebergTableLocationProviderBase locationProvider = - new IcebergStaticTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(), - tableIdentifier); - - return new IcebergTableImpl( - tableDef, - tableIdentifier.toString(), - RegionedTableComponentFactoryImpl.INSTANCE, - locationProvider, - null); - } - - final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); - final IcebergTableLocationProviderBase locationProvider; - - if (instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) { - locationProvider = new IcebergManualRefreshTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(), - this, - tableIdentifier); - } else { - locationProvider = new IcebergAutoRefreshTableLocationProvider<>( - StandaloneTableKey.getInstance(), - keyFinder, - new IcebergTableLocationFactory(), - TableDataRefreshService.getSharedRefreshService(), - instructions.updateMode().autoRefreshMs(), - this, - tableIdentifier); - } - - return new IcebergTableImpl( - tableDef, - tableIdentifier.toString(), - RegionedTableComponentFactoryImpl.INSTANCE, - locationProvider, - updateSourceRegistrar); + /** + * Load an Iceberg table from the catalog. + * + * @param tableIdentifier The identifier of the table to load. + * @return The {@link IcebergTableAdapter table adapter} for the Iceberg table. + */ + @Deprecated(forRemoval = true) + public IcebergTableAdapter loadTable(final String tableIdentifier) { + return loadTable(TableIdentifier.parse(tableIdentifier)); } /** diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java index a0ae36a6eaa..c9d4b031354 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java @@ -11,7 +11,6 @@ public interface IcebergTable extends Table { /** * Update the table with the latest snapshot from the catalog. */ - @SuppressWarnings("unused") void update(); /** @@ -21,7 +20,6 @@ public interface IcebergTable extends Table { * * @param snapshotId The identifier of the snapshot to use when updating the table. */ - @SuppressWarnings("unused") void update(final long snapshotId); /** @@ -30,6 +28,5 @@ public interface IcebergTable extends Table { * * @param snapshot The snapshot to use when updating the table. */ - @SuppressWarnings("unused") void update(final @NotNull Snapshot snapshot); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java new file mode 100644 index 00000000000..4f1971d0dea --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -0,0 +1,587 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.api.util.NameValidator; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.TableKey; +import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; +import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.engine.util.TableTools; +import io.deephaven.iceberg.layout.*; +import io.deephaven.iceberg.location.IcebergTableLocationFactory; +import io.deephaven.iceberg.location.IcebergTableLocationKey; +import io.deephaven.time.DateTimeUtils; +import io.deephaven.util.annotations.VisibleForTesting; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.*; +import java.util.stream.Collectors; + +/** + * This class manages an Iceberg {@link org.apache.iceberg.Table table} and provides methods to interact with it. + */ +public class IcebergTableAdapter { + @VisibleForTesting + static final TableDefinition SNAPSHOT_DEFINITION = TableDefinition.of( + ColumnDefinition.ofLong("Id"), + ColumnDefinition.ofTime("Timestamp"), + ColumnDefinition.ofString("Operation"), + ColumnDefinition.fromGenericType("Summary", Map.class), + ColumnDefinition.fromGenericType("SnapshotObject", Snapshot.class)); + + private final org.apache.iceberg.Table table; + private final TableIdentifier tableIdentifier; + + public IcebergTableAdapter( + final TableIdentifier tableIdentifier, + final org.apache.iceberg.Table table) { + this.table = table; + this.tableIdentifier = tableIdentifier; + } + + /** + * Get the current {@link Snapshot snapshot} of a given Iceberg table. + * + * @return The current snapshot of the table. + */ + public Snapshot currentSnapshot() { + final List snapshots = listSnapshots(); + if (snapshots.isEmpty()) { + throw new IllegalStateException("No snapshots found for table " + tableIdentifier); + } + return snapshots.get(snapshots.size() - 1); + } + + /** + * List all {@link Snapshot snapshots} of the Iceberg table. + * + * @return A list of all snapshots of the given table. + */ + public List listSnapshots() { + final List snapshots = new ArrayList<>(); + table.snapshots().forEach(snapshots::add); + return snapshots; + } + + /** + * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting + * table will be static and contain the following columns: + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
Column NameDescription
IdThe snapshot identifier (can be used for updating the table or loading a specific snapshot)
TimestampMsThe timestamp of the snapshot
OperationThe data operation that created this snapshot
SummaryAdditional information about the snapshot from the Iceberg metadata
SnapshotObjectA Java object containing the Iceberg API snapshot
+ * + * @return A Table containing a list of all tables in the given namespace. + */ + public Table snapshots() { + final List snapshots = listSnapshots(); + final long size = snapshots.size(); + + // Create and return a table containing the namespaces as strings + final Map> columnSourceMap = new LinkedHashMap<>(); + + // Create the column source(s) + final long[] idArr = new long[(int) size]; + columnSourceMap.put("Id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr, long.class, null)); + + final long[] timestampArr = new long[(int) size]; + columnSourceMap.put("Timestamp", + InMemoryColumnSource.getImmutableMemoryColumnSource(timestampArr, Instant.class, null)); + + final String[] operatorArr = new String[(int) size]; + columnSourceMap.put("Operation", + InMemoryColumnSource.getImmutableMemoryColumnSource(operatorArr, String.class, null)); + + final Map[] summaryArr = new Map[(int) size]; + columnSourceMap.put("Summary", + InMemoryColumnSource.getImmutableMemoryColumnSource(summaryArr, Map.class, null)); + + final Snapshot[] snapshotArr = new Snapshot[(int) size]; + columnSourceMap.put("SnapshotObject", + InMemoryColumnSource.getImmutableMemoryColumnSource(snapshotArr, Snapshot.class, null)); + + // Populate the column source(s) + for (int i = 0; i < size; i++) { + final Snapshot snapshot = snapshots.get(i); + idArr[i] = snapshot.snapshotId(); + // Provided as millis from epoch, convert to nanos + timestampArr[i] = DateTimeUtils.millisToNanos(snapshot.timestampMillis()); + operatorArr[i] = snapshot.operation(); + summaryArr[i] = snapshot.summary(); + snapshotArr[i] = snapshot; + } + + // Create and return the table + return new QueryTable(SNAPSHOT_DEFINITION, RowSetFactory.flat(size).toTracking(), columnSourceMap); + } + + /** + * Retrieve a specific {@link Snapshot snapshot} of an Iceberg table. + * + * @param snapshotId The identifier of the snapshot to load. + * + * @return An Optional containing the requested snapshot if it exists. + */ + private Optional snapshot(final long snapshotId) { + return listSnapshots().stream() + .filter(snapshot -> snapshot.snapshotId() == snapshotId) + .findFirst(); + } + + /** + * Return {@link TableDefinition table definition}. + * + * @return The table definition + */ + public TableDefinition definition() { + // Load the table from the catalog. + return definition(null, null); + } + + /** + * Return {@link TableDefinition table definition} with optional instructions for customizations while reading. + * + * @param instructions The instructions for customizations while reading (or null for default instructions) + * @return The table definition + */ + public TableDefinition definition(@Nullable final IcebergInstructions instructions) { + // Load the table from the catalog. + return definition(null, instructions); + } + + /** + * Return {@link TableDefinition table definition} for the Iceberg table and snapshot id, with optional instructions + * for customizations while reading. + * + * @param snapshotId The identifier of the snapshot to load + * @param instructions The instructions for customizations while reading (or null for default instructions) + * @return The table definition + */ + public TableDefinition definition( + final long snapshotId, + @Nullable final IcebergInstructions instructions) { + + // Find the snapshot with the given snapshot id + final Snapshot tableSnapshot = + snapshot(snapshotId).orElseThrow(() -> new IllegalArgumentException( + "Snapshot with id " + snapshotId + " not found for table " + tableIdentifier)); + + // Load the table from the catalog. + return definition(tableSnapshot, instructions); + } + + /** + * Return {@link TableDefinition table definition} for the Iceberg table and snapshot, with optional instructions + * for customizations while reading. + * + * @param tableSnapshot The snapshot to load + * @param instructions The instructions for customizations while reading (or null for default instructions) + * @return The table definition + */ + public TableDefinition definition( + @Nullable final Snapshot tableSnapshot, + @Nullable final IcebergInstructions instructions) { + final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); + final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema(); + + final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; + + return fromSchema(schema, + table.spec(), + userInstructions.tableDefinition().orElse(null), + getRenameColumnMap(table, schema, userInstructions)); + } + + /** + * Return {@link Table table} containing the {@link TableDefinition definition} of the Iceberg table. + * + * @return The table definition as a Deephaven table + */ + public Table definitionTable() { + return definitionTable(null, null); + } + + /** + * Return {@link Table table} containing the {@link TableDefinition definition} of the Iceberg table, with optional + * instructions for customizations while reading. + * + * @param instructions The instructions for customizations while reading + * @return The table definition as a Deephaven table + */ + public Table definitionTable(@Nullable final IcebergInstructions instructions) { + return definitionTable(null, instructions); + } + + /** + * Return {@link Table table} containing the {@link TableDefinition definition} of a given Iceberg table and + * snapshot id, with optional instructions for customizations while reading. + * + * @param snapshotId The identifier of the snapshot to load + * @param instructions The instructions for customizations while reading (or null for default instructions) + * @return The table definition as a Deephaven table + */ + public Table definitionTable( + final long snapshotId, + @Nullable final IcebergInstructions instructions) { + + // Find the snapshot with the given snapshot id + final Snapshot tableSnapshot = + snapshot(snapshotId).orElseThrow(() -> new IllegalArgumentException( + "Snapshot with id " + snapshotId + " not found for table " + tableIdentifier)); + + return definitionTable(tableSnapshot, instructions); + } + + /** + * Return {@link Table table} containing the {@link TableDefinition definition} of a given Iceberg table and + * snapshot id, with optional instructions for customizations while reading. + * + * @param tableSnapshot The snapshot to load + * @param instructions The instructions for customizations while reading (or null for default instructions) + * @return The table definition as a Deephaven table + */ + public Table definitionTable( + @Nullable final Snapshot tableSnapshot, + @Nullable final IcebergInstructions instructions) { + final TableDefinition definition = definition(tableSnapshot, instructions); + return TableTools.metaTable(definition); + } + + /** + * Read the latest snapshot of an Iceberg table from the Iceberg catalog as a Deephaven {@link Table table}. + * + * @param instructions The instructions for customizations while reading (or null for default instructions) + * @return The loaded table + */ + @SuppressWarnings("unused") + public IcebergTable table(@Nullable final IcebergInstructions instructions) { + return table(null, instructions); + } + + /** + * Read a snapshot of an Iceberg table from the Iceberg catalog. + * + * @param tableSnapshotId The snapshot id to load + * @param instructions The instructions for customizations while reading (or null for default instructions) + * @return The loaded table + */ + @SuppressWarnings("unused") + public IcebergTable table(final long tableSnapshotId, @Nullable final IcebergInstructions instructions) { + // Find the snapshot with the given snapshot id + final Snapshot tableSnapshot = + snapshot(tableSnapshotId).orElseThrow(() -> new IllegalArgumentException( + "Snapshot with id " + tableSnapshotId + " not found for table " + tableIdentifier)); + + return table(tableSnapshot, instructions); + } + + /** + * Read a snapshot of an Iceberg table from the Iceberg catalog. + * + * @param tableSnapshot The snapshot id to load + * @param instructions The instructions for customizations while reading + * @return The loaded table + */ + public IcebergTable table( + @Nullable final Snapshot tableSnapshot, + @Nullable final IcebergInstructions instructions) { + + // Do we want the latest or a specific snapshot? + final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); + final Schema schema = table.schemas().get(snapshot.schemaId()); + + // Load the partitioning schema. + final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); + + // Get default instructions if none are provided + final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; + + // Get the user supplied table definition. + final TableDefinition userTableDef = userInstructions.tableDefinition().orElse(null); + + // Map all the column names in the schema to their legalized names. + final Map legalizedColumnRenames = getRenameColumnMap(table, schema, userInstructions); + + // Get the table definition from the schema (potentially limited by the user supplied table definition and + // applying column renames). + final TableDefinition tableDef = fromSchema(schema, partitionSpec, userTableDef, legalizedColumnRenames); + + final IcebergBaseLayout keyFinder; + + if (partitionSpec.isUnpartitioned()) { + // Create the flat layout location key finder + keyFinder = new IcebergFlatLayout(this, snapshot, userInstructions); + } else { + // Create the partitioning column location key finder + keyFinder = new IcebergKeyValuePartitionedLayout(this, snapshot, partitionSpec, + userInstructions); + } + + if (instructions == null + || instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) { + final IcebergTableLocationProviderBase locationProvider = + new IcebergStaticTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + tableIdentifier); + + return new IcebergTableImpl( + tableDef, + tableIdentifier.toString(), + RegionedTableComponentFactoryImpl.INSTANCE, + locationProvider, + null); + } + + final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); + final IcebergTableLocationProviderBase locationProvider; + + if (instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) { + locationProvider = new IcebergManualRefreshTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + this, + tableIdentifier); + } else { + locationProvider = new IcebergAutoRefreshTableLocationProvider<>( + StandaloneTableKey.getInstance(), + keyFinder, + new IcebergTableLocationFactory(), + TableDataRefreshService.getSharedRefreshService(), + instructions.updateMode().autoRefreshMs(), + this, + tableIdentifier); + } + + return new IcebergTableImpl( + tableDef, + tableIdentifier.toString(), + RegionedTableComponentFactoryImpl.INSTANCE, + locationProvider, + updateSourceRegistrar); + } + + /** + * Refresh the table with the latest information from the Iceberg catalog, including new snapshots and schema. + */ + public void refresh() { + table.refresh(); + } + + /** + * Return the underlying Iceberg table. + */ + public org.apache.iceberg.Table icebergTable() { + return table; + } + + @Override + public String toString() { + return table.toString(); + } + + /** + * Get a legalized column rename map from a table schema and user instructions. + */ + private Map getRenameColumnMap( + @NotNull final org.apache.iceberg.Table table, + @NotNull final Schema schema, + @NotNull final IcebergInstructions instructions) { + + final Set takenNames = new HashSet<>(); + + // Map all the column names in the schema to their legalized names. + final Map legalizedColumnRenames = new HashMap<>(); + + // Validate user-supplied names meet legalization instructions + for (final Map.Entry entry : instructions.columnRenames().entrySet()) { + final String destinationName = entry.getValue(); + if (!NameValidator.isValidColumnName(destinationName)) { + throw new TableDataException( + String.format("%s - invalid column name provided (%s)", table, destinationName)); + } + // Add these renames to the legalized list. + legalizedColumnRenames.put(entry.getKey(), destinationName); + takenNames.add(destinationName); + } + + for (final Types.NestedField field : schema.columns()) { + final String name = field.name(); + // Do we already have a valid rename for this column from the user or a partitioned column? + if (!legalizedColumnRenames.containsKey(name)) { + final String legalizedName = + NameValidator.legalizeColumnName(name, s -> s.replace(" ", "_"), takenNames); + if (!legalizedName.equals(name)) { + legalizedColumnRenames.put(name, legalizedName); + takenNames.add(legalizedName); + } + } + } + + return legalizedColumnRenames; + } + + /** + * Create a single {@link TableDefinition} from a given Schema, PartitionSpec, and TableDefinition. Takes into + * account {@link Map<> column rename instructions} + * + * @param schema The schema of the table. + * @param partitionSpec The partition specification of the table. + * @param userTableDef The table definition. + * @param columnRename The map for renaming columns. + * @return The generated TableDefinition. + */ + private static TableDefinition fromSchema( + @NotNull final Schema schema, + @NotNull final PartitionSpec partitionSpec, + @Nullable final TableDefinition userTableDef, + @NotNull final Map columnRename) { + + final Set columnNames = userTableDef != null + ? userTableDef.getColumnNameSet() + : null; + + final Set partitionNames = + partitionSpec.fields().stream() + .map(PartitionField::name) + .map(colName -> columnRename.getOrDefault(colName, colName)) + .collect(Collectors.toSet()); + + final List> columns = new ArrayList<>(); + + for (final Types.NestedField field : schema.columns()) { + final String name = columnRename.getOrDefault(field.name(), field.name()); + // Skip columns that are not in the provided table definition. + if (columnNames != null && !columnNames.contains(name)) { + continue; + } + final Type type = field.type(); + final io.deephaven.qst.type.Type qstType = convertPrimitiveType(type); + final ColumnDefinition column; + if (partitionNames.contains(name)) { + column = ColumnDefinition.of(name, qstType).withPartitioning(); + } else { + column = ColumnDefinition.of(name, qstType); + } + columns.add(column); + } + + final TableDefinition icebergTableDef = TableDefinition.of(columns); + if (userTableDef == null) { + return icebergTableDef; + } + + // If the user supplied a table definition, make sure it's fully compatible. + final TableDefinition tableDef = icebergTableDef.checkCompatibility(userTableDef); + + // Ensure that the user has not marked non-partitioned columns as partitioned. + final Set userPartitionColumns = userTableDef.getPartitioningColumns().stream() + .map(ColumnDefinition::getName) + .collect(Collectors.toSet()); + final Set partitionColumns = tableDef.getPartitioningColumns().stream() + .map(ColumnDefinition::getName) + .collect(Collectors.toSet()); + + // The working partitioning column set must be a super-set of the user-supplied set. + if (!partitionColumns.containsAll(userPartitionColumns)) { + final Set invalidColumns = new HashSet<>(userPartitionColumns); + invalidColumns.removeAll(partitionColumns); + + throw new TableDataException("The following columns are not partitioned in the Iceberg table: " + + invalidColumns); + } + return tableDef; + } + + /** + * Convert an Iceberg data type to a Deephaven type. + * + * @param icebergType The Iceberg data type to be converted. + * @return The converted Deephaven type. + */ + static io.deephaven.qst.type.Type convertPrimitiveType(@NotNull final Type icebergType) { + final Type.TypeID typeId = icebergType.typeId(); + switch (typeId) { + case BOOLEAN: + return io.deephaven.qst.type.Type.booleanType().boxedType(); + case DOUBLE: + return io.deephaven.qst.type.Type.doubleType(); + case FLOAT: + return io.deephaven.qst.type.Type.floatType(); + case INTEGER: + return io.deephaven.qst.type.Type.intType(); + case LONG: + return io.deephaven.qst.type.Type.longType(); + case STRING: + return io.deephaven.qst.type.Type.stringType(); + case TIMESTAMP: + final Types.TimestampType timestampType = (Types.TimestampType) icebergType; + return timestampType.shouldAdjustToUTC() + ? io.deephaven.qst.type.Type.find(Instant.class) + : io.deephaven.qst.type.Type.find(LocalDateTime.class); + case DATE: + return io.deephaven.qst.type.Type.find(java.time.LocalDate.class); + case TIME: + return io.deephaven.qst.type.Type.find(java.time.LocalTime.class); + case DECIMAL: + return io.deephaven.qst.type.Type.find(java.math.BigDecimal.class); + case FIXED: // Fall through + case BINARY: + return io.deephaven.qst.type.Type.find(byte[].class); + case UUID: // Fall through + case STRUCT: // Fall through + case LIST: // Fall through + case MAP: // Fall through + default: + throw new TableDataException("Unsupported iceberg column type " + typeId.name()); + } + } +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java index 02e3db6e115..a73a55d40be 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java @@ -57,15 +57,4 @@ public void update(final long snapshotId) { public void update(final @NotNull Snapshot snapshot) { locationProvider.update(snapshot); } - - @Override - public int currentSnapshotId() { - // TODO: implement the TableAdapter - return 0; - } - - @Override - public Snapshot currentSnapshot() { - return null; - } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java index bcdda326dca..33fb26a3647 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java @@ -3,17 +3,16 @@ // package io.deephaven.iceberg.util; +import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.io.FileIO; /** * Tools for accessing tables in the Iceberg table format. */ public abstract class IcebergTools { @SuppressWarnings("unused") - public static IcebergCatalogAdapter createAdapter( - final Catalog catalog, - final FileIO fileIO) { - return new IcebergCatalogAdapter(catalog, fileIO); + public static IcebergCatalogAdapter createAdapter(final Catalog catalog) { + Configuration conf = new Configuration(); + return new IcebergCatalogAdapter(catalog); } } diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 623cccc6c16..79e0eb15796 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -15,6 +15,7 @@ _JIcebergInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergInstructions") _JIcebergUpdateMode = jpy.get_type("io.deephaven.iceberg.util.IcebergUpdateMode") _JIcebergCatalogAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergCatalogAdapter") +_JIcebergTableAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergTableAdapter") _JIcebergTable = jpy.get_type("io.deephaven.iceberg.util.IcebergTable") # IcebergToolsS3 is an optional library @@ -166,6 +167,79 @@ def j_object(self) -> jpy.JType: return self.j_table +class IcebergTableAdapter(JObjectWrapper): + """ + This class provides an interface for interacting with Iceberg tables. It allows listing snapshots, table + definitions, as well as reading Iceberg tables into Deephaven tables. + """ + j_object_type = _JIcebergTableAdapter or type(None) + + def __init__(self, j_object: _JIcebergTableAdapter): + self.j_table_adapter = j_object + + def snapshots(self) -> Table: + """ + Returns information on the snapshots of this table as a Deephaven table. The table contains the + following columns: + - `Id`: the snapshot identifier (can be used for updating the table or loading a specific snapshot). + - `TimestampMs`: the timestamp of the snapshot. + - `Operation`: the data operation that created this snapshot. + - `Summary`: additional information about this snapshot from the Iceberg metadata. + - `SnapshotObject`: a Java object containing the Iceberg API snapshot. + + Args: + table_identifier (str): the table from which to list snapshots. + + Returns: + a table containing the snapshot information. + """ + return Table(self.j_object.snapshots()) + + def definition(self, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> Table: + """ + Returns information on the tables in the specified namespace as a Deephaven table. + + Args: + namespace (str): the namespace from which to list tables. + + Returns: + a table containing the tables in the provided namespace. + """ + + if snapshot_id is not None: + return Table(self.j_object.definitionTable(snapshot_id, instructions)) + return Table(self.j_object.definitionTable(instructions)) + + def table(self, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> IcebergTable: + """ + Reads the table using the provided instructions. Optionally, a snapshot id can be provided to + read a specific snapshot of the table. + + Args: + table_identifier (str): the table to read. + instructions (Optional[IcebergInstructions]): the instructions for reading the table. These instructions + can include column renames, table definition, and specific data instructions for reading the data files + from the provider. If omitted, the table will be read with default instructions. + snapshot_id (Optional[int]): the snapshot id to read; if omitted the most recent snapshot will be selected. + + Returns: + Table: the table read from the catalog. + """ + + if instructions is not None: + instructions_object = instructions.j_object + else: + instructions_object = _JIcebergInstructions.DEFAULT + + if snapshot_id is not None: + return IcebergTable(self.j_object.table(snapshot_id, instructions_object)) + return IcebergTable(self.j_object.readTable(instructions_object)) + + @property + def j_object(self) -> jpy.JType: + return self.j_table_adapter + + class IcebergCatalogAdapter(JObjectWrapper): """ This class provides an interface for interacting with Iceberg catalogs. It allows listing namespaces, tables and @@ -190,10 +264,10 @@ def namespaces(self, namespace: Optional[str] = None) -> Table: """ if namespace is not None: - return Table(self.j_object.listNamespaces(namespace)) - return Table(self.j_object.listNamespacesAsTable()) + return Table(self.j_object.namespaces(namespace)) + return Table(self.j_object.namespaces()) - def tables(self, namespace: str) -> Table: + def tables(self, namespace: str) -> IcebergTableAdapter: """ Returns information on the tables in the specified namespace as a Deephaven table. @@ -204,12 +278,25 @@ def tables(self, namespace: str) -> Table: a table containing the tables in the provided namespace. """ - if namespace is not None: - return Table(self.j_object.listTablesAsTable(namespace)) - return Table(self.j_object.listTablesAsTable()) + return Table(self.j_object.tables(namespace)) + + def load_table(self, table_identifier: str) -> IcebergTable: + """ + Load the table from the catalog using the provided instructions. + + Args: + table_identifier (str): the table to read. + + Returns: + Table: the table read from the catalog. + """ + + return IcebergTableAdapter(self.j_object.loadTable(table_identifier)) def snapshots(self, table_identifier: str) -> Table: """ + NOTE: deprecated, use `load_table(table_identifier).snapshots()` instead. + Returns information on the snapshots of the specified table as a Deephaven table. The table contains the following columns: - `Id`: the snapshot identifier (can be used for updating the table or loading a specific snapshot). @@ -225,10 +312,12 @@ def snapshots(self, table_identifier: str) -> Table: a table containing the snapshot information. """ - return self.j_object.listSnapshotsAsTable(table_identifier) + return self.j_object.snapshots(table_identifier) def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> IcebergTable: """ + NOTE: deprecated, use `load_table(table_identifier).table()` instead. + Reads the table from the catalog using the provided instructions. Optionally, a snapshot id can be provided to read a specific snapshot of the table. From 68e45460b4690d9dd0544f49cd019856829fa043 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 30 Aug 2024 16:12:47 -0700 Subject: [PATCH 22/56] Incorporated external PR to update PartitioningColumnDataIndex for refreshing tables. --- .../regioned/PartitioningColumnDataIndex.java | 151 ++++++++++++------ .../regioned/RegionedColumnSourceManager.java | 23 +-- 2 files changed, 114 insertions(+), 60 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java index ae95d86b8ee..1fc5e540b88 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java @@ -6,12 +6,7 @@ import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderRandom; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.RowSetShiftData; -import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.Table; @@ -56,7 +51,7 @@ class PartitioningColumnDataIndex extends AbstractDataIndex { /** Provides fast lookup from keys to positions in the index table **/ private final TObjectIntHashMap keyPositionMap; - private final ModifiedColumnSet upstreamLocationModified; + private final ModifiedColumnSet upstreamKeyModified; private final ModifiedColumnSet upstreamRowSetModified; private final ModifiedColumnSet downstreamRowSetModified; @@ -65,7 +60,7 @@ class PartitioningColumnDataIndex extends AbstractDataIndex { * {@link RegionedColumnSourceManager} at a time when there cannot be any concurrent "refresh" behavior, and so we * can safely use the {@link RegionedColumnSourceManager#locationTable() location table} without snapshotting or * considering previous values. - * + * * @param keyColumnName The key column name * @param keySource The key source in the indexed table * @param columnSourceManager The column source manager that provides locations and region indexes @@ -113,7 +108,7 @@ class PartitioningColumnDataIndex extends AbstractDataIndex { if (locationTable.isRefreshing()) { // No need to track previous values; we mutate the index table's RowSets in-place, and we never move a key. indexTable.getRowSet().writableCast().initializePreviousValue(); - upstreamLocationModified = locationTable.newModifiedColumnSet(columnSourceManager.locationColumnName()); + upstreamKeyModified = locationTable.newModifiedColumnSet(keyColumnName); upstreamRowSetModified = locationTable.newModifiedColumnSet(columnSourceManager.rowSetColumnName()); downstreamRowSetModified = indexTable.newModifiedColumnSet(rowSetColumnName()); final TableUpdateListener tableListener = new BaseTable.ListenerImpl(String.format( @@ -126,7 +121,7 @@ public void onUpdate(@NotNull final TableUpdate upstream) { locationTable.addUpdateListener(tableListener); manage(indexTable); } else { - upstreamLocationModified = null; + upstreamKeyModified = null; upstreamRowSetModified = null; downstreamRowSetModified = null; } @@ -138,29 +133,30 @@ private synchronized void processUpdate( if (upstream.empty()) { return; } - if (upstream.removed().isNonempty()) { - throw new UnsupportedOperationException("Removed locations are not currently supported"); - } if (upstream.shifted().nonempty()) { throw new UnsupportedOperationException("Shifted locations are not currently supported"); } - if (upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(upstreamLocationModified)) { - throw new UnsupportedOperationException("Modified locations are not currently supported"); + if (upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(upstreamKeyModified)) { + throw new UnsupportedOperationException("Modified location keys are not currently supported"); } Assert.assertion(initializing || isRefreshing(), "initializing || isRefreshing()"); final int previousSize = keyPositionMap.size(); - final RowSetBuilderRandom modifiedBuilder = initializing ? null : RowSetFactory.builderRandom(); + final RowSetBuilderRandom modifiedPositionBuilder = initializing ? null : RowSetFactory.builderRandom(); - if (upstream.added().isNonempty()) { - upstream.added().forAllRowKeys((final long locationRowKey) -> handleKey( - locationRowKey, false, previousSize, modifiedBuilder)); + if (upstream.removed().isNonempty()) { + Assert.eqFalse(initializing, "initializing"); + upstream.removed().forAllRowKeys((final long locationRowKey) -> handleLocation( + locationRowKey, ChangeType.REMOVE, modifiedPositionBuilder)); } - if (upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(upstreamRowSetModified)) { Assert.eqFalse(initializing, "initializing"); - upstream.modified().forAllRowKeys((final long locationRowKey) -> handleKey( - locationRowKey, true, previousSize, modifiedBuilder)); + upstream.modified().forAllRowKeys((final long locationRowKey) -> handleLocation( + locationRowKey, ChangeType.MODIFY, modifiedPositionBuilder)); + } + if (upstream.added().isNonempty()) { + upstream.added().forAllRowKeys((final long locationRowKey) -> handleLocation( + locationRowKey, ChangeType.ADD, modifiedPositionBuilder)); } final int newSize = keyPositionMap.size(); @@ -172,40 +168,89 @@ private synchronized void processUpdate( return; } - // Send the downstream updates to any listeners of the index table - final WritableRowSet modified = modifiedBuilder.build(); + final WritableRowSet modified = modifiedPositionBuilder.build(); if (previousSize == newSize && modified.isEmpty()) { modified.close(); return; } + final RowSetBuilderSequential removedPositionsBuilder = RowSetFactory.builderSequential(); + final RowSetBuilderSequential resurrectedPositionsBuilder = RowSetFactory.builderSequential(); + modified.forAllRowKeys((final long pos) -> { + final RowSet indexRowSet = indexRowSetSource.get(pos); + // noinspection DataFlowIssue + if (indexRowSet.isEmpty()) { + removedPositionsBuilder.appendKey(pos); + } else if (indexRowSet.trackingCast().prev().isEmpty()) { + resurrectedPositionsBuilder.appendKey(pos); + } + }); + + final WritableRowSet added = RowSetFactory.fromRange(previousSize, newSize - 1); + final RowSet removed = removedPositionsBuilder.build(); + modified.remove(removed); + try (final RowSet resurrected = resurrectedPositionsBuilder.build()) { + added.insert(resurrected); + modified.remove(resurrected); + } + + // Send the downstream updates to any listeners of the index table final TableUpdate downstream = new TableUpdateImpl( - RowSetFactory.fromRange(previousSize, newSize - 1), - RowSetFactory.empty(), + added, + removed, modified, RowSetShiftData.EMPTY, modified.isNonempty() ? downstreamRowSetModified : ModifiedColumnSet.EMPTY); indexTable.notifyListeners(downstream); } - private void handleKey( + private enum ChangeType { + // @formatter:off + ADD("Added"), + REMOVE("Removed"), + MODIFY("Modified"); + // @formatter:on + + private final String actionLabel; + + ChangeType(@NotNull final String actionLabel) { + this.actionLabel = actionLabel; + } + } + + private void handleLocation( final long locationRowKey, - final boolean isModify, - final int previousSize, - @Nullable final RowSetBuilderRandom modifiedBuilder) { + @NotNull final ChangeType changeType, + @Nullable final RowSetBuilderRandom modifiedPositionBuilder) { final KEY_TYPE locationKey = locationTableKeySource.get(locationRowKey); final Object locationKeyReinterpreted = locationTableKeySourceReinterpreted.get(locationRowKey); - final RowSet regionRowSet = locationTableRowSetSource.get(locationRowKey); - if (regionRowSet == null) { - throw new IllegalStateException(String.format("Null row set found at location index %d", locationRowKey)); + + final RowSet currentRegionRowSet = changeType == ChangeType.REMOVE + ? null + : locationTableRowSetSource.get(locationRowKey); + final RowSet previousRegionRowSet = changeType == ChangeType.ADD + ? null + : locationTableRowSetSource.getPrev(locationRowKey); + + if (changeType != ChangeType.REMOVE && (currentRegionRowSet == null || currentRegionRowSet.isEmpty())) { + throw new IllegalStateException(String.format( + "%s partition (index=%d, key=%s): Unexpected null or empty current row set", + changeType.actionLabel, locationRowKey, locationKey)); + } + if (changeType != ChangeType.ADD && (previousRegionRowSet == null || previousRegionRowSet.isEmpty())) { + throw new IllegalStateException(String.format( + "%s partition (index=%d, key=%s): Unexpected null or empty previous row set", + changeType.actionLabel, locationRowKey, locationKey)); } - final long regionFirstRowKey = RegionedColumnSource.getFirstRowKey(Math.toIntExact(locationRowKey)); // Test using the (maybe) reinterpreted key final int pos = keyPositionMap.get(locationKeyReinterpreted); + + // Inserting a new bucket if (pos == KEY_NOT_FOUND) { - if (isModify) { - throw new IllegalStateException(String.format("Modified partition key %s not found", locationKey)); + if (changeType == ChangeType.REMOVE || changeType == ChangeType.MODIFY) { + throw new IllegalStateException(String.format("%s partition (index=%d, key=%s): Key not found", + changeType.actionLabel, locationRowKey, locationKey)); } final int addedKeyPos = keyPositionMap.size(); // Store the (maybe) reinterpreted key in the lookup hashmap. @@ -216,21 +261,27 @@ private void handleKey( indexKeySource.set(addedKeyPos, locationKey); indexRowSetSource.ensureCapacity(addedKeyPos + 1); - indexRowSetSource.set(addedKeyPos, regionRowSet.shift(regionFirstRowKey).toTracking()); - } else { - // noinspection DataFlowIssue - final WritableRowSet existingRowSet = indexRowSetSource.get(pos).writableCast(); - try (final WritableRowSet shiftedRowSet = regionRowSet.shift(regionFirstRowKey)) { - // We could assert that: - // 1. an added location is non-overlapping with the key's existing row set - // 2. a modified location's current row set is a superset of its previous row set - // 3. a modified location's previous row set is a subset of the key's existing row set - existingRowSet.insert(shiftedRowSet); - } + indexRowSetSource.set(addedKeyPos, currentRegionRowSet.copy().toTracking()); + return; + } - if (modifiedBuilder != null && pos < previousSize) { - modifiedBuilder.addKey(pos); - } + // Updating an existing bucket + // noinspection DataFlowIssue + final WritableRowSet existingRowSet = indexRowSetSource.get(pos).writableCast(); + // We _could_ assert that: + // 1. An added location is non-overlapping with the key's existing row set + // 2. A modified location's current row set is a superset of its previous row set (with existing RCSM) + // 3. A removed or modified location's previous row set is a subset of the key's existing row set + if (previousRegionRowSet != null) { + existingRowSet.remove(previousRegionRowSet); + } + if (currentRegionRowSet != null) { + existingRowSet.insert(currentRegionRowSet); + } + if (modifiedPositionBuilder != null) { + // Note that once done processing everything, we're going to adjust this to pull out transitions _from_ + // empty as adds and _to_ empty as removes. + modifiedPositionBuilder.addKey(pos); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 274f603835a..cb71d45053f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -35,6 +35,9 @@ import java.util.stream.Stream; import java.util.stream.StreamSupport; +import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK; +import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.getFirstRowKey; + /** * Manage column sources made up of regions in their own row key address space. */ @@ -369,10 +372,10 @@ private TableUpdateImpl update(final boolean initializing) { if (entry.pollUpdates(addedRowSetBuilder)) { // Changes were detected, update the row set in the table and mark the row/column as modified. /* - * Since TableLocationState.getRowSet() returns a copy(), we own entry.rowSetAtLastUpdate and can - * propagate it without making another copy(). + * We should consider adding an UpdateCommitter to close() the previous row sets for modified locations. + * This is not important for current implementations, since they always allocate new, flat RowSets. */ - rowSetSource.set(entry.regionIndex, entry.rowSetAtLastUpdate); + rowSetSource.set(entry.regionIndex, entry.rowSetAtLastUpdate.shift(getFirstRowKey(entry.regionIndex))); if (modifiedRegionBuilder != null) { modifiedRegionBuilder.appendKey(entry.regionIndex); } @@ -426,7 +429,7 @@ private TableUpdateImpl update(final boolean initializing) { wcs.set(entry.regionIndex, entry.location.getKey().getPartitionValue(key))); // @formatter:on locationSource.set(entry.regionIndex, entry.location); - rowSetSource.set(entry.regionIndex, entry.rowSetAtLastUpdate); + rowSetSource.set(entry.regionIndex, entry.rowSetAtLastUpdate.shift(getFirstRowKey(entry.regionIndex))); addedRegionBuilder.appendKey(entry.regionIndex); }); } @@ -584,14 +587,14 @@ private void processInitial(final RowSetBuilderSequential addedRowSetBuilder, fi Assert.neqNull(initialRowSet, "initialRowSet"); Assert.eqTrue(initialRowSet.isNonempty(), "initialRowSet.isNonempty()"); Assert.eqNull(rowSetAtLastUpdate, "rowSetAtLastUpdate"); - if (initialRowSet.lastRowKey() > RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK) { + if (initialRowSet.lastRowKey() > ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK) { throw new TableDataException(String.format( "Location %s has initial last key %#016X, larger than maximum supported key %#016X", location, initialRowSet.lastRowKey(), - RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK)); + ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK)); } - final long regionFirstKey = RegionedColumnSource.getFirstRowKey(regionIndex); + final long regionFirstKey = getFirstRowKey(regionIndex); initialRowSet.forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> addedRowSetBuilder .appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey)); @@ -644,11 +647,11 @@ private boolean pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) { // Nothing to do return false; } - if (updateRowSet.lastRowKey() > RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK) { + if (updateRowSet.lastRowKey() > ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK) { throw new TableDataException(String.format( "Location %s has updated last key %#016X, larger than maximum supported key %#016X", location, updateRowSet.lastRowKey(), - RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK)); + ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK)); } if (log.isDebugEnabled()) { @@ -657,7 +660,7 @@ private boolean pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) { .append(",TO:").append(updateRowSet.size()).endl(); } try (final RowSet addedRowSet = updateRowSet.minus(rowSetAtLastUpdate)) { - final long regionFirstKey = RegionedColumnSource.getFirstRowKey(regionIndex); + final long regionFirstKey = getFirstRowKey(regionIndex); addedRowSet.forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> addedRowSetBuilder .appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey)); } From 273f5c137829ccc5c94bb147fb832d11dd2f3e93 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 3 Sep 2024 15:28:43 -0700 Subject: [PATCH 23/56] Added additional snapshots with removes to IcebergToolsTest resources. --- .../iceberg/util/IcebergToolsTest.java | 110 ++++++++-- ...efb-4a24-a6a6-46b2b34b986c-0-00001.parquet | 3 + ...e7d-4830-aedb-14dab6fabdd6-0-00001.parquet | 3 - ...fc0-40f8-9349-fcd5763dd691-0-00001.parquet | 3 + ...f10-4e08-89ae-f1b1b578ce63-0-00001.parquet | 3 - ...bcf-44c9-8482-90d09621c2ee-0-00001.parquet | 3 + ...256-4a04-9248-d7c69bec4881-0-00001.parquet | 3 - ...5de-4aac-bafc-cf750f9f7f70-0-00001.parquet | 3 + ...b08-46a6-bd9d-c8008dc66519-0-00001.parquet | 3 + ...3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet | 3 - ...9c89-46e8-b0f7-b4ec128b0f5e.metadata.json} | 30 +-- ...5f28-4965-b390-ab426779517b.metadata.json} | 56 ++--- ...fa56-47ec-9639-079621849f5a.metadata.json} | 82 +++---- ...a6bf-4d78-b6f0-d927d58feb4c.metadata.json} | 106 ++++----- ...-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json | 186 ++++++++++++++++ ...-1647-44c6-930e-df3fe7867d8f.metadata.json | 206 ++++++++++++++++++ ...480b9-ac20-4445-b67e-7cd3f7ef27d0-m0.avro} | Bin 6976 -> 6977 bytes ...446ba-c04a-440a-afad-f090aa27c81a-m0.avro} | Bin 6978 -> 6978 bytes ...0d7049-0780-4636-972a-7f46bb6c395a-m0.avro | Bin 0 -> 6981 bytes ...2a84e-df73-4cb9-a92f-18d737a54489-m0.avro} | Bin 6978 -> 6978 bytes ...d1067-e40a-4222-bab9-959f5af4dd4d-m0.avro} | Bin 6978 -> 6978 bytes ...bb0d7049-0780-4636-972a-7f46bb6c395a.avro} | Bin 4302 -> 4226 bytes ...226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro} | Bin 4399 -> 4399 bytes ...bdc6d711-4699-4d21-a7f8-907c7740c70c.avro} | Bin 4224 -> 4095 bytes ...e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro} | Bin 4353 -> 4352 bytes ...-4b9446ba-c04a-440a-afad-f090aa27c81a.avro | Bin 0 -> 4301 bytes ...-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro | Bin 0 -> 4225 bytes .../iceberg/util/IcebergCatalogAdapter.java | 1 - 28 files changed, 633 insertions(+), 171 deletions(-) create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-2-6f8d12ab-3efb-4a24-a6a6-46b2b34b986c-0-00001.parquet delete mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-75fe700e-5fc0-40f8-9349-fcd5763dd691-0-00001.parquet delete mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-4be14704-ebcf-44c9-8482-90d09621c2ee-0-00001.parquet delete mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-efdf14cf-15de-4aac-bafc-cf750f9f7f70-0-00001.parquet create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-01e5d979-ab08-46a6-bd9d-c8008dc66519-0-00001.parquet delete mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json => 00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json} (72%) rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json => 00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json} (64%) rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json => 00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json} (60%) rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json => 00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json} (58%) create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00005-3476adcd-1647-44c6-930e-df3fe7867d8f.metadata.json rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro => 226480b9-ac20-4445-b67e-7cd3f7ef27d0-m0.avro} (95%) rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro => 4b9446ba-c04a-440a-afad-f090aa27c81a-m0.avro} (95%) create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/bb0d7049-0780-4636-972a-7f46bb6c395a-m0.avro rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{77bfad1c-123c-452c-814d-298a1483a99f-m0.avro => ddb2a84e-df73-4cb9-a92f-18d737a54489-m0.avro} (95%) rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro => e1bd1067-e40a-4222-bab9-959f5af4dd4d-m0.avro} (95%) rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro => snap-1277776933184906785-1-bb0d7049-0780-4636-972a-7f46bb6c395a.avro} (90%) rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro => snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro} (88%) rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro => snap-3825168261540020388-1-bdc6d711-4699-4d21-a7f8-907c7740c70c.avro} (92%) rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro => snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro} (89%) create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 07b6b127cdb..75fddadbc35 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -235,20 +235,22 @@ public void testListSnapshots() { adapter.listSnapshots(tableIdentifier) .forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); - Assert.eq(snapshotIds.size(), "snapshots.size()", 4, "4 snapshots for sales/sales_multi"); + Assert.eq(snapshotIds.size(), "snapshots.size()", 6, "6 snapshots for sales/sales_multi"); - Assert.eqTrue(snapshotIds.contains(2001582482032951248L), "snapshots.contains(2001582482032951248)"); - Assert.eqTrue(snapshotIds.contains(8325605756612719366L), "snapshots.contains(8325605756612719366L)"); - Assert.eqTrue(snapshotIds.contains(3247344357341484163L), "snapshots.contains(3247344357341484163L)"); - Assert.eqTrue(snapshotIds.contains(1792185872197984875L), "snapshots.contains(1792185872197984875L)"); + Assert.eqTrue(snapshotIds.contains(8121674823173822790L), "snapshots.contains(8121674823173822790L)"); + Assert.eqTrue(snapshotIds.contains(6040260770188877244L), "snapshots.contains(6040260770188877244L)"); + Assert.eqTrue(snapshotIds.contains(5693547373255393922L), "snapshots.contains(5693547373255393922L)"); + Assert.eqTrue(snapshotIds.contains(3445166522731196553L), "snapshots.contains(3445166522731196553L)"); + Assert.eqTrue(snapshotIds.contains(1277776933184906785L), "snapshots.contains(1277776933184906785L)"); + Assert.eqTrue(snapshotIds.contains(3825168261540020388L), "snapshots.contains(3825168261540020388L)"); Table table = adapter.snapshots(tableIdentifier); - Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); + Assert.eq(table.size(), "table.size()", 6, "6 snapshots for sales/sales_multi"); Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION); // Test the string versions of the methods table = adapter.snapshots("sales.sales_multi"); - Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); + Assert.eq(table.size(), "table.size()", 6, "6 snapshots for sales/sales_multi"); Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION); } @@ -286,14 +288,15 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + // This table ends up with zero records + Assert.eq(table.size(), "table.size()", 0, "0 rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); // Test the string versions of the methods table = adapter.readTable("sales.sales_multi", instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 0, "0 rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } @@ -729,23 +732,33 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept // Verify we retrieved all the rows. final io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0).snapshotId(), instructions); - Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); + Assert.eq(table0.size(), "table0.size()", 18073, "18266 rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions); - Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); + Assert.eq(table1.size(), "table1.size()", 54433, "54373 rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions); - Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); + Assert.eq(table2.size(), "table2.size()", 72551, "72603 rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions); Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION); + + final io.deephaven.engine.table.Table table4 = + adapter.readTable(tableId, snapshots.get(4).snapshotId(), instructions); + Assert.eq(table4.size(), "table4.size()", 100_000, "100_000 rows in the table"); + Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); + + final io.deephaven.engine.table.Table table5 = + adapter.readTable(tableId, snapshots.get(5).snapshotId(), instructions); + Assert.eq(table5.size(), "table5.size()", 0, "0 rows in the table"); + Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @Test @@ -760,39 +773,55 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx // Verify we retrieved all the rows. io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions); - Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); + Assert.eq(table0.size(), "table0.size()", 18073, "18073 rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1), instructions); - Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); + Assert.eq(table1.size(), "table1.size()", 54433, "54433 rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2), instructions); - Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); + Assert.eq(table2.size(), "table2.size()", 72551, "72551 rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3), instructions); Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION); + io.deephaven.engine.table.Table table4 = adapter.readTable(tableId, snapshots.get(4), instructions); + Assert.eq(table4.size(), "table4.size()", 100_000, "100_000 rows in the table"); + Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); + + io.deephaven.engine.table.Table table5 = adapter.readTable(tableId, snapshots.get(5), instructions); + Assert.eq(table5.size(), "table5.size()", 0, "0 rows in the table"); + Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); + // Test the string versions of the methods // Verify we retrieved all the rows. table0 = adapter.readTable("sales.sales_multi", snapshots.get(0).snapshotId(), instructions); - Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table"); + Assert.eq(table0.size(), "table0.size()", 18073, "18073 rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); - table1 = adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions); - Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table"); + table1 = adapter.readTable("sales.sales_multi", snapshots.get(1).snapshotId(), instructions); + Assert.eq(table1.size(), "table1.size()", 54433, "54433 rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); - table2 = adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions); - Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table"); + table2 = adapter.readTable("sales.sales_multi", snapshots.get(2).snapshotId(), instructions); + Assert.eq(table2.size(), "table2.size()", 72551, "72551 rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); - table3 = adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions); + table3 = adapter.readTable("sales.sales_multi", snapshots.get(3).snapshotId(), instructions); Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); Assert.equals(table3.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); + + table4 = adapter.readTable("sales.sales_multi", snapshots.get(4).snapshotId(), instructions); + Assert.eq(table4.size(), "table4.size()", 100_000, "100_000 rows in the table"); + Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); + + table5 = adapter.readTable("sales.sales_multi", snapshots.get(5).snapshotId(), instructions); + Assert.eq(table5.size(), "table5.size()", 0, "0 rows in the table"); + Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @Test @@ -908,4 +937,43 @@ public void testTableDefinitionWithInstructions() { tableDef = adapter.getTableDefinition("sales.sales_multi", localInstructions); Assert.equals(tableDef, "tableDef", userTableDef); } + + @Test + public void testManualRefreshingTable() { + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .dataInstructions(instructions.dataInstructions().get()) + .updateMode(IcebergUpdateMode.manualRefreshingMode()) + .build(); + + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); + + final List snapshots = tableAdapter.listSnapshots(); + + // Use string and current snapshot + final IcebergTable table = tableAdapter.table(snapshots.get(0).snapshotId(), localInstructions); + Assert.eq(table.size(), "table.size()", 18073, "18073 rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + table.update(snapshots.get(1).snapshotId()); + Assert.eq(table.size(), "table.size()", 54433, "54433 rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + table.update(snapshots.get(2).snapshotId()); + Assert.eq(table.size(), "table.size()", 72551, "72551 rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + table.update(snapshots.get(3).snapshotId()); + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + table.update(snapshots.get(4).snapshotId()); + Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + table.update(snapshots.get(5).snapshotId()); + Assert.eq(table.size(), "table.size()", 0, "0 rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + } } diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-2-6f8d12ab-3efb-4a24-a6a6-46b2b34b986c-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-2-6f8d12ab-3efb-4a24-a6a6-46b2b34b986c-0-00001.parquet new file mode 100644 index 00000000000..4ec41a05ae1 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-2-6f8d12ab-3efb-4a24-a6a6-46b2b34b986c-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:463d2a2753ebcf86deacc9cea3042918c2ad8adc2f124dd5ca2a5eb6cf86ae54 +size 175359 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet deleted file mode 100644 index 04d259decea..00000000000 --- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:9749ae2922aa9d21b7e779142d6c2476d0444c2c24f7e93397e6750147180724 -size 176970 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-75fe700e-5fc0-40f8-9349-fcd5763dd691-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-75fe700e-5fc0-40f8-9349-fcd5763dd691-0-00001.parquet new file mode 100644 index 00000000000..634d73c5d51 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-75fe700e-5fc0-40f8-9349-fcd5763dd691-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:432d8cdd6e09203748f800b914728a63b2541b99fceaa215d764910adaa000e3 +size 333827 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet deleted file mode 100644 index 206e1f84a6c..00000000000 --- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:d5007c70d0e8be33ef012c48d7f067b47812b22747c145cfa4bab4bef944fb0f -size 331675 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-4be14704-ebcf-44c9-8482-90d09621c2ee-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-4be14704-ebcf-44c9-8482-90d09621c2ee-0-00001.parquet new file mode 100644 index 00000000000..1cb0281c073 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-4be14704-ebcf-44c9-8482-90d09621c2ee-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:3f5d595ce8cd13180368f05c2dfe57f4f2c8fba7c7437066bf998fc805662dc0 +size 175594 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet deleted file mode 100644 index 6d552bf0e9a..00000000000 --- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:c02909448106787f93e8a2550e9db360aedd9ab25c925e8a536b18096aa8ed91 -size 176482 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-efdf14cf-15de-4aac-bafc-cf750f9f7f70-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-efdf14cf-15de-4aac-bafc-cf750f9f7f70-0-00001.parquet new file mode 100644 index 00000000000..3455ea39555 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-efdf14cf-15de-4aac-bafc-cf750f9f7f70-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:c127833dec01dac8258db8dbd0acd9a3bc7e23957e4dce46c0a07c1174498875 +size 257345 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-01e5d979-ab08-46a6-bd9d-c8008dc66519-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-01e5d979-ab08-46a6-bd9d-c8008dc66519-0-00001.parquet new file mode 100644 index 00000000000..1a06c24b196 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-01e5d979-ab08-46a6-bd9d-c8008dc66519-0-00001.parquet @@ -0,0 +1,3 @@ +version https://git-lfs.github.com/spec/v1 +oid sha256:d5dbc201439bcf1bf4afb778a5cfc4e40a6b5d9d1a9a9b77cacd25bea21c2486 +size 729342 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet deleted file mode 100644 index 540e5bc3392..00000000000 --- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet +++ /dev/null @@ -1,3 +0,0 @@ -version https://git-lfs.github.com/spec/v1 -oid sha256:c872a3defa86826496fbc76fa4235c858d45edb5441e196c2c812f4c8a1166e7 -size 256855 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json similarity index 72% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json index 53c1afcba7b..90bd5751ba7 100644 --- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json @@ -1,9 +1,9 @@ { "format-version" : 2, - "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79", "location" : "s3://warehouse/sales/sales_multi", "last-sequence-number" : 1, - "last-updated-ms" : 1716234545155, + "last-updated-ms" : 1725396294486, "last-column-id" : 5, "current-schema-id" : 0, "schemas" : [ { @@ -49,43 +49,43 @@ } ], "properties" : { "owner" : "root", - "created-at" : "2024-05-20T19:49:04.669537174Z", + "created-at" : "2024-09-03T20:44:52.796187633Z", "write.format.default" : "parquet", "write.parquet.compression-codec" : "zstd" }, - "current-snapshot-id" : 2001582482032951248, + "current-snapshot-id" : 8121674823173822790, "refs" : { "main" : { - "snapshot-id" : 2001582482032951248, + "snapshot-id" : 8121674823173822790, "type" : "branch" } }, "snapshots" : [ { "sequence-number" : 1, - "snapshot-id" : 2001582482032951248, - "timestamp-ms" : 1716234545155, + "snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396294486, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "18266", - "added-files-size" : "176970", + "added-records" : "18073", + "added-files-size" : "175359", "changed-partition-count" : "1", - "total-records" : "18266", - "total-files-size" : "176970", + "total-records" : "18073", + "total-files-size" : "175359", "total-data-files" : "1", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro", "schema-id" : 0 } ], "statistics" : [ ], "partition-statistics" : [ ], "snapshot-log" : [ { - "timestamp-ms" : 1716234545155, - "snapshot-id" : 2001582482032951248 + "timestamp-ms" : 1725396294486, + "snapshot-id" : 8121674823173822790 } ], "metadata-log" : [ ] } \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json similarity index 64% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json index 5aa1996ce92..9ce7f8b1bd1 100644 --- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json @@ -1,9 +1,9 @@ { "format-version" : 2, - "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79", "location" : "s3://warehouse/sales/sales_multi", "last-sequence-number" : 2, - "last-updated-ms" : 1716234545529, + "last-updated-ms" : 1725396298768, "last-column-id" : 5, "current-schema-id" : 0, "schemas" : [ { @@ -49,70 +49,70 @@ } ], "properties" : { "owner" : "root", - "created-at" : "2024-05-20T19:49:04.669537174Z", + "created-at" : "2024-09-03T20:44:52.796187633Z", "write.format.default" : "parquet", "write.parquet.compression-codec" : "zstd" }, - "current-snapshot-id" : 8325605756612719366, + "current-snapshot-id" : 6040260770188877244, "refs" : { "main" : { - "snapshot-id" : 8325605756612719366, + "snapshot-id" : 6040260770188877244, "type" : "branch" } }, "snapshots" : [ { "sequence-number" : 1, - "snapshot-id" : 2001582482032951248, - "timestamp-ms" : 1716234545155, + "snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396294486, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "18266", - "added-files-size" : "176970", + "added-records" : "18073", + "added-files-size" : "175359", "changed-partition-count" : "1", - "total-records" : "18266", - "total-files-size" : "176970", + "total-records" : "18073", + "total-files-size" : "175359", "total-data-files" : "1", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro", "schema-id" : 0 }, { "sequence-number" : 2, - "snapshot-id" : 8325605756612719366, - "parent-snapshot-id" : 2001582482032951248, - "timestamp-ms" : 1716234545529, + "snapshot-id" : 6040260770188877244, + "parent-snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396298768, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "36107", - "added-files-size" : "331675", + "added-records" : "36360", + "added-files-size" : "333827", "changed-partition-count" : "1", - "total-records" : "54373", - "total-files-size" : "508645", + "total-records" : "54433", + "total-files-size" : "509186", "total-data-files" : "2", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro", "schema-id" : 0 } ], "statistics" : [ ], "partition-statistics" : [ ], "snapshot-log" : [ { - "timestamp-ms" : 1716234545155, - "snapshot-id" : 2001582482032951248 + "timestamp-ms" : 1725396294486, + "snapshot-id" : 8121674823173822790 }, { - "timestamp-ms" : 1716234545529, - "snapshot-id" : 8325605756612719366 + "timestamp-ms" : 1725396298768, + "snapshot-id" : 6040260770188877244 } ], "metadata-log" : [ { - "timestamp-ms" : 1716234545155, - "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json" + "timestamp-ms" : 1725396294486, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json" } ] } \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json similarity index 60% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json index 1be00194c4e..f233d8d55b1 100644 --- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json @@ -1,9 +1,9 @@ { "format-version" : 2, - "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79", "location" : "s3://warehouse/sales/sales_multi", "last-sequence-number" : 3, - "last-updated-ms" : 1716234545865, + "last-updated-ms" : 1725396300069, "last-column-id" : 5, "current-schema-id" : 0, "schemas" : [ { @@ -49,97 +49,97 @@ } ], "properties" : { "owner" : "root", - "created-at" : "2024-05-20T19:49:04.669537174Z", + "created-at" : "2024-09-03T20:44:52.796187633Z", "write.format.default" : "parquet", "write.parquet.compression-codec" : "zstd" }, - "current-snapshot-id" : 3247344357341484163, + "current-snapshot-id" : 5693547373255393922, "refs" : { "main" : { - "snapshot-id" : 3247344357341484163, + "snapshot-id" : 5693547373255393922, "type" : "branch" } }, "snapshots" : [ { "sequence-number" : 1, - "snapshot-id" : 2001582482032951248, - "timestamp-ms" : 1716234545155, + "snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396294486, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "18266", - "added-files-size" : "176970", + "added-records" : "18073", + "added-files-size" : "175359", "changed-partition-count" : "1", - "total-records" : "18266", - "total-files-size" : "176970", + "total-records" : "18073", + "total-files-size" : "175359", "total-data-files" : "1", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro", "schema-id" : 0 }, { "sequence-number" : 2, - "snapshot-id" : 8325605756612719366, - "parent-snapshot-id" : 2001582482032951248, - "timestamp-ms" : 1716234545529, + "snapshot-id" : 6040260770188877244, + "parent-snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396298768, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "36107", - "added-files-size" : "331675", + "added-records" : "36360", + "added-files-size" : "333827", "changed-partition-count" : "1", - "total-records" : "54373", - "total-files-size" : "508645", + "total-records" : "54433", + "total-files-size" : "509186", "total-data-files" : "2", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro", "schema-id" : 0 }, { "sequence-number" : 3, - "snapshot-id" : 3247344357341484163, - "parent-snapshot-id" : 8325605756612719366, - "timestamp-ms" : 1716234545865, + "snapshot-id" : 5693547373255393922, + "parent-snapshot-id" : 6040260770188877244, + "timestamp-ms" : 1725396300069, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "18230", - "added-files-size" : "176482", + "added-records" : "18118", + "added-files-size" : "175594", "changed-partition-count" : "1", - "total-records" : "72603", - "total-files-size" : "685127", + "total-records" : "72551", + "total-files-size" : "684780", "total-data-files" : "3", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro", "schema-id" : 0 } ], "statistics" : [ ], "partition-statistics" : [ ], "snapshot-log" : [ { - "timestamp-ms" : 1716234545155, - "snapshot-id" : 2001582482032951248 + "timestamp-ms" : 1725396294486, + "snapshot-id" : 8121674823173822790 }, { - "timestamp-ms" : 1716234545529, - "snapshot-id" : 8325605756612719366 + "timestamp-ms" : 1725396298768, + "snapshot-id" : 6040260770188877244 }, { - "timestamp-ms" : 1716234545865, - "snapshot-id" : 3247344357341484163 + "timestamp-ms" : 1725396300069, + "snapshot-id" : 5693547373255393922 } ], "metadata-log" : [ { - "timestamp-ms" : 1716234545155, - "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json" + "timestamp-ms" : 1725396294486, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json" }, { - "timestamp-ms" : 1716234545529, - "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json" + "timestamp-ms" : 1725396298768, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json" } ] } \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json similarity index 58% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json index 16cca0f3f8d..f63543c318f 100644 --- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json @@ -1,9 +1,9 @@ { "format-version" : 2, - "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512", + "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79", "location" : "s3://warehouse/sales/sales_multi", "last-sequence-number" : 4, - "last-updated-ms" : 1716234546189, + "last-updated-ms" : 1725396301293, "last-column-id" : 5, "current-schema-id" : 0, "schemas" : [ { @@ -49,124 +49,124 @@ } ], "properties" : { "owner" : "root", - "created-at" : "2024-05-20T19:49:04.669537174Z", + "created-at" : "2024-09-03T20:44:52.796187633Z", "write.format.default" : "parquet", "write.parquet.compression-codec" : "zstd" }, - "current-snapshot-id" : 1792185872197984875, + "current-snapshot-id" : 3445166522731196553, "refs" : { "main" : { - "snapshot-id" : 1792185872197984875, + "snapshot-id" : 3445166522731196553, "type" : "branch" } }, "snapshots" : [ { "sequence-number" : 1, - "snapshot-id" : 2001582482032951248, - "timestamp-ms" : 1716234545155, + "snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396294486, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "18266", - "added-files-size" : "176970", + "added-records" : "18073", + "added-files-size" : "175359", "changed-partition-count" : "1", - "total-records" : "18266", - "total-files-size" : "176970", + "total-records" : "18073", + "total-files-size" : "175359", "total-data-files" : "1", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro", "schema-id" : 0 }, { "sequence-number" : 2, - "snapshot-id" : 8325605756612719366, - "parent-snapshot-id" : 2001582482032951248, - "timestamp-ms" : 1716234545529, + "snapshot-id" : 6040260770188877244, + "parent-snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396298768, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "36107", - "added-files-size" : "331675", + "added-records" : "36360", + "added-files-size" : "333827", "changed-partition-count" : "1", - "total-records" : "54373", - "total-files-size" : "508645", + "total-records" : "54433", + "total-files-size" : "509186", "total-data-files" : "2", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro", "schema-id" : 0 }, { "sequence-number" : 3, - "snapshot-id" : 3247344357341484163, - "parent-snapshot-id" : 8325605756612719366, - "timestamp-ms" : 1716234545865, + "snapshot-id" : 5693547373255393922, + "parent-snapshot-id" : 6040260770188877244, + "timestamp-ms" : 1725396300069, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "18230", - "added-files-size" : "176482", + "added-records" : "18118", + "added-files-size" : "175594", "changed-partition-count" : "1", - "total-records" : "72603", - "total-files-size" : "685127", + "total-records" : "72551", + "total-files-size" : "684780", "total-data-files" : "3", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro", "schema-id" : 0 }, { "sequence-number" : 4, - "snapshot-id" : 1792185872197984875, - "parent-snapshot-id" : 3247344357341484163, - "timestamp-ms" : 1716234546189, + "snapshot-id" : 3445166522731196553, + "parent-snapshot-id" : 5693547373255393922, + "timestamp-ms" : 1725396301293, "summary" : { "operation" : "append", - "spark.app.id" : "local-1716234462547", + "spark.app.id" : "local-1725396284766", "added-data-files" : "1", - "added-records" : "27397", - "added-files-size" : "256855", + "added-records" : "27449", + "added-files-size" : "257345", "changed-partition-count" : "1", "total-records" : "100000", - "total-files-size" : "941982", + "total-files-size" : "942125", "total-data-files" : "4", "total-delete-files" : "0", "total-position-deletes" : "0", "total-equality-deletes" : "0" }, - "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro", + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro", "schema-id" : 0 } ], "statistics" : [ ], "partition-statistics" : [ ], "snapshot-log" : [ { - "timestamp-ms" : 1716234545155, - "snapshot-id" : 2001582482032951248 + "timestamp-ms" : 1725396294486, + "snapshot-id" : 8121674823173822790 }, { - "timestamp-ms" : 1716234545529, - "snapshot-id" : 8325605756612719366 + "timestamp-ms" : 1725396298768, + "snapshot-id" : 6040260770188877244 }, { - "timestamp-ms" : 1716234545865, - "snapshot-id" : 3247344357341484163 + "timestamp-ms" : 1725396300069, + "snapshot-id" : 5693547373255393922 }, { - "timestamp-ms" : 1716234546189, - "snapshot-id" : 1792185872197984875 + "timestamp-ms" : 1725396301293, + "snapshot-id" : 3445166522731196553 } ], "metadata-log" : [ { - "timestamp-ms" : 1716234545155, - "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json" + "timestamp-ms" : 1725396294486, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json" }, { - "timestamp-ms" : 1716234545529, - "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json" + "timestamp-ms" : 1725396298768, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json" }, { - "timestamp-ms" : 1716234545865, - "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json" + "timestamp-ms" : 1725396300069, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json" } ] } \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json new file mode 100644 index 00000000000..32367a81a58 --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json @@ -0,0 +1,186 @@ +{ + "format-version" : 2, + "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 5, + "last-updated-ms" : 1725396302818, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-09-03T20:44:52.796187633Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 1277776933184906785, + "refs" : { + "main" : { + "snapshot-id" : 1277776933184906785, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396294486, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "18073", + "added-files-size" : "175359", + "changed-partition-count" : "1", + "total-records" : "18073", + "total-files-size" : "175359", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 6040260770188877244, + "parent-snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396298768, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "36360", + "added-files-size" : "333827", + "changed-partition-count" : "1", + "total-records" : "54433", + "total-files-size" : "509186", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro", + "schema-id" : 0 + }, { + "sequence-number" : 3, + "snapshot-id" : 5693547373255393922, + "parent-snapshot-id" : 6040260770188877244, + "timestamp-ms" : 1725396300069, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "18118", + "added-files-size" : "175594", + "changed-partition-count" : "1", + "total-records" : "72551", + "total-files-size" : "684780", + "total-data-files" : "3", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro", + "schema-id" : 0 + }, { + "sequence-number" : 4, + "snapshot-id" : 3445166522731196553, + "parent-snapshot-id" : 5693547373255393922, + "timestamp-ms" : 1725396301293, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "27449", + "added-files-size" : "257345", + "changed-partition-count" : "1", + "total-records" : "100000", + "total-files-size" : "942125", + "total-data-files" : "4", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro", + "schema-id" : 0 + }, { + "sequence-number" : 5, + "snapshot-id" : 1277776933184906785, + "timestamp-ms" : 1725396302818, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "100000", + "added-files-size" : "729342", + "changed-partition-count" : "1", + "total-records" : "100000", + "total-files-size" : "729342", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-1277776933184906785-1-bb0d7049-0780-4636-972a-7f46bb6c395a.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1725396302818, + "snapshot-id" : 1277776933184906785 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1725396294486, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json" + }, { + "timestamp-ms" : 1725396298768, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json" + }, { + "timestamp-ms" : 1725396300069, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json" + }, { + "timestamp-ms" : 1725396301293, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00005-3476adcd-1647-44c6-930e-df3fe7867d8f.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00005-3476adcd-1647-44c6-930e-df3fe7867d8f.metadata.json new file mode 100644 index 00000000000..fe4f6176a8d --- /dev/null +++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00005-3476adcd-1647-44c6-930e-df3fe7867d8f.metadata.json @@ -0,0 +1,206 @@ +{ + "format-version" : 2, + "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79", + "location" : "s3://warehouse/sales/sales_multi", + "last-sequence-number" : 6, + "last-updated-ms" : 1725396304540, + "last-column-id" : 5, + "current-schema-id" : 0, + "schemas" : [ { + "type" : "struct", + "schema-id" : 0, + "fields" : [ { + "id" : 1, + "name" : "Region", + "required" : false, + "type" : "string" + }, { + "id" : 2, + "name" : "Item_Type", + "required" : false, + "type" : "string" + }, { + "id" : 3, + "name" : "Units_Sold", + "required" : false, + "type" : "int" + }, { + "id" : 4, + "name" : "Unit_Price", + "required" : false, + "type" : "double" + }, { + "id" : 5, + "name" : "Order_Date", + "required" : false, + "type" : "timestamptz" + } ] + } ], + "default-spec-id" : 0, + "partition-specs" : [ { + "spec-id" : 0, + "fields" : [ ] + } ], + "last-partition-id" : 999, + "default-sort-order-id" : 0, + "sort-orders" : [ { + "order-id" : 0, + "fields" : [ ] + } ], + "properties" : { + "owner" : "root", + "created-at" : "2024-09-03T20:44:52.796187633Z", + "write.format.default" : "parquet", + "write.parquet.compression-codec" : "zstd" + }, + "current-snapshot-id" : 3825168261540020388, + "refs" : { + "main" : { + "snapshot-id" : 3825168261540020388, + "type" : "branch" + } + }, + "snapshots" : [ { + "sequence-number" : 1, + "snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396294486, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "18073", + "added-files-size" : "175359", + "changed-partition-count" : "1", + "total-records" : "18073", + "total-files-size" : "175359", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro", + "schema-id" : 0 + }, { + "sequence-number" : 2, + "snapshot-id" : 6040260770188877244, + "parent-snapshot-id" : 8121674823173822790, + "timestamp-ms" : 1725396298768, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "36360", + "added-files-size" : "333827", + "changed-partition-count" : "1", + "total-records" : "54433", + "total-files-size" : "509186", + "total-data-files" : "2", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro", + "schema-id" : 0 + }, { + "sequence-number" : 3, + "snapshot-id" : 5693547373255393922, + "parent-snapshot-id" : 6040260770188877244, + "timestamp-ms" : 1725396300069, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "18118", + "added-files-size" : "175594", + "changed-partition-count" : "1", + "total-records" : "72551", + "total-files-size" : "684780", + "total-data-files" : "3", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro", + "schema-id" : 0 + }, { + "sequence-number" : 4, + "snapshot-id" : 3445166522731196553, + "parent-snapshot-id" : 5693547373255393922, + "timestamp-ms" : 1725396301293, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "27449", + "added-files-size" : "257345", + "changed-partition-count" : "1", + "total-records" : "100000", + "total-files-size" : "942125", + "total-data-files" : "4", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro", + "schema-id" : 0 + }, { + "sequence-number" : 5, + "snapshot-id" : 1277776933184906785, + "timestamp-ms" : 1725396302818, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "added-data-files" : "1", + "added-records" : "100000", + "added-files-size" : "729342", + "changed-partition-count" : "1", + "total-records" : "100000", + "total-files-size" : "729342", + "total-data-files" : "1", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-1277776933184906785-1-bb0d7049-0780-4636-972a-7f46bb6c395a.avro", + "schema-id" : 0 + }, { + "sequence-number" : 6, + "snapshot-id" : 3825168261540020388, + "timestamp-ms" : 1725396304540, + "summary" : { + "operation" : "append", + "spark.app.id" : "local-1725396284766", + "changed-partition-count" : "0", + "total-records" : "0", + "total-files-size" : "0", + "total-data-files" : "0", + "total-delete-files" : "0", + "total-position-deletes" : "0", + "total-equality-deletes" : "0" + }, + "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3825168261540020388-1-bdc6d711-4699-4d21-a7f8-907c7740c70c.avro", + "schema-id" : 0 + } ], + "statistics" : [ ], + "partition-statistics" : [ ], + "snapshot-log" : [ { + "timestamp-ms" : 1725396304540, + "snapshot-id" : 3825168261540020388 + } ], + "metadata-log" : [ { + "timestamp-ms" : 1725396294486, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json" + }, { + "timestamp-ms" : 1725396298768, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json" + }, { + "timestamp-ms" : 1725396300069, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json" + }, { + "timestamp-ms" : 1725396301293, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json" + }, { + "timestamp-ms" : 1725396302818, + "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json" + } ] +} \ No newline at end of file diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/226480b9-ac20-4445-b67e-7cd3f7ef27d0-m0.avro similarity index 95% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/226480b9-ac20-4445-b67e-7cd3f7ef27d0-m0.avro index d871fbb9fd098d297903c3e1dcd7f46abe821c6e..2de91cb1f3498958d4ca022d64344afaf83b7f0e 100644 GIT binary patch delta 315 zcmV-B0mS~mHo-Qq5E_5xdjE7NJO^-6$Ep_b4_~qZoCITHnsjLW_WNsYoR4Q(sPb4oJxQxZ!O^$kEk*Hkw(EhWv+Bsopj&@?4g z*Ca79SvM&$Em=1?&D_)=%`(kA&D=m22!X&*uOP9gur#%VFTj5>C@|DDn+$=+jh12+&;7^ zcRri2_+|UFX|iu}w7(Sy) za!IqWu~4_rXAzmpU7I3Y4jBbq4)zwB+#r|oSRjBsllA45iYrW4ZvFLn^>gQUhqVE( JB^b~{2mlTkiNgQ@ diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/4b9446ba-c04a-440a-afad-f090aa27c81a-m0.avro similarity index 95% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/4b9446ba-c04a-440a-afad-f090aa27c81a-m0.avro index 21c52f9fe1cc6652cda25045a38657547255ea89..d91a8b5dc742a129031b1bdffc135b033a882d58 100644 GIT binary patch delta 316 zcmV-C0mJ^nHo`Wr5E_3;gM=+}Te^;EkhX-Qz$PRDo&;lJ`tfbu-Fv52Twcz|z`$^X zvDny3U%x!DC^aL$v^Z72I58)+7(&M9mgbaX>Zc@@Bh`E3mB+S6Vz{bGAz{SA8#Kge> z65((x&P-$xa!O39RB+4BPhsN#`GZ41Gp7I0*A+?*TmlA`hnC-d!wOcd5L}w18=RPt zSd^Hj;FwmF#UeBkmYSYgl9&YaARC8(I!G%>r^cfepjOAYWjD^f2Wo{{#{d9HgM=+} OTe^;EkhX-Qz$PT4#hCs8 delta 316 zcmX?PcF1glfK+|^n;$dY`pvxbVAa)GuO&XFGR(80>uo5;sVVrk`WR&BrGI$NJz|> zVIkq5B-3|vW~YvmU#I6(uDqW=HFIQo9O?@=GLC(b|Fz5MWTSV&%F92`Zoj{wDO8(f zy4p)N^)wYR)n}R(%S2N*c9^s*UABChiW;NsiDQZj_%3l>TKRDW>k3o-4;Mcdzi*6M Lc+HT30X>WWF7=GM diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/bb0d7049-0780-4636-972a-7f46bb6c395a-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/bb0d7049-0780-4636-972a-7f46bb6c395a-m0.avro new file mode 100644 index 0000000000000000000000000000000000000000..463995b0f52698b291b22bb0b7438348b572b1c3 GIT binary patch literal 6981 zcmb_gTWl0n7?v&ujnHTj@KWJ8i3xJTuBTzTTxaqimQwVSNd0I}|g|*gvSqv}l zHg(UHhAbl!aa%XNP;YC|RcVK#t5J$GR$ei*@OE2xd%Kg-9BGT}McwxFoaTCR&h`#+ z)1%X?kX5KL1U&Df!jK;Z)mX-5{$EEReRua!jg3xNFs!3)(r)W+c z(nLmDRyQ=sU9iYjMr20zWC_a+O{)sBBfyz*U{%ZX0QT@c=Odf3RLJqo{!15GS>%mt z2y05CL?>1oM|`Jkxw;2cfHIn)d72wXLMQkq8@e|YJPWjvF!3U5Jpsg)y>Y4?st^|q z#D^HrZ>cg-uzRo%9nigT)H{SoK4FiEH@@6Nfx;%sRgPNq9k}?Gm z4}JFwD^Re@@??Xg3v%H&79SjEgT^L0AMu+>#n6AnOY3pe&C)?2f z0yJKcZH%?=A<{F*v*3pjBH-Zq(cS%EQT3*jNR`^;2Y7Ly>Hw&OL_JV-l$u3xEo2-6 zm5^8fR3x9{kV&eM&p?krxN#%d%%A~zAR0T?NPJ5oL0v=u1s(%fRN~<@iq18G?I;h- zH9@EgK!Owz4ms#|h3c8cvz^qG%|!=GOaWM8M;+NFmK8x%wuv2ef`vytu2;w_L{JOc zqhG<1$tJSR!>|s(&ZGe5utEk6X8}D;XvcY$tOGisy?UWm=*&X0F6@ZEi1-@`lF-Ij zleg`~1qpXEv7T-wu#KXm7%PAz*4NI6zqNqGW;=8dDW`cd-V(|(iSvyM=S`2PI#V{; zo8TS_?jgmnl*qV_UsKNBrJqE*eeTLpGb${5EkpnKr*1l0@>Bu zk;tw#!EOng{X8u4H`-!@+NnW3!NRIqIasoE7v?b;TBvUe))HYGo5rr?09u43RpK4k zh%i7ya2A$KAXK0uQbntuV|0k(3GgJLRjQTI)JCw*08ZBIud*$Zn&ZMk-XL({zSx5I zMTkQ~q8X?k^tQ1ABfGjWT%T$qzd(lriWSf?N`8SpQ|^ns#U)-$ncX5{Mz0q0X0kT~ zmnrcrCr&j9p!Aopf;y#8^3F0YY5?neIjAIh%r^v)8b#?>Wnqg?#{y*e1Pv#Fra;81 z*ik~CqKFyQK);p{4Iw3~O-FBy6piYmVG6S!s;jcLh%hYzJ)<_!3@#yNmCXwJr_X<* zi5gWx6Uxu>5{g!fs;ZQYYDtClUqaTX{Gw70A;xql&cQ75#{r)t|G)RA~X3EdDOEei9c0`Hio7a}B56)!{x-NL_RJ_&$b;FF@8GWm>39<)#;53C$^f2ZctvM*WHspOhUMyOsgB2zN;o6m06q4^>m zLXZ#a5?#f7E;PVzF_GIeRYPY_>Ec{PPuLv|-NoM-ix*TF8-)kZVhW#9f)i9=QA8_j z*3>+jgC$Ee6;?Tj6|I9rq}CPh*Hm7SanW0LXlE;JWu*%>6ps5Ugv>AGl?|tss!F=`_Mc}ynj850)$G1~ zsY}C#@yR)9X83~-k4YmZKb20Ncz^JGf9OwcxFYltbO*f7j%?*E{v*@E|j*c82`KMuB QXV<;8r&mop7b^qlf1tq^#{d8T literal 0 HcmV?d00001 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/ddb2a84e-df73-4cb9-a92f-18d737a54489-m0.avro similarity index 95% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/ddb2a84e-df73-4cb9-a92f-18d737a54489-m0.avro index 9501a6ce37e2dbcae20429b8d3568c73ba343666..3e3c02833a0ee7470e59c3f9e8a9d42accf7081e 100644 GIT binary patch delta 298 zcmX?PcF1glfK+{#TM-kZ4F8OPtBI?Qx3Q!!&0|SUdR1w)GTH0?{R~$W6`bt)wIevVjOsc(h?FAk{={INJ>aZ zdX&J>SzsZSuC|H8QdKQfYsb^)%QqA@{5hz&K=N|^i`QJL?0SuEdzZH7?@wf19mb*O rdx_WYjE9cbncxB+?NdpDlli2@wpuK(M`G>;`YDQowZtgmx^%B#B=8W_kV>X>c2 zbY0(H$Ls6~eO=V4PSws|YzhmqS7j&7n>1@m+MG4BQ&O^) zWlzdTSd^WalC*Gk(!89snNw!vBqb)U$X>u5l)EB)LPSW!tf)ic^$#ROBwk5=JaF{D zfg?u_G6+2^&^hOmB3R_*vnr(E^m+fp#|bRInw=etm;JwRUDS&&hG~-0*5`Tq6IoY> uap?J8;`KY@q2qNXxWGsIRFdFiK4~$z)ya1@Nk6~Gywz!KG6Ms8hyVcaD~6>2 diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/e1bd1067-e40a-4222-bab9-959f5af4dd4d-m0.avro similarity index 95% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/e1bd1067-e40a-4222-bab9-959f5af4dd4d-m0.avro index 462894acbdef4351207c59d84392e303f1c0d9eb..2cda88a5e7816bf1018a9d0e6f34322df26e4d2d 100644 GIT binary patch delta 316 zcmX?PcF1glfKS9aNo=g{8{z_c^*%C456F=@;nls7g@025xTxyvGE|^|4_4G0|F*G;0=q@gL+8#PuI=;FO>$wYfxOhr=U(P&R zlDb<%So|_m+O*1?9AU8yg?kd7?)uu<vRC517`OyLqiQf72IrxT!!@mat7kXaWe<^L9#5~<$h1-)~xAP+ot`cqX zo?Ox_Y%J6*^jSpaa@VE^7n4IlmxH~9CO62XJQfIG&t!dhrQ!&1NdF`Po&;lJTJV4WtIVG9;DTyVC`UW7NYpiQ(o@!*2m~5eIo@$<= zYhqz+pqrSQlB8>Bl9HHYmX?^5l47Q7pbJuMs8^6!R9Ko?!WVzw7!(-l8nR_B)0DHK zOk7O!m|2#wvE5tB#_@rb>p&Af15gYo$pXY|K+FNeTtEyGW?*4pW8h%mVqjom;$Q%Y za5xrcCb9@QB_>rWxaH@kuyHT}{lOui8Pk90>k1_YE`d;u$o041u!2=91eYf11}A1D z7A58>IHnb4v4}K;rKYErBqjkp$i^X{4$=zJsqv@&1NdF{%kvR~USsEJ~T9{ZGn3-FcPCmx2#Av$sB|AqwM_y@84#SM+)`q5+ z%1=#UmY%7Xp2x@ZfHC@TL{84q_3`r}rp%fcA2nxH{G1t+qj(e_7kd|b8GS7d_7(~j z@)r6!c@j_18D1ZqQrb%%n+YCiUw23YQl1ei1s|^=WsJ iq{NOLd}k}FvYv09ynE_Q2`LG_GmjDz5|S7&+yDSb!cAHL delta 268 zcmZotJg2x}GrO>bv5~2nfvLHvnVF%HxuK=8nc3uH>`IJAi(j&H)TLG3-R)$W3OT-nvnFA)96pqF3U8_XtrY+A&P$hhg#rp~za>9xt~ z%US1)q8A?Int5FK>E{mTq_kvSClA(JEK)n#1=%_@LOEudA99_z-pDDYynMgy@7nFP z7E>lTzZP=x{CvD4+1@P7<3m!mLx!e7SozwQsP! KAMRr z1Y`u=-P-ISsOiVG&WeTAu|zCpm&J}9D=aK_mQL_XTEr2%+2(cs^Z3%P3}ue8j1=?s z$A{Qhd2>E)|65~Iud%)9&a{*jOB^~_g4c-#s97i^@mNh1OG#5wUn_j6;KWa>-2S%R zmdShXz4`a*+uEwSH`P75t4xpCgl%QWUEzB|m{ZJ>kxS7b`0eV5oik*lnxDzfxstHm z+A^X}?-_q}`rN2F^Qv-PuW%V&WRW}`l@n2va-i^NXUmyO6T)2Y$Gg|X?%Q)tf8IIi zk1x*UYfCL=U7B&wLXt7fH%?WiW#z<)OC$_KS1)?*YoGS}c>8_x_q`RhpMHO2M-L(Z DuXvvn delta 360 zcmZ3lv|ee$W_Dpib4w#b3sVboBSTAbOA8YVbJNYo*i|^{jg3sqjZI99O@XALiG_)w znK8o?i4}Le_up_hoZUY&m9uyP#~hZa=MB9NJBS?kn60xk{@Khsr;qObym-fn-FK`- zq_})nt!@_4Y2tena&hOilj|;dc0G9V;MIkLuI%=WY_2b6J>CEB$N8w#U9!j9O&;w1 zx8cc-C34g2@9MVg&8oN8+o9wydUR$j1Cl0=v?$ijrQP9=G4f+pIcR_WO(b&zHre8zuH{ z&w5jIFHh^zv@^Eayq@Rerd;NlGnHjp@E5zye5&j8-7cQ}o>Xctt6R!@Zhf72Jw^}# E05%b$DF6Tf diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3825168261540020388-1-bdc6d711-4699-4d21-a7f8-907c7740c70c.avro similarity index 92% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3825168261540020388-1-bdc6d711-4699-4d21-a7f8-907c7740c70c.avro index 28d439215a04848e617058919ca10091fdd5fbd9..ca67d791ceb320cec91cbf101405a982357071ce 100644 GIT binary patch delta 54 zcmV-60LlM=A^#t+vj-A0I5IUcHaIdiF*P(WFfuSRIFrQ(BLOzE=?4Y~5QOgc%H<>U Mi-YW^^k4{~+29lu`~Uy| delta 184 zcmew_-=Mf*GrN$Hfq|i^g^`Jck%6(1rKzEj$>gK#%8Z7aU$V1t)N8%=5HfzzF`Z}k zy?O=yU3pA*7^4qIr{LF`kC;Ax^r*wWb2$Y}F1b`_3#GXoO?BQpbYa|1&Q3k!2| zBNG#bv~}ATe!4O%=z^fsff7f6dPCj)|+?*I2=y;bO~=|yhI^=7j| z;=)Y6T-~?)X-WitO8v|1q*b23jubAcVzbd=x)Rl_oPIFPEJb93q@-7=)8&sDfm|2g z>gkom-=F>U&pu(!jeD|%IUS#{qiW@$a-mJ`W!L|{s&%Po;gr}iY7_jU8Oy`O!2 zdwbZPs@{IlnRA!P*cKRQCR}_o@han_IlfsM&S%~(lyE(L{`KeCr|(af=nt#e_qYBJ HGrFGv(#KhPXNE(`0m>8NFZ$8GZ!clKwY-DO?U}|n^W@cz)ZfI$2 zX2!73OCV%b$@-VAJT=_Qn|*e$9ble%ey5M0A9@yqB*_PohCLGA62Da23qCJm=fvHPyX#lRnK^{xZcVds0rt zoLTqfu5T`^)_Q%VzUaxcrCawsTEV1oh0&*zagLGT1R3qeCc>cs;>>ol{hna+j@{c{AuIdt6Dvy!gHQYTo5- zoBn<;-|o9l+PiE*6xEU!a%eVvyw;$8)WI{kN3}BKN4?=`xqaJqFTX1J=w{i2XYbNK IN~3!k09J*Ln*aa+ diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro new file mode 100644 index 0000000000000000000000000000000000000000..790df47dae8eeaab9e9d153b5d693497193327b3 GIT binary patch literal 4301 zcmbVPTWB0r7_LDOf(f;>p^c?I4z>7@$joN5yIGpr)+h$!g>3uMu&ifi&Ss~~&a5+M zvaW435a>(QRy0pu&=*CaTA_GpeJNVnO7kKVD(X|AAZio(AOs3}&Rotpv%9kwA9wc5 z_xv2adqY7NwbzH*rDw492ja19lq!G!475cjZO}&I1Tqx-V zB9BWtGBiiZ&K++$QH8inP6R*>wsmu!{MIa$;LKPm#JXjIBeT<>rsFbrItM&8%}U_}6~LL2Keo`O?FYJo7b zz;zR|sB`2`Fg*oa2WbFXpl0j1G2Vm#t6C;T-upooM6f6QKOJC|04^gCrZx-YNeyB+ z0+dM&8K^08B`r`m=w%m~Dk_+6rHE`EcO)gb;1uvrfm#`v5mHct4uG15=*_*N`dL!O z1+I!t^3xUHj8!8|0>Z-~EvbZf(*~-PK;?x}a7GN6b}&oa;vlw}|mS1eYTfR3ZtdqaVuvduIvFDBAXphU8u%{!MNOb_}rbC950*AOo%Q6r&$pc>$58EzL zs}g`_-IuR29ZE7hh5=ZdU;!F#qv%Ws9mvT%=;)oKkTc(j6q+qKHB^GGfu$^sDJm2h zAq0brzg;MjiQ+;UW>xaFnDM_DN-yx<0^VDoWZ3|cYkRXv6BJJ@E(o_;{U5~MTfGx2 zi}}S7iA-CCNJFqxGK=hDIk!@QL^0@95LC-Noo(+UrwQyM9~TVm?^&-|l9e zOt!PD(mm%J0Piw#WV;8;xY-BBEUY=V12LqLgynj$2b?vp3Prk>+c%7A$y$7n|j$XLyz$!47Z+UfwW0JrjiKX@xIy8DKg? zjRwz42^tQQOQy#LF_Gk0!){>@1P{WLTlf<5Rdj-NE_VdJ{ow2otq!e0Cuh*gj_{0T zTF`n%gtJEO%p6H4+7^a{vyx>Me4$3YM=YsslHE)fvG+OJjS1mod_P6~M$csN#v@`a zI4-Rh@f?28Z43r^eGh3k>Ptl1kWms!DxJ(E#+39}CXq-_$eXSm1ZOY*d}rgaU#@TZ z`ti zF;iUcFK&70)Q!7yum5^<$N81vkJm0-{s@0Qb^E6;KR9vb$mGh!{-JGKubljD+wzuG zP#QSY|LkX};q@Q(99y#USI!mQa30yWwsAB6#iy(F{5yxPpZ;+3tnu&mvu`d={dM&d tc;NjXPrh{T-Ql+fmN!3l`|7>7W}ja>buBV#0lH?Qm|dVY99!#b==xE;i0u;p zKt*>01^Z^eu#0$rh&p!(a(vVqGeWa)Q(!{@9^#NWJf1Ec5~~Z$B5=oNB5D)>f<3D1T0sO{Ke7}yee#fS#pOdEc#l3g?9?=T0vA*smnYs z2nrdFQFctpk-n}#l207Upwr_4nTq;OdU6V|n*5{{?Af?i>62n0zh|$&~JEe&o6u$WA)7I1KR^{)S=q zKP@mq@%8%KAJ>KaU&2hxx(mG+Ui zm>GE<2F2i4A=nzobWXjG-U8UwcpF=RO{}wUbr&;oE)zpwy4W_F#%621-CQ$TYwc#U zwWXg29uKp|AqgogvgPK5Onn=P=r0@R9gDz@I7rw0Pp^wkhV2|#e9t8YAij*A+wSQy zE*gP!&4;iToDkMW2cU>(@degC@1KOlD6)u!U%`_Tt4aePDVl;p43@I`D@fGsU2X*g zfzc8K(ge%_yND@}WE+AktGToT6{NX@nG4th$+{_6%Za9KC8G#ADfOH}nIl0p_K+n)t%a6ePEvEAo?T zzJj$7!5gyJB0memi~J~dKr+bkX6ad&AWKgfzAj}z=nOX+GB17XQX!W@k6kd4aBO0? zhzE)X5z4K6i6tt!B|29+g2aAE_MBE{)?k)1D6=CnqwNyf*oa8h+?|CZnMGS-NF*yP ztMm&s?>#C>9UpcxU&Qg}=sYGwlF9v)_Zth7RU40-wUD^HUf?;Bpr;rNc>PUiIPOan z+wc;1ZO?u9&C+H1*3;G1-+#W*{Pf3Hmj77V|K*nJe*1m*@vX1E{<3@f(RbZPCl7DG z{KiwdP2X)kqvm$atZmnxnR^eus5N%>BJ=dgakTs3VLe)D9G|i?jlCzwzpd= Date: Wed, 4 Sep 2024 09:56:41 -0700 Subject: [PATCH 24/56] Manual and auto refreshing tests for Iceberg. --- .../iceberg/util/IcebergToolsTest.java | 163 ++++++++----- .../iceberg/util/IcebergCatalogAdapter.java | 8 - .../IcebergRefreshingTestTable.java | 218 ++++++++++++++++++ 3 files changed, 326 insertions(+), 63 deletions(-) create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 335082ea355..66b1a08ef3b 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -5,10 +5,14 @@ import gnu.trove.list.array.TLongArrayList; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.testutil.ControlledUpdateGraph; +import io.deephaven.engine.testutil.GenerateTableUpdates; +import io.deephaven.engine.testutil.TstUtils; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; @@ -201,12 +205,12 @@ public void testListNamespaces() { final Collection namespaceNames = namespaces.stream().map(Namespace::toString).collect(Collectors.toList()); - Assert.eq(namespaceNames.size(), "namespaceNames.size()", 2, "2 namespace in the catalog"); + Assert.eq(namespaceNames.size(), "namespaceNames.size()", 2, "namespace in the catalog"); Assert.eqTrue(namespaceNames.contains("sales"), "namespaceNames.contains(sales)"); Assert.eqTrue(namespaceNames.contains("sample"), "namespaceNames.contains(sample)"); final Table table = adapter.namespaces(); - Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog"); + Assert.eq(table.size(), "table.size()", 2, "namespace in the catalog"); Assert.equals(table.getDefinition(), "table.getDefinition()", NAMESPACE_DEFINITION); } @@ -217,7 +221,7 @@ public void testListTables() { final Namespace ns = Namespace.of("sales"); Collection tables = adapter.listTables(ns); - Assert.eq(tables.size(), "tables.size()", 4, "4 tables in the namespace"); + Assert.eq(tables.size(), "tables.size()", 4, "tables in the namespace"); Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_multi")), "tables.contains(sales_multi)"); Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_partitioned")), "tables.contains(sales_partitioned)"); @@ -225,12 +229,12 @@ public void testListTables() { Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_renamed")), "tables.contains(sales_renamed)"); Table table = adapter.tables(ns); - Assert.eq(table.size(), "table.size()", 4, "4 tables in the namespace"); + Assert.eq(table.size(), "table.size()", 4, "tables in the namespace"); Assert.equals(table.getDefinition(), "table.getDefinition()", TABLES_DEFINITION); // Test the string versions of the methods table = adapter.tables("sales"); - Assert.eq(table.size(), "table.size()", 4, "4 tables in the namespace"); + Assert.eq(table.size(), "table.size()", 4, "tables in the namespace"); Assert.equals(table.getDefinition(), "table.getDefinition()", TABLES_DEFINITION); } @@ -243,7 +247,7 @@ public void testListSnapshots() { adapter.listSnapshots(tableIdentifier) .forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); - Assert.eq(snapshotIds.size(), "snapshots.size()", 6, "6 snapshots for sales/sales_multi"); + Assert.eq(snapshotIds.size(), "snapshots.size()", 6, "snapshots for sales/sales_multi"); Assert.eqTrue(snapshotIds.contains(8121674823173822790L), "snapshots.contains(8121674823173822790L)"); Assert.eqTrue(snapshotIds.contains(6040260770188877244L), "snapshots.contains(6040260770188877244L)"); @@ -253,12 +257,12 @@ public void testListSnapshots() { Assert.eqTrue(snapshotIds.contains(3825168261540020388L), "snapshots.contains(3825168261540020388L)"); Table table = adapter.snapshots(tableIdentifier); - Assert.eq(table.size(), "table.size()", 6, "6 snapshots for sales/sales_multi"); + Assert.eq(table.size(), "table.size()", 6, "snapshots for sales/sales_multi"); Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION); // Test the string versions of the methods table = adapter.snapshots("sales.sales_multi"); - Assert.eq(table.size(), "table.size()", 6, "6 snapshots for sales/sales_multi"); + Assert.eq(table.size(), "table.size()", 6, "snapshots for sales/sales_multi"); Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION); } @@ -274,14 +278,14 @@ public void testOpenTableA() throws ExecutionException, InterruptedException, Ti io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_PARTITIONED_DEFINITION); // Test the string versions of the methods table = adapter.readTable("sales.sales_partitioned", instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_PARTITIONED_DEFINITION); } @@ -297,14 +301,14 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // This table ends up with zero records - Assert.eq(table.size(), "table.size()", 0, "0 rows in the table"); + Assert.eq(table.size(), "table.size()", 0, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); // Test the string versions of the methods table = adapter.readTable("sales.sales_multi", instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 0, "0 rows in the table"); + Assert.eq(table.size(), "table.size()", 0, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } @@ -320,14 +324,14 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_SINGLE_DEFINITION); // Test the string versions of the methods table = adapter.readTable("sales.sales_single", instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_SINGLE_DEFINITION); } @@ -342,7 +346,7 @@ public void testOpenTableS3Only() throws ExecutionException, InterruptedExceptio final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_PARTITIONED_DEFINITION); } @@ -363,7 +367,7 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_PARTITIONED_DEFINITION); } @@ -435,7 +439,7 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", renamed); } @@ -465,7 +469,7 @@ public void testSkippedPartitioningColumn() throws ExecutionException, Interrupt final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", tableDef); } @@ -495,7 +499,7 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", tableDef); } @@ -516,7 +520,7 @@ public void testZeroPartitioningColumns() throws ExecutionException, Interrupted final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } @@ -609,7 +613,7 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); } @Test @@ -628,7 +632,7 @@ public void testOpenTableColumnLegalization() throws ExecutionException, Interru final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_RENAMED_DEFINITION); } @@ -658,7 +662,7 @@ public void testOpenTableColumnLegalizationRename() ColumnDefinition.ofTime("Order_Date")); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", expected); } @@ -740,32 +744,32 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept // Verify we retrieved all the rows. final io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0).snapshotId(), instructions); - Assert.eq(table0.size(), "table0.size()", 18073, "18266 rows in the table"); + Assert.eq(table0.size(), "table0.size()", 18073, "rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions); - Assert.eq(table1.size(), "table1.size()", 54433, "54373 rows in the table"); + Assert.eq(table1.size(), "table1.size()", 54433, "rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions); - Assert.eq(table2.size(), "table2.size()", 72551, "72603 rows in the table"); + Assert.eq(table2.size(), "table2.size()", 72551, "rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions); - Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table3.size(), "table3.size()", 100_000, "rows in the table"); Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table4 = adapter.readTable(tableId, snapshots.get(4).snapshotId(), instructions); - Assert.eq(table4.size(), "table4.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table4.size(), "table4.size()", 100_000, "rows in the table"); Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table5 = adapter.readTable(tableId, snapshots.get(5).snapshotId(), instructions); - Assert.eq(table5.size(), "table5.size()", 0, "0 rows in the table"); + Assert.eq(table5.size(), "table5.size()", 0, "rows in the table"); Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @@ -781,54 +785,54 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx // Verify we retrieved all the rows. io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions); - Assert.eq(table0.size(), "table0.size()", 18073, "18073 rows in the table"); + Assert.eq(table0.size(), "table0.size()", 18073, "rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1), instructions); - Assert.eq(table1.size(), "table1.size()", 54433, "54433 rows in the table"); + Assert.eq(table1.size(), "table1.size()", 54433, "rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2), instructions); - Assert.eq(table2.size(), "table2.size()", 72551, "72551 rows in the table"); + Assert.eq(table2.size(), "table2.size()", 72551, "rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3), instructions); - Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table3.size(), "table3.size()", 100_000, "rows in the table"); Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table4 = adapter.readTable(tableId, snapshots.get(4), instructions); - Assert.eq(table4.size(), "table4.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table4.size(), "table4.size()", 100_000, "rows in the table"); Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table5 = adapter.readTable(tableId, snapshots.get(5), instructions); - Assert.eq(table5.size(), "table5.size()", 0, "0 rows in the table"); + Assert.eq(table5.size(), "table5.size()", 0, "rows in the table"); Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); // Test the string versions of the methods // Verify we retrieved all the rows. table0 = adapter.readTable("sales.sales_multi", snapshots.get(0).snapshotId(), instructions); - Assert.eq(table0.size(), "table0.size()", 18073, "18073 rows in the table"); + Assert.eq(table0.size(), "table0.size()", 18073, "rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); table1 = adapter.readTable("sales.sales_multi", snapshots.get(1).snapshotId(), instructions); - Assert.eq(table1.size(), "table1.size()", 54433, "54433 rows in the table"); + Assert.eq(table1.size(), "table1.size()", 54433, "rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); table2 = adapter.readTable("sales.sales_multi", snapshots.get(2).snapshotId(), instructions); - Assert.eq(table2.size(), "table2.size()", 72551, "72551 rows in the table"); + Assert.eq(table2.size(), "table2.size()", 72551, "rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); table3 = adapter.readTable("sales.sales_multi", snapshots.get(3).snapshotId(), instructions); - Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table3.size(), "table3.size()", 100_000, "rows in the table"); Assert.equals(table3.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); table4 = adapter.readTable("sales.sales_multi", snapshots.get(4).snapshotId(), instructions); - Assert.eq(table4.size(), "table4.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table4.size(), "table4.size()", 100_000, "rows in the table"); Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); table5 = adapter.readTable("sales.sales_multi", snapshots.get(5).snapshotId(), instructions); - Assert.eq(table5.size(), "table5.size()", 0, "0 rows in the table"); + Assert.eq(table5.size(), "table5.size()", 0, "rows in the table"); Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @@ -843,7 +847,7 @@ public void testOpenAllTypesTable() throws ExecutionException, InterruptedExcept // Verify we retrieved all the rows. final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions).select(); - Assert.eq(table.size(), "table.size()", 10, "10 rows in the table"); + Assert.eq(table.size(), "table.size()", 10, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", ALL_TYPES_DEF); } @@ -883,25 +887,25 @@ public void testTableDefinitionTable() { // Use string and current snapshot Table tableDefTable = adapter.getTableDefinitionTable("sales.sales_multi", null); - Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "5 rows in the table"); + Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); // Use TableIdentifier and Snapshot tableDefTable = adapter.getTableDefinitionTable(tableId, null); - Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "5 rows in the table"); + Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); // Use string and long snapshot ID tableDefTable = adapter.getTableDefinitionTable("sales.sales_multi", snapshots.get(0).snapshotId(), null); - Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "5 rows in the table"); + Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); // Use TableIdentifier and Snapshot tableDefTable = adapter.getTableDefinitionTable(tableId, snapshots.get(0), null); - Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "5 rows in the table"); + Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); } @@ -947,7 +951,9 @@ public void testTableDefinitionWithInstructions() { } @Test - public void testManualRefreshingTable() { + public void testManualRefreshingTable() throws ExecutionException, InterruptedException, TimeoutException { + uploadSalesMulti(); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); final IcebergInstructions localInstructions = IcebergInstructions.builder() @@ -959,29 +965,76 @@ public void testManualRefreshingTable() { final List snapshots = tableAdapter.listSnapshots(); - // Use string and current snapshot + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + final IcebergTable table = tableAdapter.table(snapshots.get(0).snapshotId(), localInstructions); - Assert.eq(table.size(), "table.size()", 18073, "18073 rows in the table"); + Assert.eq(table.size(), "table.size()", 18073, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - table.update(snapshots.get(1).snapshotId()); - Assert.eq(table.size(), "table.size()", 54433, "54433 rows in the table"); + updateGraph.runWithinUnitTestCycle(() ->{ + table.update(snapshots.get(1).snapshotId()); + }); + Assert.eq(table.size(), "table.size()", 54433, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(2).snapshotId()); - Assert.eq(table.size(), "table.size()", 72551, "72551 rows in the table"); + Assert.eq(table.size(), "table.size()", 72551, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(3).snapshotId()); - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(4).snapshotId()); - Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + table.update(snapshots.get(5).snapshotId()); + Assert.eq(table.size(), "table.size()", 0, "rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + } + + @Test + public void testAutoRefreshingTable() throws ExecutionException, InterruptedException, TimeoutException { + uploadSalesMulti(); + + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); + + final IcebergInstructions localInstructions = IcebergInstructions.builder() + .dataInstructions(instructions.dataInstructions().get()) + .updateMode(IcebergUpdateMode.manualRefreshingMode()) + .build(); + + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); + + final List snapshots = tableAdapter.listSnapshots(); + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final IcebergTable table = tableAdapter.table(snapshots.get(0).snapshotId(), localInstructions); + Assert.eq(table.size(), "table.size()", 18073, "rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + updateGraph.runWithinUnitTestCycle(() ->{ + table.update(snapshots.get(1).snapshotId()); + }); + Assert.eq(table.size(), "table.size()", 54433, "rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + table.update(snapshots.get(2).snapshotId()); + Assert.eq(table.size(), "table.size()", 72551, "rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + table.update(snapshots.get(3).snapshotId()); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + + table.update(snapshots.get(4).snapshotId()); + Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(5).snapshotId()); - Assert.eq(table.size(), "table.size()", 0, "0 rows in the table"); + Assert.eq(table.size(), "table.size()", 0, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index c06531b6e8f..45684617998 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -506,12 +506,4 @@ public IcebergTableAdapter loadTable(final String tableIdentifier) { public Catalog catalog() { return catalog; } - - /** - * Returns the underlying Iceberg {@link FileIO} used by this adapter. - */ - @SuppressWarnings("unused") - public FileIO fileIO() { - return fileIO; - } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java new file mode 100644 index 00000000000..76688e0b421 --- /dev/null +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java @@ -0,0 +1,218 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.TestCatalog; + +import org.apache.iceberg.*; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.LocationProvider; +import org.testcontainers.shaded.org.apache.commons.lang3.NotImplementedException; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class IcebergRefreshingTestTable implements Table { + private final IcebergTestTable testTable; + + private long currentSnapshotSequenceNumber; + + private IcebergRefreshingTestTable(final IcebergTestTable testTable) { + this.testTable = testTable; + currentSnapshotSequenceNumber = 1; + } + + public static IcebergRefreshingTestTable fromTestTable(final IcebergTestTable testTable) { + return new IcebergRefreshingTestTable(testTable); + } + + public void advanceSequenceNumber() { + currentSnapshotSequenceNumber++; + } + + @Override + public void refresh() {} + + @Override + public TableScan newScan() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public Schema schema() { + return testTable.schema(); + } + + @Override + public Map schemas() { + return testTable.schemas(); + } + + @Override + public PartitionSpec spec() { + return testTable.spec(); + } + + @Override + public Map specs() { + return testTable.specs(); + + } + + @Override + public SortOrder sortOrder() { + return testTable.sortOrder(); + } + + @Override + public Map sortOrders() { + return testTable.sortOrders(); + + } + + @Override + public Map properties() { + return testTable.properties(); + } + + @Override + public String location() { + return testTable.location(); + } + + @Override + public Snapshot currentSnapshot() { + Snapshot snapshot = null; + for (final Snapshot s : snapshots()) { + snapshot = s; // grab the last snapshot + } + return snapshot; + } + + @Override + public Snapshot snapshot(long l) { + for (final Snapshot snapshot : snapshots()) { + if (snapshot.snapshotId() == l) { + return snapshot; + } + } + return null; + } + + @Override + public Iterable snapshots() { + final List snapshots = new ArrayList<>(); + for (final Snapshot s : testTable.snapshots()) { + if (s.sequenceNumber() <= currentSnapshotSequenceNumber) { + snapshots.add(s); + } + } + return snapshots; + } + + @Override + public List history() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public UpdateSchema updateSchema() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public UpdatePartitionSpec updateSpec() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public UpdateProperties updateProperties() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public ReplaceSortOrder replaceSortOrder() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public UpdateLocation updateLocation() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public AppendFiles newAppend() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public RewriteFiles newRewrite() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public RewriteManifests rewriteManifests() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public OverwriteFiles newOverwrite() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public RowDelta newRowDelta() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public ReplacePartitions newReplacePartitions() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public DeleteFiles newDelete() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public ExpireSnapshots expireSnapshots() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public ManageSnapshots manageSnapshots() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public Transaction newTransaction() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public FileIO io() { + return testTable.io(); + } + + @Override + public EncryptionManager encryption() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public LocationProvider locationProvider() { + throw new NotImplementedException("Not implemented"); + } + + @Override + public List statisticsFiles() { + return testTable.statisticsFiles(); + } + + @Override + public Map refs() { + throw new NotImplementedException("Not implemented"); + } +} From 5c7ff1296a93050cb076286d660bfda7f6f95677 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 4 Sep 2024 11:42:02 -0700 Subject: [PATCH 25/56] Manual and auto refreshing tests for Iceberg, not passing. --- .../deephaven/iceberg/util/IcebergToolsTest.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 66b1a08ef3b..5918f180987 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -11,12 +11,12 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.testutil.ControlledUpdateGraph; -import io.deephaven.engine.testutil.GenerateTableUpdates; -import io.deephaven.engine.testutil.TstUtils; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.extensions.s3.S3Instructions; +import io.deephaven.iceberg.TestCatalog.IcebergRefreshingTestTable; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; import io.deephaven.iceberg.TestCatalog.IcebergTestFileIO; +import io.deephaven.iceberg.TestCatalog.IcebergTestTable; import io.deephaven.test.types.OutOfBandTest; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; @@ -998,14 +998,18 @@ public void testManualRefreshingTable() throws ExecutionException, InterruptedEx public void testAutoRefreshingTable() throws ExecutionException, InterruptedException, TimeoutException { uploadSalesMulti(); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); + final TableIdentifier tableId = TableIdentifier.parse("sales.sales_multi"); + + // Create a custom table adapter on top of the refreshing Iceberge table. + final IcebergRefreshingTestTable icebergTable = IcebergRefreshingTestTable.fromTestTable( + (IcebergTestTable) resourceCatalog.loadTable(tableId)); + final IcebergTableAdapter tableAdapter = new IcebergTableAdapter(tableId, icebergTable); final IcebergInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) - .updateMode(IcebergUpdateMode.manualRefreshingMode()) + .updateMode(IcebergUpdateMode.autoRefreshingMode(1)) .build(); - final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); final List snapshots = tableAdapter.listSnapshots(); @@ -1015,8 +1019,8 @@ public void testAutoRefreshingTable() throws ExecutionException, InterruptedExce Assert.eq(table.size(), "table.size()", 18073, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + icebergTable.advanceSequenceNumber(); updateGraph.runWithinUnitTestCycle(() ->{ - table.update(snapshots.get(1).snapshotId()); }); Assert.eq(table.size(), "table.size()", 54433, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); From 92eec6140c13bd4748f0bc467cfd5509154f7644 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 4 Sep 2024 15:37:18 -0700 Subject: [PATCH 26/56] PR comments addressed. --- .../impl/locations/TableLocationProvider.java | 59 ++----- .../locations/impl/AbstractTableLocation.java | 2 +- .../impl/AbstractTableLocationProvider.java | 165 +++++++++--------- .../impl/FilteredTableDataService.java | 16 +- .../impl/TableLocationSubscriptionBuffer.java | 18 +- .../iceberg/util/IcebergToolsTest.java | 4 +- 6 files changed, 107 insertions(+), 157 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index b6205431a59..e74b5db56e2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -40,24 +40,6 @@ interface Listener extends BasicTableDataListener { */ void endTransaction(@NotNull Object token); - /** - *

- * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location - * subscription. This should occur at most once per location, but the order of delivery is not - * guaranteed. - *

- *

- * If transactionToken is {@code null}, the key will be added to the pending additions immediately. - *

- * - * @param tableLocationKey The new table location key. - * @param transactionToken The token identifying the transaction, or {@code null} if this addition is not part - * of a transaction. - */ - void handleTableLocationKeyAdded( - @NotNull ImmutableTableLocationKey tableLocationKey, - @Nullable Object transactionToken); - /** * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location * subscription. This should occur at most once per location, but the order of delivery is not @@ -66,50 +48,33 @@ void handleTableLocationKeyAdded( * * @param tableLocationKey The new table location key. */ - default void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey) { - handleTableLocationKeyAdded(tableLocationKey, null); - } + void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey); /** - *

- * Notify the listener of a {@link TableLocationKey} that has been removed. - *

- *

- * If transactionToken is {@code null}, the key will be added to the pending removals immediately. - *

- * - * @param tableLocationKey The table location key that was removed. - * @param transactionToken The token identifying the transaction, or {@code null} if this addition is not part - * of a transaction. - */ - void handleTableLocationKeyRemoved( - @NotNull ImmutableTableLocationKey tableLocationKey, - @Nullable Object transactionToken); - - /** - * Notify the listener of a {@link TableLocationKey} that has been removed. This addition is not part of any - * transaction, and is equivalent to {@code handleTableLocationKeyAdded(tableLocationKey, null);} by default. + * Notify the listener of a {@link TableLocationKey} that has been removed. This removal is not part of any + * transaction, and is equivalent to {@code handleTableLocationKeyRemoved(tableLocationKey, null);} by default. * * @param tableLocationKey The table location key that was removed. */ @SuppressWarnings("unused") - default void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey) { - handleTableLocationKeyRemoved(tableLocationKey, null); - } + void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey); /** *

* Notify the listener of collections of {@link TableLocationKey TableLocationKeys} added or removed while - * initiating or maintaining the location subscription. This should occur at most once per location, but the - * order of delivery is not guaranteed. + * initiating or maintaining the location subscription. Addition or removal should occur at most once per + * location, but the order of delivery is not guaranteed. *

* * @param addedKeys Collection of table location keys that were added. * @param removedKeys Collection of table location keys that were removed. */ - void handleTableLocationKeysUpdate( - @Nullable Collection addedKeys, - @Nullable Collection removedKeys); + default void handleTableLocationKeysUpdate( + @NotNull Collection addedKeys, + @NotNull Collection removedKeys) { + removedKeys.forEach(this::handleTableLocationKeyRemoved); + addedKeys.forEach(this::handleTableLocationKeyAdded); + } } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index 1c3f429fad7..d945e7c12fc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java @@ -258,7 +258,7 @@ public final void incrementReferenceCount() { /** * Decrement the reference count by one, when the reference count reaches zero this location will be cleared. * - * @throws IllegalStateException If the reference count was not successfully incremented + * @throws IllegalStateException If the reference count was not successfully decremented */ public void decrementReferenceCount() { referenceCounted.decrementReferenceCount(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 3b49fa56634..721e8a6cbc0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -25,46 +25,55 @@ public abstract class AbstractTableLocationProvider extends SubscriptionAggregator implements TableLocationProvider { - private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet(); + private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet(); /** * Helper class to manage a transaction of added and removed location keys. */ private static class Transaction { - Set locationsAdded = EMPTY_TABLE_LOCATION_KEYS; - Set locationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + Set locationsAdded = EMPTY_TABLE_LOCATION_KEYS; + Set locationsRemoved = EMPTY_TABLE_LOCATION_KEYS; - void addlocationKey(TableLocationKey locationKey) { + void addlocationKey(ImmutableTableLocationKey locationKey) { if (locationsAdded == EMPTY_TABLE_LOCATION_KEYS) { locationsAdded = new HashSet<>(); - } else if (locationsAdded.contains(locationKey)) { + } + // When adding a location in a transaction, check for logical consistency. + // 1. If the location was already added in this transaction, we have a problem. A transaction should not + // add the same location twice. + // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a + // logical error (although it may not be supported by all consumers). + if (!locationsAdded.add(locationKey)) { throw new IllegalStateException("TableLocationKey " + locationKey + " was already added in this transaction."); } - locationsAdded.add(locationKey); } - void removeLocationKey(TableLocationKey locationKey) { + void removeLocationKey(ImmutableTableLocationKey locationKey) { if (locationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { locationsRemoved = new HashSet<>(); - } else if (locationsRemoved.contains(locationKey)) { + } + // When removing a location in a transaction, check for logical consistency. + // 1. If the location was already removed in this transaction, we have a problem. A transaction should not + // remove the same location twice. + if (!locationsRemoved.add(locationKey)) { throw new IllegalStateException("TableLocationKey " + locationKey + " was already removed and has not been replaced."); - } else if (locationsAdded.contains(locationKey)) { + } + // 2. If the location was already added in this transaction, we have a problem. A transaction should not + // add then remove the same location. + if (locationsAdded.contains(locationKey)) { throw new IllegalStateException("TableLocationKey " + locationKey + " was removed after being added in the same transaction."); } - locationsRemoved.add(locationKey); } } private final ImmutableTableKey tableKey; - // These sets represent open transactions that are being accumulated. + // Open transactions that are being accumulated private final Map transactions = new HashMap<>(); - private final Object transactionLock = new Object(); - /** * Map from {@link TableLocationKey} to itself, or to a {@link TableLocation}. The values are {@link TableLocation}s * if: @@ -122,7 +131,7 @@ public final ImmutableTableKey getKey() { @Override protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) { - listener.handleTableLocationKeysUpdate(unmodifiableTableLocationKeys, null); + listener.handleTableLocationKeysUpdate(unmodifiableTableLocationKeys, List.of()); } /** @@ -131,12 +140,14 @@ protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider * @param token A token to identify the transaction */ protected void beginTransaction(@NotNull final Object token) { - synchronized (transactionLock) { + synchronized (transactions) { // Verify that we can start a new transaction with this token. - if (transactions.containsKey(token)) { - throw new IllegalStateException("A transaction with token " + token + " is currently open."); - } - transactions.put(token, new Transaction()); + transactions.compute(token, (key, val) -> { + if (val != null) { + throw new IllegalStateException("A transaction with token " + token + " is already open."); + } + return new Transaction(); + }); } } @@ -147,7 +158,7 @@ protected void beginTransaction(@NotNull final Object token) { */ protected void endTransaction(@NotNull final Object token) { final Transaction transaction; - synchronized (transactionLock) { + synchronized (transactions) { // Verify that this transaction is open. transaction = transactions.remove(token); if (transaction == null) { @@ -155,44 +166,42 @@ protected void endTransaction(@NotNull final Object token) { } } - final Collection addedImmutableKeys = + final Collection addedKeys = new ArrayList<>(transaction.locationsAdded.size()); - final Collection removedImmutableKeys = + final Collection removedKeys = new ArrayList<>(transaction.locationsRemoved.size()); - // Process the accumulated adds and removes under a lock on `tableLocations` to keep modifications atomic to - // other holders of this lock. - synchronized (tableLocations) { - if (transaction.locationsAdded != EMPTY_TABLE_LOCATION_KEYS - || transaction.locationsRemoved != EMPTY_TABLE_LOCATION_KEYS) { - for (TableLocationKey locationKey : transaction.locationsAdded) { - locationCreatedRecorder = false; - final Object result = tableLocations.putIfAbsent(locationKey, this::observeInsert); - visitLocationKey(locationKey); - if (locationCreatedRecorder) { - verifyPartitionKeys(locationKey); - addedImmutableKeys.add(toKeyImmutable(result)); - } - } + // Return early if there are no changes to process. + if (transaction.locationsAdded == EMPTY_TABLE_LOCATION_KEYS + && transaction.locationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + return; + } - for (TableLocationKey locationKey : transaction.locationsRemoved) { - final Object removedLocation = tableLocations.remove(locationKey); - if (removedLocation != null) { - removedImmutableKeys.add(toKeyImmutable(locationKey)); - } - } + for (ImmutableTableLocationKey locationKey : transaction.locationsRemoved) { + final Object removedLocation = tableLocations.remove(locationKey); + if (removedLocation != null) { + removedKeys.add(locationKey); } } - if (subscriptions != null) { + for (ImmutableTableLocationKey locationKey : transaction.locationsAdded) { + locationCreatedRecorder = false; + tableLocations.putIfAbsent(locationKey, this::observeInsert); + visitLocationKey(locationKey); + if (locationCreatedRecorder) { + verifyPartitionKeys(locationKey); + addedKeys.add(locationKey); + } + } + + if (subscriptions != null && (!addedKeys.isEmpty() || !removedKeys.isEmpty())) { synchronized (subscriptions) { // Push the notifications to the subscribers. - if ((!addedImmutableKeys.isEmpty() || !removedImmutableKeys.isEmpty()) - && subscriptions.deliverNotification( - Listener::handleTableLocationKeysUpdate, - addedImmutableKeys, - removedImmutableKeys, - true)) { + if (subscriptions.deliverNotification( + Listener::handleTableLocationKeysUpdate, + addedKeys, + removedKeys, + true)) { onEmpty(); } } @@ -220,29 +229,27 @@ protected final void handleTableLocationKeyAdded( @NotNull final TableLocationKey locationKey, @Nullable final Object transactionToken) { - if (!supportsSubscriptions()) { - tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable); - visitLocationKey(locationKey); - return; - } - if (transactionToken != null) { - // When adding a location in a transaction, check for logical consistency. - // 1. If the location was already added in this transaction, we have a problem. A transaction should not - // add the same location twice. - // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a - // logical error (although it may not be supported by all consumers). - synchronized (transactionLock) { - final Transaction transaction = transactions.get(transactionToken); + final Transaction transaction; + synchronized (transactions) { + transaction = transactions.get(transactionToken); if (transaction == null) { throw new IllegalStateException( "No transaction with token " + transactionToken + " is currently open."); } - transaction.addlocationKey(locationKey); } + // Store an immutable key + transaction.addlocationKey(toKeyImmutable(locationKey)); return; } + if (!supportsSubscriptions()) { + tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable); + visitLocationKey(locationKey); + return; + } + + // If we're not in a transaction, we should push this key immediately. synchronized (subscriptions) { // Since we're holding the lock on subscriptions, the following code is overly complicated - we could @@ -328,12 +335,9 @@ protected void doInitialization() { @Override @NotNull public final Collection getTableLocationKeys() { - // This lock is held while `endTransaction()` updates `tableLocations` with the accumulated adds/removes. - // Locking here ensures that this call won't return while `tableLocations` (and `unmodifiableTableLocationKeys`) - // contain a partial transaction. - synchronized (tableLocations) { - return unmodifiableTableLocationKeys; - } + final List result = new ArrayList<>(tableLocations.size()); + tableLocations.keySet().forEach(key -> result.add((ImmutableTableLocationKey) key)); + return result; } @Override @@ -413,25 +417,22 @@ protected final void handleTableLocationKeyRemoved(@NotNull final TableLocationK protected void handleTableLocationKeyRemoved( @NotNull final TableLocationKey locationKey, @Nullable final Object transactionToken) { - if (!supportsSubscriptions()) { - return; - } - - // When removing a location in a transaction, check for logical consistency. - // 1. If the location was already removed in this transaction, we have a problem. A transaction should not - // remove the same location twice. - // 2. If the location was already added in this transaction, we have a problem. A transaction should not - // add then remove the same location. if (transactionToken != null) { - synchronized (transactionLock) { - final Transaction transaction = transactions.get(transactionToken); + final Transaction transaction; + synchronized (transactions) { + transaction = transactions.get(transactionToken); if (transaction == null) { throw new IllegalStateException( "No transaction with token " + transactionToken + " is currently open."); } - transaction.removeLocationKey(locationKey); - return; } + transaction.removeLocationKey(toKeyImmutable(locationKey)); + return; + } + + if (!supportsSubscriptions()) { + tableLocations.remove(locationKey); + return; } // If we're not in a transaction, we should push this key immediately. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 948fb1fe38b..7aa365c1ce1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -175,31 +175,27 @@ public void endTransaction(@NotNull final Object token) { } @Override - public void handleTableLocationKeyAdded( - @NotNull final ImmutableTableLocationKey tableLocationKey, - @Nullable final Object transactionToken) { + public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) { final TableLocationProvider.Listener outputListener = getWrapped(); // We can't try to clean up null listeners here, the underlying implementation may not allow concurrent // unsubscribe operations. if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { - outputListener.handleTableLocationKeyAdded(tableLocationKey, transactionToken); + outputListener.handleTableLocationKeyAdded(tableLocationKey); } } @Override - public void handleTableLocationKeyRemoved( - @NotNull final ImmutableTableLocationKey tableLocationKey, - @Nullable final Object transactionToken) { + public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) { final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { - outputListener.handleTableLocationKeyRemoved(tableLocationKey, transactionToken); + outputListener.handleTableLocationKeyRemoved(tableLocationKey); } } @Override public void handleTableLocationKeysUpdate( - @Nullable Collection addedKeys, - @Nullable Collection removedKeys) { + @NotNull Collection addedKeys, + @NotNull Collection removedKeys) { final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null) { outputListener.handleTableLocationKeysUpdate(addedKeys, removedKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index 33a121c95ca..5ac8d7aee1b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -71,7 +71,7 @@ public synchronized LocationUpdate processPending() { // NB: Providers that don't support subscriptions don't tick - this single call to run is // sufficient. tableLocationProvider.refresh(); - handleTableLocationKeysUpdate(tableLocationProvider.getTableLocationKeys(), null); + handleTableLocationKeysUpdate(tableLocationProvider.getTableLocationKeys(), List.of()); } subscribed = true; } @@ -126,14 +126,8 @@ public void endTransaction(@NotNull final Object token) { } @Override - public void handleTableLocationKeyAdded( - @NotNull final ImmutableTableLocationKey tableLocationKey, - @Nullable Object transactionToken) { + public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) { synchronized (updateLock) { - if (transactionToken != null) { - throw new UnsupportedOperationException("Transactions are not supported by this provider."); - } - // Need to verify that we don't have stacked adds (without intervening removes). if (pendingLocationsAdded.contains(tableLocationKey)) { throw new IllegalStateException("TableLocationKey " + tableLocationKey @@ -147,14 +141,8 @@ public void handleTableLocationKeyAdded( } @Override - public void handleTableLocationKeyRemoved( - @NotNull final ImmutableTableLocationKey tableLocationKey, - final Object transactionToken) { + public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) { synchronized (updateLock) { - if (transactionToken != null) { - throw new UnsupportedOperationException("Transactions are not supported by this provider."); - } - // If we have a pending add, it is being cancelled by this remove. if (pendingLocationsAdded.remove(tableLocationKey)) { return; diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 5918f180987..871337cd2e0 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -971,7 +971,7 @@ public void testManualRefreshingTable() throws ExecutionException, InterruptedEx Assert.eq(table.size(), "table.size()", 18073, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - updateGraph.runWithinUnitTestCycle(() ->{ + updateGraph.runWithinUnitTestCycle(() -> { table.update(snapshots.get(1).snapshotId()); }); Assert.eq(table.size(), "table.size()", 54433, "rows in the table"); @@ -1020,7 +1020,7 @@ public void testAutoRefreshingTable() throws ExecutionException, InterruptedExce Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); icebergTable.advanceSequenceNumber(); - updateGraph.runWithinUnitTestCycle(() ->{ + updateGraph.runWithinUnitTestCycle(() -> { }); Assert.eq(table.size(), "table.size()", 54433, "rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); From 95194b723e88e09d090b925f8afc8ec4d89d4100 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 4 Sep 2024 17:55:11 -0700 Subject: [PATCH 27/56] Implemented improved location reference counting in AbstractTableLocationProvider and RegionedColumnSourceManager --- .../impl/AbstractTableLocationProvider.java | 115 +++++++++--------- .../regioned/RegionedColumnSourceManager.java | 32 ++--- 2 files changed, 73 insertions(+), 74 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 721e8a6cbc0..67f61501e3c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -34,7 +34,7 @@ private static class Transaction { Set locationsAdded = EMPTY_TABLE_LOCATION_KEYS; Set locationsRemoved = EMPTY_TABLE_LOCATION_KEYS; - void addlocationKey(ImmutableTableLocationKey locationKey) { + synchronized void addLocationKey(ImmutableTableLocationKey locationKey) { if (locationsAdded == EMPTY_TABLE_LOCATION_KEYS) { locationsAdded = new HashSet<>(); } @@ -49,7 +49,7 @@ void addlocationKey(ImmutableTableLocationKey locationKey) { } } - void removeLocationKey(ImmutableTableLocationKey locationKey) { + synchronized void removeLocationKey(ImmutableTableLocationKey locationKey) { if (locationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { locationsRemoved = new HashSet<>(); } @@ -72,7 +72,7 @@ void removeLocationKey(ImmutableTableLocationKey locationKey) { private final ImmutableTableKey tableKey; // Open transactions that are being accumulated - private final Map transactions = new HashMap<>(); + private final Map transactions = Collections.synchronizedMap(new HashMap<>()); /** * Map from {@link TableLocationKey} to itself, or to a {@link TableLocation}. The values are {@link TableLocation}s @@ -140,15 +140,13 @@ protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider * @param token A token to identify the transaction */ protected void beginTransaction(@NotNull final Object token) { - synchronized (transactions) { - // Verify that we can start a new transaction with this token. - transactions.compute(token, (key, val) -> { - if (val != null) { - throw new IllegalStateException("A transaction with token " + token + " is already open."); - } - return new Transaction(); - }); - } + // Verify that we can start a new transaction with this token. + transactions.compute(token, (key, val) -> { + if (val != null) { + throw new IllegalStateException("A transaction with token " + token + " is already open."); + } + return new Transaction(); + }); } /** @@ -157,40 +155,42 @@ protected void beginTransaction(@NotNull final Object token) { * @param token A token to identify the transaction */ protected void endTransaction(@NotNull final Object token) { - final Transaction transaction; - synchronized (transactions) { - // Verify that this transaction is open. - transaction = transactions.remove(token); - if (transaction == null) { - throw new IllegalStateException("No transaction with token " + token + " is currently open."); - } + // Verify that this transaction is open. + final Transaction transaction = transactions.remove(token); + if (transaction == null) { + throw new IllegalStateException("No transaction with token " + token + " is currently open."); } - final Collection addedKeys = - new ArrayList<>(transaction.locationsAdded.size()); - final Collection removedKeys = - new ArrayList<>(transaction.locationsRemoved.size()); - // Return early if there are no changes to process. if (transaction.locationsAdded == EMPTY_TABLE_LOCATION_KEYS && transaction.locationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { return; } - for (ImmutableTableLocationKey locationKey : transaction.locationsRemoved) { - final Object removedLocation = tableLocations.remove(locationKey); - if (removedLocation != null) { - removedKeys.add(locationKey); + final Collection addedKeys = + new ArrayList<>(transaction.locationsAdded.size()); + final Collection removedKeys = + new ArrayList<>(transaction.locationsRemoved.size()); + + synchronized (tableLocations) { + for (ImmutableTableLocationKey locationKey : transaction.locationsRemoved) { + final Object removedLocation = tableLocations.remove(locationKey); + if (removedLocation != null) { + if (removedLocation instanceof AbstractTableLocation) { + ((AbstractTableLocation) removedLocation).decrementReferenceCount(); + } + removedKeys.add(locationKey); + } } - } - for (ImmutableTableLocationKey locationKey : transaction.locationsAdded) { - locationCreatedRecorder = false; - tableLocations.putIfAbsent(locationKey, this::observeInsert); - visitLocationKey(locationKey); - if (locationCreatedRecorder) { - verifyPartitionKeys(locationKey); - addedKeys.add(locationKey); + for (ImmutableTableLocationKey locationKey : transaction.locationsAdded) { + locationCreatedRecorder = false; + tableLocations.putIfAbsent(locationKey, this::observeInsert); + visitLocationKey(locationKey); + if (locationCreatedRecorder) { + verifyPartitionKeys(locationKey); + addedKeys.add(locationKey); + } } } @@ -230,16 +230,13 @@ protected final void handleTableLocationKeyAdded( @Nullable final Object transactionToken) { if (transactionToken != null) { - final Transaction transaction; - synchronized (transactions) { - transaction = transactions.get(transactionToken); - if (transaction == null) { - throw new IllegalStateException( - "No transaction with token " + transactionToken + " is currently open."); - } + final Transaction transaction = transactions.get(transactionToken); + if (transaction == null) { + throw new IllegalStateException( + "No transaction with token " + transactionToken + " is currently open."); } // Store an immutable key - transaction.addlocationKey(toKeyImmutable(locationKey)); + transaction.addLocationKey(toKeyImmutable(locationKey)); return; } @@ -335,9 +332,11 @@ protected void doInitialization() { @Override @NotNull public final Collection getTableLocationKeys() { - final List result = new ArrayList<>(tableLocations.size()); - tableLocations.keySet().forEach(key -> result.add((ImmutableTableLocationKey) key)); - return result; + synchronized (tableLocations) { + // Note that we never store keys that aren't immutable. + // noinspection unchecked,rawtypes + return new ArrayList<>((Set) (Set) tableLocations.keySet()); + } } @Override @@ -363,6 +362,9 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t if (immutableKey == current) { // Note, this may contend for the lock on tableLocations tableLocations.add(current = makeTableLocation(immutableKey)); + if (current instanceof AbstractTableLocation) { + ((AbstractTableLocation) current).incrementReferenceCount(); + } } } } @@ -418,20 +420,20 @@ protected void handleTableLocationKeyRemoved( @NotNull final TableLocationKey locationKey, @Nullable final Object transactionToken) { if (transactionToken != null) { - final Transaction transaction; - synchronized (transactions) { - transaction = transactions.get(transactionToken); - if (transaction == null) { - throw new IllegalStateException( - "No transaction with token " + transactionToken + " is currently open."); - } + final Transaction transaction = transactions.get(transactionToken); + if (transaction == null) { + throw new IllegalStateException( + "No transaction with token " + transactionToken + " is currently open."); } transaction.removeLocationKey(toKeyImmutable(locationKey)); return; } if (!supportsSubscriptions()) { - tableLocations.remove(locationKey); + final Object removedLocation = tableLocations.remove(locationKey); + if (removedLocation instanceof AbstractTableLocation) { + ((AbstractTableLocation) removedLocation).decrementReferenceCount(); + } return; } @@ -439,6 +441,9 @@ protected void handleTableLocationKeyRemoved( synchronized (subscriptions) { final Object removedLocation = tableLocations.remove(locationKey); if (removedLocation != null) { + if (removedLocation instanceof AbstractTableLocation) { + ((AbstractTableLocation) removedLocation).decrementReferenceCount(); + } if (subscriptions.deliverNotification( Listener::handleTableLocationKeyRemoved, locationKey.makeImmutable(), diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index cb71d45053f..7eba2f7332f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -107,9 +107,6 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col @ReferentialIntegrity private final Collection retainedDataIndexes = new ArrayList<>(); - private final List locationsToClear; - private final UpdateCommitter locationClearCommitter; - /** * A reference to a delayed error notifier for the {@link #includedLocationsTable}, if one is pending. */ @@ -199,19 +196,6 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col } } - - locationsToClear = new ArrayList<>(); - locationClearCommitter = new UpdateCommitter<>(this, - ExecutionContext.getContext().getUpdateGraph(), - (ignored) -> { - locationsToClear.forEach(location -> { - location.handleUpdate(null, System.currentTimeMillis()); - location.clearColumnLocations(); - - }); - locationsToClear.clear(); - }); - invalidateCommitter = new UpdateCommitter<>(this, ExecutionContext.getContext().getUpdateGraph(), (ignored) -> { @@ -222,6 +206,16 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col }); } + @Override + public void destroy() { + super.destroy(); + includedTableLocations.keySet().forEach(location -> { + if (location instanceof AbstractTableLocation) { + ((AbstractTableLocation) location).decrementReferenceCount(); + } + }); + } + @Override public synchronized void addLocation(@NotNull final TableLocation tableLocation) { final IncludedTableLocationEntry includedLocation = includedTableLocations.get(tableLocation.getKey()); @@ -232,9 +226,6 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation) log.debug().append("LOCATION_ADDED:").append(tableLocation.toString()).endl(); } emptyTableLocations.add(new EmptyTableLocationEntry(tableLocation)); - if (tableLocation instanceof AbstractTableLocation) { - ((AbstractTableLocation) tableLocation).incrementReferenceCount(); - } } else { // Duplicate location - not allowed final TableLocation duplicateLocation = @@ -420,6 +411,9 @@ private TableUpdateImpl update(final boolean initializing) { includedTableLocations.add(entry); orderedIncludedTableLocations.add(entry); entry.processInitial(addedRowSetBuilder, entryToInclude.initialRowSet); + if (entry.location instanceof AbstractTableLocation) { + ((AbstractTableLocation) entry.location).incrementReferenceCount(); + } // We have a new location, add the row set to the table and mark the row as added. // @formatter:off From 09e2b6eada6521162a72a479dbfd1ee273c1ff08 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 5 Sep 2024 09:41:05 -0700 Subject: [PATCH 28/56] Fixing doc problem. --- .../impl/locations/impl/AbstractTableLocationProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 67f61501e3c..ce0bf61db09 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -16,7 +16,7 @@ * Partial {@link TableLocationProvider} implementation for standalone use or as part of a {@link TableDataService}. *

* Presents an interface similar to {@link TableLocationProvider.Listener} for subclasses to use when communicating with - * the parent; see {@link #handleTableLocationKeyAdded(TableLocationKey, Object). + * the parent; see {@link #handleTableLocationKeyAdded(TableLocationKey, Object)}. *

* Note that subclasses are responsible for determining when it's appropriate to call {@link #setInitialized()} and/or * override {@link #doInitialization()}. From 30910dd0d79301a0b36a9fe10c190a103eb22c54 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 11 Sep 2024 17:36:14 -0700 Subject: [PATCH 29/56] For review only, does not compile :( --- .../table/impl/ColumnSourceManager.java | 9 +- .../table/impl/PartitionAwareSourceTable.java | 10 +- .../table/impl/SourcePartitionedTable.java | 11 +- .../engine/table/impl/SourceTable.java | 42 +-- .../locations/ImmutableTableLocationKey.java | 3 - .../table/impl/locations/TableLocation.java | 3 +- .../impl/locations/TableLocationProvider.java | 40 ++- .../locations/TrackedTableLocationKey.java | 90 +++++++ .../locations/impl/AbstractTableLocation.java | 40 +-- .../impl/AbstractTableLocationProvider.java | 246 +++++++++++------- .../impl/CompositeTableDataService.java | 15 +- .../impl/FilteredTableDataService.java | 15 +- .../impl/SingleTableLocationProvider.java | 10 +- .../impl/TableLocationSubscriptionBuffer.java | 56 ++-- .../regioned/RegionedColumnSourceManager.java | 37 +-- 15 files changed, 413 insertions(+), 214 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java index bc56a42d246..080e11be44d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java @@ -3,13 +3,12 @@ // package io.deephaven.engine.table.impl; -import io.deephaven.engine.liveness.LivenessReferent; +import io.deephaven.engine.liveness.LivenessNode; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.TrackingWritableRowSet; -import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.engine.table.impl.locations.TrackedTableLocationKey; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -19,7 +18,7 @@ /** * Manager for ColumnSources in a Table. */ -public interface ColumnSourceManager extends LivenessReferent { +public interface ColumnSourceManager extends LivenessNode { /** * Get a map of name to {@link ColumnSource} for the column sources maintained by this manager. @@ -116,5 +115,5 @@ public interface ColumnSourceManager extends LivenessReferent { * @return true if the location key was actually removed * @param tableLocationKey the location key being removed */ - boolean removeLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey); + boolean removeLocationKey(@NotNull TrackedTableLocationKey tableLocationKey); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java index a43b6a88068..bdae90034e0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java @@ -7,6 +7,7 @@ import io.deephaven.api.filter.Filter; import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.locations.TrackedTableLocationKey; import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; @@ -221,11 +222,12 @@ private static ColumnSource makePartitionSource(@NotNull final Co } @Override - protected final Collection filterLocationKeys( - @NotNull final Collection foundLocationKeys) { + protected final Collection filterLocationKeys( + @NotNull final Collection foundLocationKeys) { if (partitioningColumnFilters.length == 0) { return foundLocationKeys; } + // TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table final List partitionTableColumnNames = Stream.concat( partitioningColumnDefinitions.keySet().stream(), @@ -243,6 +245,10 @@ protected final Collection filterLocationKeys( if (filteredColumnPartitionTable.size() == foundLocationKeys.size()) { return foundLocationKeys; } + + // TODO: Not sure what to do here. Seems like there is a big disconnect between the location keys and the + // tracked location keys. + final Iterable iterable = () -> filteredColumnPartitionTable.columnIterator(LOCATION_KEY_COLUMN_NAME); return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toList()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index e772241ca3a..0328f7a60f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -213,9 +213,14 @@ private Table makeConstituentTable(@NotNull final TableLocation tableLocation) { } private void processPendingLocations(final boolean notifyListeners) { - final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = subscriptionBuffer.processPending(); - final RowSet removed = processRemovals(locationUpdate); - final RowSet added = processAdditions(locationUpdate); + final RowSet removed; + final RowSet added; + + try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = + subscriptionBuffer.processPending()) { + removed = processRemovals(locationUpdate); + added = processAdditions(locationUpdate); + } resultRows.update(added, removed); if (notifyListeners) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index a9a6dce38cf..8d605464798 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -10,9 +10,7 @@ import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.TableUpdateListener; -import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; -import io.deephaven.engine.table.impl.locations.TableDataException; -import io.deephaven.engine.table.impl.locations.TableLocationProvider; +import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; import io.deephaven.engine.table.impl.locations.impl.TableLocationSubscriptionBuffer; @@ -140,11 +138,11 @@ private void initializeAvailableLocations() { if (isRefreshing()) { final TableLocationSubscriptionBuffer locationBuffer = new TableLocationSubscriptionBuffer(locationProvider); - final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = - locationBuffer.processPending(); - - maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); - maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); + try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = + locationBuffer.processPending()) { + maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); + maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); + } updateSourceRegistrar.addSource(locationChangePoller = new LocationChangePoller(locationBuffer)); } else { locationProvider.refresh(); @@ -155,24 +153,28 @@ private void initializeAvailableLocations() { } } - private void maybeAddLocations(@NotNull final Collection locationKeys) { + private void maybeAddLocations(@NotNull final Collection locationKeys) { if (locationKeys.isEmpty()) { return; } filterLocationKeys(locationKeys) .parallelStream() - .forEach(lk -> columnSourceManager.addLocation(locationProvider.getTableLocation(lk))); + .forEach(lk -> { + // Unconditionally manage all locations added to the column source manager + columnSourceManager.manage(lk); + columnSourceManager.addLocation(locationProvider.getTableLocation(lk)); + }); } - private ImmutableTableLocationKey[] maybeRemoveLocations( - @NotNull final Collection removedKeys) { + private TrackedTableLocationKey[] maybeRemoveLocations( + @NotNull final Collection removedKeys) { if (removedKeys.isEmpty()) { - return ImmutableTableLocationKey.ZERO_LENGTH_IMMUTABLE_TABLE_LOCATION_KEY_ARRAY; + return TrackedTableLocationKey.ZERO_LENGTH_TRACKED_TABLE_LOCATION_KEY_ARRAY; } return filterLocationKeys(removedKeys).stream() .filter(columnSourceManager::removeLocationKey) - .toArray(ImmutableTableLocationKey[]::new); + .toArray(TrackedTableLocationKey[]::new); } private void initializeLocationSizes() { @@ -212,10 +214,10 @@ private LocationChangePoller(@NotNull final TableLocationSubscriptionBuffer loca @Override protected void instrumentedRefresh() { - final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending(); - final ImmutableTableLocationKey[] removedKeys = - maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); - maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); + try(final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending()) { + maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); + maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); + } // This class previously had functionality to notify "location listeners", but it was never used. // Resurrect from git history if needed. @@ -252,8 +254,8 @@ protected void onRefreshError(@NotNull final Exception error) { * {@link TableLocationProvider}, but not yet incorporated into the table * @return A sub-collection of the input */ - protected Collection filterLocationKeys( - @NotNull final Collection foundLocationKeys) { + protected Collection filterLocationKeys( + @NotNull final Collection foundLocationKeys) { return foundLocationKeys; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/ImmutableTableLocationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/ImmutableTableLocationKey.java index 407d6df9917..96799cf5a42 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/ImmutableTableLocationKey.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/ImmutableTableLocationKey.java @@ -12,9 +12,6 @@ */ @Immutable public interface ImmutableTableLocationKey extends TableLocationKey { - - ImmutableTableLocationKey[] ZERO_LENGTH_IMMUTABLE_TABLE_LOCATION_KEY_ARRAY = new ImmutableTableLocationKey[0]; - @FinalDefault default ImmutableTableLocationKey makeImmutable() { return this; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java index 9d3eb5b55e7..a3d3ad99011 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java @@ -6,6 +6,7 @@ import io.deephaven.api.SortColumn; import io.deephaven.base.log.LogOutput; import io.deephaven.base.log.LogOutputAppendable; +import io.deephaven.engine.liveness.LivenessNode; import io.deephaven.engine.table.BasicDataIndex; import io.deephaven.engine.table.Table; import io.deephaven.io.log.impl.LogOutputStringImpl; @@ -21,7 +22,7 @@ * location allows access to columns, size, and possibly other metadata for a single partition that may be included in a * source table. */ -public interface TableLocation extends NamedImplementation, LogOutputAppendable, TableLocationState { +public interface TableLocation extends NamedImplementation, LogOutputAppendable, TableLocationState, LivenessNode { /** * Listener interface for anything that wants to know about changes to a location. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index e74b5db56e2..6dbac0dcee0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -3,11 +3,13 @@ // package io.deephaven.engine.table.impl.locations; +import io.deephaven.engine.liveness.LivenessManager; import io.deephaven.util.type.NamedImplementation; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Collection; +import java.util.function.Predicate; /** * Discovery utility for {@link TableLocation}s for a given table. @@ -41,23 +43,24 @@ interface Listener extends BasicTableDataListener { void endTransaction(@NotNull Object token); /** - * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location - * subscription. This should occur at most once per location, but the order of delivery is not + * Notify the listener of a {@link TrackedTableLocationKey} encountered while initiating or maintaining the + * location subscription. This should occur at most once per location, but the order of delivery is not * guaranteed. This addition is not part of any transaction, and is equivalent to * {@code handleTableLocationKeyAdded(tableLocationKey, null);} by default. * * @param tableLocationKey The new table location key. */ - void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey); + void handleTableLocationKeyAdded(@NotNull TrackedTableLocationKey tableLocationKey); /** - * Notify the listener of a {@link TableLocationKey} that has been removed. This removal is not part of any - * transaction, and is equivalent to {@code handleTableLocationKeyRemoved(tableLocationKey, null);} by default. + * Notify the listener of a {@link TrackedTableLocationKey} that has been removed. This removal is not part of + * any transaction, and is equivalent to {@code handleTableLocationKeyRemoved(tableLocationKey, null);} by + * default. * * @param tableLocationKey The table location key that was removed. */ @SuppressWarnings("unused") - void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey); + void handleTableLocationKeyRemoved(@NotNull TrackedTableLocationKey tableLocationKey); /** *

@@ -70,8 +73,8 @@ interface Listener extends BasicTableDataListener { * @param removedKeys Collection of table location keys that were removed. */ default void handleTableLocationKeysUpdate( - @NotNull Collection addedKeys, - @NotNull Collection removedKeys) { + @NotNull Collection addedKeys, + @NotNull Collection removedKeys) { removedKeys.forEach(this::handleTableLocationKeyRemoved); addedKeys.forEach(this::handleTableLocationKeyAdded); } @@ -132,7 +135,26 @@ default void handleTableLocationKeysUpdate( * @return A collection of keys for locations available from this provider. */ @NotNull - Collection getTableLocationKeys(); + default Collection getTableLocationKeys() { + return getTableLocationKeys(key -> true); + } + + /** + *

+ * Get the provider's currently known location keys which pass the supplied filter. The locations specified by the + * keys returned may have null size - that is, they may not "exist" for application purposes. + * {@link #getTableLocation(TableLocationKey)} is guaranteed to succeed for all results. + *

+ * + *

+ * This call also adds a management reference to the TLK from the provide {@link LivenessManager} + *

+ * + * @param filter A filter to apply to the location keys. + * @return A collection of keys for locations available from this provider. + */ + @NotNull + Collection getTableLocationKeys(final Predicate filter); /** * Check if this provider knows the supplied location key. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java new file mode 100644 index 00000000000..4d127d0ba06 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java @@ -0,0 +1,90 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.locations; + +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; +import org.jetbrains.annotations.NotNull; + +import java.util.Set; +import java.util.function.Consumer; + +/** + * Sub-interface of {@link TableLocationKey} to mark immutable implementations. + */ +public class TrackedTableLocationKey extends ReferenceCountedLivenessNode implements ImmutableTableLocationKey { + + public static TrackedTableLocationKey[] ZERO_LENGTH_TRACKED_TABLE_LOCATION_KEY_ARRAY = + new TrackedTableLocationKey[0]; + + final ImmutableTableLocationKey locationKey; + final Consumer zeroCountConsumer; + + private TableLocation tableLocation; + + public TrackedTableLocationKey( + final ImmutableTableLocationKey locationKey, + final Consumer zeroCountConsumer) { + super(false); + + this.locationKey = locationKey; + this.zeroCountConsumer = zeroCountConsumer; + + tableLocation = null; + } + + public ImmutableTableLocationKey getKey() { + return locationKey; + } + + /** + * This {@link TrackedTableLocationKey} should manage the given {@link TableLocation} and store a reference to it. + * + * @param tableLocation The {@link TableLocation} to manage. + */ + public void manageTableLocation(TableLocation tableLocation) { + Assert.eqNull(this.tableLocation, "this.tableLocation"); + this.tableLocation = tableLocation; + manage(tableLocation); + } + + /** + * Unmanage the {@link TableLocation} and the release the reference. + */ + public void unmanageTableLocation() { + Assert.neqNull(this.tableLocation, "this.tableLocation"); + unmanage(tableLocation); + tableLocation = null; + } + + @Override + protected void destroy() { + super.destroy(); + if (tableLocation != null) { + unmanageTableLocation(); + } + zeroCountConsumer.accept(this); + } + + @Override + public > PARTITION_VALUE_TYPE getPartitionValue( + @NotNull String partitionKey) { + return locationKey.getPartitionValue(partitionKey); + } + + @Override + public Set getPartitionKeys() { + return locationKey.getPartitionKeys(); + } + + @Override + public int compareTo(@NotNull TableLocationKey o) { + return locationKey.compareTo(o); + } + + @Override + public boolean equals(Object o) { + return locationKey.equals(o); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index d945e7c12fc..efd8da0526e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java @@ -4,6 +4,7 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.base.verify.Require; +import io.deephaven.engine.liveness.*; import io.deephaven.engine.table.BasicDataIndex; import io.deephaven.engine.table.impl.util.FieldUtils; import io.deephaven.engine.util.string.StringUtils; @@ -12,7 +13,6 @@ import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; import io.deephaven.util.annotations.InternalUseOnly; -import io.deephaven.util.referencecounting.ReferenceCounted; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -27,7 +27,7 @@ */ public abstract class AbstractTableLocation extends SubscriptionAggregator - implements TableLocation { + implements TableLocation, DelegatingLivenessNode { private final ImmutableTableKey tableKey; private final ImmutableTableLocationKey tableLocationKey; @@ -36,7 +36,7 @@ public abstract class AbstractTableLocation private final KeyedObjectHashMap columnLocations = new KeyedObjectHashMap<>(StringUtils.charSequenceKey()); - private final ReferenceCounted referenceCounted; + private final ReferenceCountedLivenessNode livenessNode; @SuppressWarnings("rawtypes") private static final AtomicReferenceFieldUpdater CACHED_DATA_INDEXES_UPDATER = @@ -62,14 +62,12 @@ protected AbstractTableLocation(@NotNull final TableKey tableKey, this.tableKey = Require.neqNull(tableKey, "tableKey").makeImmutable(); this.tableLocationKey = Require.neqNull(tableLocationKey, "tableLocationKey").makeImmutable(); - referenceCounted = new ReferenceCounted() { + livenessNode = new ReferenceCountedLivenessNode(false) { @Override - protected void onReferenceCountAtZero() { - // Call the location's onReferenceCountAtZero method - AbstractTableLocation.this.onReferenceCountAtZero(); + protected void destroy() { + AbstractTableLocation.this.destroy(); } }; - } @Override @@ -77,6 +75,10 @@ public final String toString() { return toStringHelper(); } + @Override + public LivenessNode asLivenessNode() { + return livenessNode; + } // ------------------------------------------------------------------------------------------------------------------ // TableLocationState implementation @@ -247,27 +249,9 @@ public final BasicDataIndex getDataIndex(@NotNull final String... columns) { // ------------------------------------------------------------------------------------------------------------------ /** - * Increment the reference count by one. - * - * @throws IllegalStateException If the reference count was not successfully incremented - */ - public final void incrementReferenceCount() { - referenceCounted.incrementReferenceCount(); - } - - /** - * Decrement the reference count by one, when the reference count reaches zero this location will be cleared. - * - * @throws IllegalStateException If the reference count was not successfully decremented - */ - public void decrementReferenceCount() { - referenceCounted.decrementReferenceCount(); - } - - /** - * The reference count has reached zero, we can clear this location and release any resources. + * The reference count has reached zero or we are being GC'd, we can clear this location and release any resources. */ - private void onReferenceCountAtZero() { + private void destroy() { handleUpdate(null, System.currentTimeMillis()); clearColumnLocations(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index ce0bf61db09..117c61a8eab 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -3,14 +3,21 @@ // package io.deephaven.engine.table.impl.locations.impl; +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.liveness.DelegatingLivenessNode; +import io.deephaven.engine.liveness.LivenessNode; +import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.hash.KeyedObjectHashMap; +import io.deephaven.hash.KeyedObjectHashSet; import io.deephaven.hash.KeyedObjectKey; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.*; +import java.util.function.Predicate; +import java.util.stream.Collectors; /** * Partial {@link TableLocationProvider} implementation for standalone use or as part of a {@link TableDataService}. @@ -23,7 +30,7 @@ */ public abstract class AbstractTableLocationProvider extends SubscriptionAggregator - implements TableLocationProvider { + implements TableLocationProvider, DelegatingLivenessNode { private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet(); @@ -70,6 +77,7 @@ synchronized void removeLocationKey(ImmutableTableLocationKey locationKey) { } private final ImmutableTableKey tableKey; + private final ReferenceCountedLivenessNode livenessNode; // Open transactions that are being accumulated private final Map transactions = Collections.synchronizedMap(new HashMap<>()); @@ -82,13 +90,19 @@ synchronized void removeLocationKey(ImmutableTableLocationKey locationKey) { * {@link #getTableLocationIfPresent(TableLocationKey)} *
  • The {@link TableLocationKey} is a {@link TableLocation}
  • * + * + * These values will not be cleared until all references to the {@link TableLocation} have been released by its + * managers (i.e. {@link TableLocationSubscriptionBuffer subscriptions} and + * {@link io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSourceManager column source managers}). */ private final KeyedObjectHashMap tableLocations = new KeyedObjectHashMap<>(LocationKeyDefinition.INSTANCE); - @SuppressWarnings("unchecked") - private final Collection unmodifiableTableLocationKeys = - (Collection) (Collection) Collections - .unmodifiableCollection(tableLocations.keySet()); + + /** + * The set of active location keys that will be returned to new subscribers. + */ + private final KeyedObjectHashSet liveLocationKeys = + new KeyedObjectHashSet<>(ImmutableTableLocationKeyDefinition.INSTANCE); private volatile boolean initialized; @@ -105,6 +119,13 @@ protected AbstractTableLocationProvider(@NotNull final TableKey tableKey, final super(supportsSubscriptions); this.tableKey = tableKey.makeImmutable(); this.partitionKeys = null; + + livenessNode = new ReferenceCountedLivenessNode(false) { + @Override + protected void destroy() { + AbstractTableLocationProvider.this.destroy(); + } + }; } /** @@ -125,13 +146,21 @@ public final ImmutableTableKey getKey() { return tableKey; } + @Override + public LivenessNode asLivenessNode() { + return livenessNode; + } + // ------------------------------------------------------------------------------------------------------------------ // TableLocationProvider/SubscriptionAggregator implementation // ------------------------------------------------------------------------------------------------------------------ @Override protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) { - listener.handleTableLocationKeysUpdate(unmodifiableTableLocationKeys, List.of()); + // Lock the live set and deliver a copy to the listener + synchronized (liveLocationKeys) { + listener.handleTableLocationKeysUpdate(liveLocationKeys, List.of()); + } } /** @@ -161,42 +190,41 @@ protected void endTransaction(@NotNull final Object token) { throw new IllegalStateException("No transaction with token " + token + " is currently open."); } - // Return early if there are no changes to process. + // Return early if there are no changes to process if (transaction.locationsAdded == EMPTY_TABLE_LOCATION_KEYS && transaction.locationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { return; } - final Collection addedKeys = + final Collection addedKeys = new ArrayList<>(transaction.locationsAdded.size()); - final Collection removedKeys = + final Collection removedKeys = new ArrayList<>(transaction.locationsRemoved.size()); synchronized (tableLocations) { for (ImmutableTableLocationKey locationKey : transaction.locationsRemoved) { - final Object removedLocation = tableLocations.remove(locationKey); - if (removedLocation != null) { - if (removedLocation instanceof AbstractTableLocation) { - ((AbstractTableLocation) removedLocation).decrementReferenceCount(); - } - removedKeys.add(locationKey); - } + // Remove this from the live set. + final TrackedTableLocationKey trackedKey = liveLocationKeys.get(locationKey); + Assert.neqNull(trackedKey, "trackedKey"); + liveLocationKeys.remove(trackedKey); + // Pass this removed key to the subscribers + removedKeys.add(trackedKey); } for (ImmutableTableLocationKey locationKey : transaction.locationsAdded) { locationCreatedRecorder = false; - tableLocations.putIfAbsent(locationKey, this::observeInsert); + final Object result = tableLocations.putIfAbsent(locationKey, this::observeInsert); visitLocationKey(locationKey); if (locationCreatedRecorder) { verifyPartitionKeys(locationKey); - addedKeys.add(locationKey); + addedKeys.add((TrackedTableLocationKey) result); } } } if (subscriptions != null && (!addedKeys.isEmpty() || !removedKeys.isEmpty())) { synchronized (subscriptions) { - // Push the notifications to the subscribers. + // Push the notifications to the subscribers if (subscriptions.deliverNotification( Listener::handleTableLocationKeysUpdate, addedKeys, @@ -204,6 +232,9 @@ protected void endTransaction(@NotNull final Object token) { true)) { onEmpty(); } + // Release the keys that were removed only after we have delivered the notifications and the + // subscribers have had a chance to process them + removedKeys.forEach(this::unmanage); } } } @@ -241,12 +272,11 @@ protected final void handleTableLocationKeyAdded( } if (!supportsSubscriptions()) { - tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable); + tableLocations.putIfAbsent(locationKey, this::observeInsert); visitLocationKey(locationKey); return; } - // If we're not in a transaction, we should push this key immediately. synchronized (subscriptions) { // Since we're holding the lock on subscriptions, the following code is overly complicated - we could @@ -260,7 +290,7 @@ protected final void handleTableLocationKeyAdded( verifyPartitionKeys(locationKey); if (subscriptions.deliverNotification( Listener::handleTableLocationKeyAdded, - toKeyImmutable(result), + (TrackedTableLocationKey) result, true)) { onEmpty(); } @@ -268,6 +298,53 @@ protected final void handleTableLocationKeyAdded( } } + /** + * Handle a removal, optionally as part of a transaction. Notify subscribers that {@code locationKey} was removed if + * necessary. See {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics. + * + * @param locationKey the TableLocation that was removed + * @param transactionToken The token identifying the transaction (or null if not part of a transaction) + */ + protected void handleTableLocationKeyRemoved( + @NotNull final TableLocationKey locationKey, + @Nullable final Object transactionToken) { + if (transactionToken != null) { + final Transaction transaction = transactions.get(transactionToken); + if (transaction == null) { + throw new IllegalStateException( + "No transaction with token " + transactionToken + " is currently open."); + } + transaction.removeLocationKey(toKeyImmutable(locationKey)); + return; + } + + if (!supportsSubscriptions()) { + // Remove this from the live set and un-manage it. + final TrackedTableLocationKey trackedKey = liveLocationKeys.get(locationKey); + liveLocationKeys.remove(trackedKey); + unmanage(trackedKey); + return; + } + + // If we're not in a transaction, we should push this key immediately. + synchronized (subscriptions) { + final Object removedLocation = tableLocations.remove(locationKey); + if (removedLocation != null) { + final TrackedTableLocationKey trackedKey = liveLocationKeys.get(locationKey); + Assert.neqNull(trackedKey, "trackedKey"); + // Remove this from the live set and un-manage it. + liveLocationKeys.remove(trackedKey); + if (subscriptions.deliverNotification( + Listener::handleTableLocationKeyRemoved, + trackedKey, + true)) { + onEmpty(); + } + unmanage(trackedKey); + } + } + } + /** * Called after a table location has been visited by * {@link #handleTableLocationKeyAdded(TableLocationKey, Object)}, but before notifications have been delivered to @@ -282,7 +359,13 @@ protected void visitLocationKey(@NotNull final TableLocationKey locationKey) {} private Object observeInsert(@NotNull final TableLocationKey locationKey) { // NB: This must only be called while the lock on subscriptions is held. locationCreatedRecorder = true; - return locationKey.makeImmutable(); + + final TrackedTableLocationKey trackedKey = toTrackedKey(locationKey); + manage(trackedKey); + + // Add this to the live set. + liveLocationKeys.add(trackedKey); + return trackedKey; } /** @@ -331,11 +414,13 @@ protected void doInitialization() { @Override @NotNull - public final Collection getTableLocationKeys() { - synchronized (tableLocations) { - // Note that we never store keys that aren't immutable. - // noinspection unchecked,rawtypes - return new ArrayList<>((Set) (Set) tableLocations.keySet()); + public final Collection getTableLocationKeys( + final Predicate filter) { + // Lock the live set and deliver a copy to the listener after filtering. + synchronized (liveLocationKeys) { + return liveLocationKeys.stream() + .filter(tk -> filter.test(tk.getKey())) + .collect(Collectors.toList()); } } @@ -360,11 +445,12 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t synchronized (immutableKey) { current = tableLocations.get(immutableKey); if (immutableKey == current) { - // Note, this may contend for the lock on tableLocations - tableLocations.add(current = makeTableLocation(immutableKey)); - if (current instanceof AbstractTableLocation) { - ((AbstractTableLocation) current).incrementReferenceCount(); - } + // Make a new location, have the tracked key manage it, then replace the key with the + // new location in the map and return it. Note, this may contend for the lock on tableLocations + final TrackedTableLocationKey trackedKey = (TrackedTableLocationKey) current; + final TableLocation newLocation = makeTableLocation(immutableKey); + trackedKey.manageTableLocation(newLocation); + tableLocations.add(current = newLocation); } } } @@ -374,7 +460,7 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t /** * Remove a {@link TableLocationKey} and its corresponding {@link TableLocation} (if it was created). All * subscribers to this TableLocationProvider will be - * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(ImmutableTableLocationKey) notified}. If the + * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(TrackedTableLocationKey) notified}. If the * TableLocation was created, all of its subscribers will additionally be * {@link TableLocation.Listener#handleUpdate() notified} that it no longer exists. This TableLocationProvider will * continue to update other locations and will no longer provide or request information about the removed location. @@ -399,61 +485,6 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey) handleTableLocationKeyRemoved(locationKey, null); } - /** - * Handle a removal. Notify subscribers that {@code locationKey} was removed if necessary. See - * {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics. - * - * @param locationKey the TableLocation that was removed - */ - protected final void handleTableLocationKeyRemoved(@NotNull final TableLocationKey locationKey) { - handleTableLocationKeyRemoved(locationKey, null); - } - - /** - * Handle a removal, optionally as part of a transaction. Notify subscribers that {@code locationKey} was removed if - * necessary. See {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics. - * - * @param locationKey the TableLocation that was removed - * @param transactionToken The token identifying the transaction (or null if not part of a transaction) - */ - protected void handleTableLocationKeyRemoved( - @NotNull final TableLocationKey locationKey, - @Nullable final Object transactionToken) { - if (transactionToken != null) { - final Transaction transaction = transactions.get(transactionToken); - if (transaction == null) { - throw new IllegalStateException( - "No transaction with token " + transactionToken + " is currently open."); - } - transaction.removeLocationKey(toKeyImmutable(locationKey)); - return; - } - - if (!supportsSubscriptions()) { - final Object removedLocation = tableLocations.remove(locationKey); - if (removedLocation instanceof AbstractTableLocation) { - ((AbstractTableLocation) removedLocation).decrementReferenceCount(); - } - return; - } - - // If we're not in a transaction, we should push this key immediately. - synchronized (subscriptions) { - final Object removedLocation = tableLocations.remove(locationKey); - if (removedLocation != null) { - if (removedLocation instanceof AbstractTableLocation) { - ((AbstractTableLocation) removedLocation).decrementReferenceCount(); - } - if (subscriptions.deliverNotification( - Listener::handleTableLocationKeyRemoved, - locationKey.makeImmutable(), - true)) { - onEmpty(); - } - } - } - } - private void verifyPartitionKeys(@NotNull final TableLocationKey locationKey) { if (partitionKeys == null) { partitionKeys = new ArrayList<>(locationKey.getPartitionKeys()); @@ -479,6 +510,23 @@ public TableLocationKey getKey(@NotNull final Object keyOrLocation) { } } + /** + * Key definition for {@link TrackedTableLocationKey} lookup by {@link ImmutableTableLocationKey}. + */ + private static final class ImmutableTableLocationKeyDefinition + extends KeyedObjectKey.Basic { + + private static final KeyedObjectKey INSTANCE = + new ImmutableTableLocationKeyDefinition(); + + private ImmutableTableLocationKeyDefinition() {} + + @Override + public ImmutableTableLocationKey getKey(TrackedTableLocationKey trackedTableLocationKey) { + return trackedTableLocationKey.getKey(); + } + } + private static TableLocationKey toKey(@NotNull final Object keyOrLocation) { if (keyOrLocation instanceof TableLocation) { return ((TableLocation) keyOrLocation).getKey(); @@ -494,6 +542,11 @@ private static ImmutableTableLocationKey toKeyImmutable(@NotNull final Object ke return (ImmutableTableLocationKey) toKey(keyOrLocation); } + private TrackedTableLocationKey toTrackedKey(@NotNull final TableLocationKey locationKey) { + return new TrackedTableLocationKey(locationKey.makeImmutable(), this::releaseLocationKey); + } + + private static boolean equals(Collection c1, Collection c2) { final Iterator i2 = c2.iterator(); for (T t1 : c1) { @@ -506,4 +559,19 @@ private static boolean equals(Collection c1, Collection c2) { } return !i2.hasNext(); } + + /** + * Called when every reference to the {@link TrackedTableLocationKey key} has been released. + * + * @param locationKey the key to release + */ + private void releaseLocationKey(@NotNull final TrackedTableLocationKey locationKey) { + // We can now remove the key from the tableLocations map + tableLocations.remove(locationKey.getKey()); + } + + private void destroy() { + // TODO: release all the TTLK references. Or does that happen automatically? + throw new UnsupportedOperationException("Not yet implemented"); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index a9157da4b0f..115b33b852c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -13,6 +13,7 @@ import org.jetbrains.annotations.Nullable; import java.util.*; +import java.util.function.Predicate; import java.util.stream.Collectors; /** @@ -141,13 +142,13 @@ public TableLocationProvider ensureInitialized() { } @Override - @NotNull - public Collection getTableLocationKeys() { - final Set locationKeys = new KeyedObjectHashSet<>(KeyKeyDefinition.INSTANCE); + public @NotNull Collection getTableLocationKeys(Predicate filter) { + final Set locationKeys = new KeyedObjectHashSet<>(KeyKeyDefinition.INSTANCE); try (final SafeCloseable ignored = CompositeTableDataServiceConsistencyMonitor.INSTANCE.start()) { inputProviders.stream() .map(TableLocationProvider::getTableLocationKeys) .flatMap(Collection::stream) + .filter(tlk -> filter.test(tlk.getKey())) .filter(x -> !locationKeys.add(x)) .findFirst() .ifPresent(duplicateLocationKey -> { @@ -225,16 +226,16 @@ public String describe() { // ------------------------------------------------------------------------------------------------------------------ private static final class KeyKeyDefinition - extends KeyedObjectKey.Basic { + extends KeyedObjectKey.Basic { - private static final KeyedObjectKey INSTANCE = + private static final KeyedObjectKey INSTANCE = new KeyKeyDefinition(); private KeyKeyDefinition() {} @Override - public ImmutableTableLocationKey getKey(@NotNull final ImmutableTableLocationKey tableLocationKey) { - return tableLocationKey; + public ImmutableTableLocationKey getKey(@NotNull final TrackedTableLocationKey tableLocationKey) { + return tableLocationKey.getKey(); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 7aa365c1ce1..9a6acd26200 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -12,6 +12,7 @@ import java.util.Collection; import java.util.Map; import java.util.WeakHashMap; +import java.util.function.Predicate; import java.util.stream.Collectors; /** @@ -124,8 +125,10 @@ public TableLocationProvider ensureInitialized() { } @Override - public @NotNull Collection getTableLocationKeys() { - return inputProvider.getTableLocationKeys().stream().filter(locationKeyFilter::accept) + public @NotNull Collection getTableLocationKeys(Predicate filter) { + return inputProvider.getTableLocationKeys().stream() + .filter(locationKeyFilter::accept) + .filter(filter) .collect(Collectors.toList()); } @@ -175,7 +178,7 @@ public void endTransaction(@NotNull final Object token) { } @Override - public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyAdded(@NotNull final TrackedTableLocationKey tableLocationKey) { final TableLocationProvider.Listener outputListener = getWrapped(); // We can't try to clean up null listeners here, the underlying implementation may not allow concurrent // unsubscribe operations. @@ -185,7 +188,7 @@ public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey } @Override - public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyRemoved(@NotNull final TrackedTableLocationKey tableLocationKey) { final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { outputListener.handleTableLocationKeyRemoved(tableLocationKey); @@ -194,8 +197,8 @@ public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationK @Override public void handleTableLocationKeysUpdate( - @NotNull Collection addedKeys, - @NotNull Collection removedKeys) { + @NotNull Collection addedKeys, + @NotNull Collection removedKeys) { final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null) { outputListener.handleTableLocationKeysUpdate(addedKeys, removedKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java index a94abd87f5f..cf27efb8392 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java @@ -9,6 +9,8 @@ import java.util.Collection; import java.util.Collections; +import java.util.List; +import java.util.function.Predicate; /** * A {@link TableLocationProvider} that provides access to exactly one, previously-known {@link TableLocation}. @@ -59,10 +61,12 @@ public TableLocationProvider ensureInitialized() { return this; } - @NotNull @Override - public Collection getTableLocationKeys() { - return Collections.singleton(tableLocation.getKey()); + public @NotNull Collection getTableLocationKeys(Predicate filter) { + // TODO: should TableLocation#getKey() be a TrackedTableLocationKey? This is getting complicated. + return filter.test(tableLocation.getKey()) + ? Collections.singleton(tableLocation.getKey()) + : List.of(); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index 5ac8d7aee1b..0778e1ac245 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -4,9 +4,11 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.base.verify.Require; -import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; +import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; +import io.deephaven.engine.table.impl.locations.TrackedTableLocationKey; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocationProvider; +import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -15,9 +17,10 @@ /** * Intermediates between push-based subscription to a TableLocationProvider and polling on update source refresh. */ -public class TableLocationSubscriptionBuffer implements TableLocationProvider.Listener { +public class TableLocationSubscriptionBuffer extends ReferenceCountedLivenessNode + implements TableLocationProvider.Listener { - private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet(); + private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet(); private final TableLocationProvider tableLocationProvider; @@ -26,32 +29,39 @@ public class TableLocationSubscriptionBuffer implements TableLocationProvider.Li private final Object updateLock = new Object(); // These sets represent adds and removes from completed transactions. - private Set pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; - private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + private Set pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; + private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; private TableDataException pendingException = null; public TableLocationSubscriptionBuffer(@NotNull final TableLocationProvider tableLocationProvider) { + super(false); this.tableLocationProvider = Require.neqNull(tableLocationProvider, "tableLocationProvider"); } - public static final class LocationUpdate { - private final Collection pendingAddedLocationKeys; - private final Collection pendingRemovedLocations; + public final class LocationUpdate implements SafeCloseable { + private final Collection pendingAddedLocationKeys; + private final Collection pendingRemovedLocations; - public LocationUpdate(@NotNull final Collection pendingAddedLocationKeys, - @NotNull final Collection pendingRemovedLocations) { + public LocationUpdate(@NotNull final Collection pendingAddedLocationKeys, + @NotNull final Collection pendingRemovedLocations) { this.pendingAddedLocationKeys = pendingAddedLocationKeys; this.pendingRemovedLocations = pendingRemovedLocations; } - public Collection getPendingAddedLocationKeys() { + public Collection getPendingAddedLocationKeys() { return pendingAddedLocationKeys; } - public Collection getPendingRemovedLocationKeys() { + public Collection getPendingRemovedLocationKeys() { return pendingRemovedLocations; } + + @Override + public void close() { + pendingAddedLocationKeys.forEach(TableLocationSubscriptionBuffer.this::unmanage); + pendingRemovedLocations.forEach(TableLocationSubscriptionBuffer.this::unmanage); + } } /** @@ -75,8 +85,8 @@ public synchronized LocationUpdate processPending() { } subscribed = true; } - final Collection resultLocationKeys; - final Collection resultLocationsRemoved; + final Collection resultLocationKeys; + final Collection resultLocationsRemoved; final TableDataException resultException; synchronized (updateLock) { resultLocationKeys = pendingLocationsAdded; @@ -105,6 +115,8 @@ public synchronized void reset() { subscribed = false; } synchronized (updateLock) { + pendingLocationsAdded.forEach(this::unmanage); + pendingLocationsRemoved.forEach(this::unmanage); pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; pendingException = null; @@ -126,7 +138,7 @@ public void endTransaction(@NotNull final Object token) { } @Override - public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyAdded(@NotNull final TrackedTableLocationKey tableLocationKey) { synchronized (updateLock) { // Need to verify that we don't have stacked adds (without intervening removes). if (pendingLocationsAdded.contains(tableLocationKey)) { @@ -136,12 +148,13 @@ public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { pendingLocationsAdded = new HashSet<>(); } + manage(tableLocationKey); pendingLocationsAdded.add(tableLocationKey); } } @Override - public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyRemoved(@NotNull final TrackedTableLocationKey tableLocationKey) { synchronized (updateLock) { // If we have a pending add, it is being cancelled by this remove. if (pendingLocationsAdded.remove(tableLocationKey)) { @@ -155,17 +168,18 @@ public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationK if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { pendingLocationsRemoved = new HashSet<>(); } + manage(tableLocationKey); pendingLocationsRemoved.add(tableLocationKey); } } @Override public void handleTableLocationKeysUpdate( - @Nullable Collection addedKeys, - @Nullable Collection removedKeys) { + @Nullable Collection addedKeys, + @Nullable Collection removedKeys) { synchronized (updateLock) { if (removedKeys != null) { - for (final ImmutableTableLocationKey removedTableLocationKey : removedKeys) { + for (final TrackedTableLocationKey removedTableLocationKey : removedKeys) { // If we have a pending add, it is being cancelled by this remove. if (pendingLocationsAdded.remove(removedTableLocationKey)) { continue; @@ -178,11 +192,12 @@ public void handleTableLocationKeysUpdate( if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { pendingLocationsRemoved = new HashSet<>(); } + manage(removedTableLocationKey); pendingLocationsRemoved.add(removedTableLocationKey); } } if (addedKeys != null) { - for (final ImmutableTableLocationKey addedTableLocationKey : addedKeys) { + for (final TrackedTableLocationKey addedTableLocationKey : addedKeys) { // Need to verify that we don't have stacked adds. if (pendingLocationsAdded.contains(addedTableLocationKey)) { throw new IllegalStateException("TableLocationKey " + addedTableLocationKey @@ -191,6 +206,7 @@ public void handleTableLocationKeysUpdate( if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) { pendingLocationsAdded = new HashSet<>(); } + manage(addedTableLocationKey); pendingLocationsAdded.add(addedTableLocationKey); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 7eba2f7332f..10b8244f1d0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -5,16 +5,13 @@ import io.deephaven.base.verify.Assert; import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.liveness.LivenessArtifact; import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.indexer.DataIndexer; -import io.deephaven.engine.table.impl.locations.ColumnLocation; -import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; -import io.deephaven.engine.table.impl.locations.TableDataException; -import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; import io.deephaven.engine.table.impl.locations.impl.TableLocationUpdateSubscriptionBuffer; import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; @@ -41,7 +38,7 @@ /** * Manage column sources made up of regions in their own row key address space. */ -public class RegionedColumnSourceManager extends LivenessArtifact implements ColumnSourceManager { +public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode implements ColumnSourceManager { private static final Logger log = LoggerFactory.getLogger(RegionedColumnSourceManager.class); @@ -132,6 +129,11 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col */ private List invalidatedLocations = new ArrayList<>(); + /** + * List of tracked location keys to release at the end of the cycle. + */ + private List releasedLocations = new ArrayList<>(); + /** * Will invalidate the locations at the end of the cycle after all downstream updates are complete. */ @@ -202,6 +204,8 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col synchronized (this) { invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate); invalidatedLocations.clear(); + releasedLocations.forEach(this::unmanage); + releasedLocations.clear(); } }); } @@ -209,11 +213,7 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col @Override public void destroy() { super.destroy(); - includedTableLocations.keySet().forEach(location -> { - if (location instanceof AbstractTableLocation) { - ((AbstractTableLocation) location).decrementReferenceCount(); - } - }); + // TODO: do I need to explicitly release the managed locations or does it happen automatically? } @Override @@ -246,7 +246,7 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation) } @Override - public synchronized boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locationKey) { + public synchronized boolean removeLocationKey(final @NotNull TrackedTableLocationKey locationKey) { final IncludedTableLocationEntry includedLocation = includedTableLocations.remove(locationKey); final EmptyTableLocationEntry emptyLocation = emptyTableLocations.remove(locationKey); @@ -254,9 +254,16 @@ public synchronized boolean removeLocationKey(@NotNull final ImmutableTableLocat if (log.isDebugEnabled()) { log.debug().append("EMPTY_LOCATION_REMOVED:").append(locationKey.toString()).endl(); } + if (emptyLocation.location instanceof AbstractTableLocation) { + releasedLocations.add((AbstractTableLocation) emptyLocation.location); + invalidateCommitter.maybeActivate(); + } } else if (includedLocation != null) { orderedIncludedTableLocations.remove(includedLocation); removedTableLocations.add(includedLocation); + if (includedLocation.location instanceof AbstractTableLocation) { + releasedLocations.add((AbstractTableLocation) includedLocation.location); + } invalidateCommitter.maybeActivate(); return true; } @@ -411,9 +418,6 @@ private TableUpdateImpl update(final boolean initializing) { includedTableLocations.add(entry); orderedIncludedTableLocations.add(entry); entry.processInitial(addedRowSetBuilder, entryToInclude.initialRowSet); - if (entry.location instanceof AbstractTableLocation) { - ((AbstractTableLocation) entry.location).incrementReferenceCount(); - } // We have a new location, add the row set to the table and mark the row as added. // @formatter:off @@ -673,9 +677,6 @@ private boolean pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) { private void invalidate() { columnLocationStates.forEach(cls -> cls.source.invalidateRegion(regionIndex)); - if (location instanceof AbstractTableLocation) { - ((AbstractTableLocation) location).decrementReferenceCount(); - } } @Override From dd122408ba586958fb2c2cc9842f21111804e2cd Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Thu, 12 Sep 2024 17:42:18 -0700 Subject: [PATCH 30/56] Compiles now, still many problems --- .../table/impl/PartitionAwareSourceTable.java | 23 ++++++---- .../table/impl/SourcePartitionedTable.java | 11 +++-- .../engine/table/impl/SourceTable.java | 13 ++++-- .../impl/locations/TableLocationProvider.java | 27 +++-------- .../locations/TrackedTableLocationKey.java | 25 +--------- .../impl/AbstractTableLocationProvider.java | 29 ++++-------- .../impl/CompositeTableDataService.java | 46 ++++++++++--------- .../impl/FilteredTableDataService.java | 15 +++--- .../impl/SingleTableLocationProvider.java | 19 ++++---- .../impl/TableLocationSubscriptionBuffer.java | 4 +- .../regioned/RegionedColumnSourceManager.java | 5 ++ ...ebergAutoRefreshTableLocationProvider.java | 4 +- ...ergManualRefreshTableLocationProvider.java | 4 +- 13 files changed, 106 insertions(+), 119 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java index bdae90034e0..44132aa1a49 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java @@ -206,6 +206,7 @@ protected final Table redefine(TableDefinition newDefinitionExternal, TableDefin } private static final String LOCATION_KEY_COLUMN_NAME = "__PartitionAwareSourceTable_TableLocationKey__"; + private static final String TRACKED_KEY_COLUMN_NAME = "__PartitionAwareSourceTable_TrackedTableLocationKey__"; private static ColumnSource makePartitionSource(@NotNull final ColumnDefinition columnDefinition, @NotNull final Collection locationKeys) { @@ -228,17 +229,25 @@ protected final Collection filterLocationKeys( return foundLocationKeys; } + final Collection immutableTableLocationKeys = foundLocationKeys.stream() + .map(TrackedTableLocationKey::getKey) + .collect(Collectors.toList()); + // TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table final List partitionTableColumnNames = Stream.concat( partitioningColumnDefinitions.keySet().stream(), - Stream.of(LOCATION_KEY_COLUMN_NAME)).collect(Collectors.toList()); + Stream.of(LOCATION_KEY_COLUMN_NAME, TRACKED_KEY_COLUMN_NAME)).collect(Collectors.toList()); final List> partitionTableColumnSources = new ArrayList<>(partitioningColumnDefinitions.size() + 1); for (final ColumnDefinition columnDefinition : partitioningColumnDefinitions.values()) { - partitionTableColumnSources.add(makePartitionSource(columnDefinition, foundLocationKeys)); + partitionTableColumnSources.add(makePartitionSource(columnDefinition, immutableTableLocationKeys)); } - partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource(foundLocationKeys, + // Add the tracked and immutable keys to the table + partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource(immutableTableLocationKeys, ImmutableTableLocationKey.class, null)); + partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource(foundLocationKeys, + TrackedTableLocationKey.class, null)); + final Table filteredColumnPartitionTable = TableTools .newTable(foundLocationKeys.size(), partitionTableColumnNames, partitionTableColumnSources) .where(Filter.and(partitioningColumnFilters)); @@ -246,11 +255,9 @@ protected final Collection filterLocationKeys( return foundLocationKeys; } - // TODO: Not sure what to do here. Seems like there is a big disconnect between the location keys and the - // tracked location keys. - - final Iterable iterable = - () -> filteredColumnPartitionTable.columnIterator(LOCATION_KEY_COLUMN_NAME); + // Return the filtered tracked location keys + final Iterable iterable = + () -> filteredColumnPartitionTable.columnIterator(TRACKED_KEY_COLUMN_NAME); return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toList()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index 0328f7a60f7..35d080b85da 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -163,9 +163,12 @@ protected void instrumentedRefresh() { processNewLocationsUpdateRoot = null; removedLocationsComitter = null; tableLocationProvider.refresh(); - try (final RowSet added = sortAndAddLocations(tableLocationProvider.getTableLocationKeys().stream() - .filter(locationKeyMatcher) - .map(tableLocationProvider::getTableLocation))) { + + final Collection locations = new ArrayList<>(); + tableLocationProvider.getTableLocationKeys(tlk -> { + locations.add(tableLocationProvider.getTableLocation(tlk.getKey())); + }, locationKeyMatcher); + try (final RowSet added = sortAndAddLocations(locations.stream())) { resultRows.insert(added); } } @@ -247,6 +250,7 @@ private RowSet processAdditions(final TableLocationSubscriptionBuffer.LocationUp */ // TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table locationUpdate.getPendingAddedLocationKeys().stream() + .map(TrackedTableLocationKey::getKey) .filter(locationKeyMatcher) .map(tableLocationProvider::getTableLocation) .map(PendingLocationState::new) @@ -268,6 +272,7 @@ private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpd final Set relevantRemovedLocations = locationUpdate.getPendingRemovedLocationKeys() .stream() + .map(TrackedTableLocationKey::getKey) .filter(locationKeyMatcher) .collect(Collectors.toSet()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index 8d605464798..a6c652a8510 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -20,6 +20,7 @@ import org.apache.commons.lang3.mutable.MutableObject; import org.jetbrains.annotations.NotNull; +import java.util.ArrayList; import java.util.Collection; /** @@ -146,7 +147,9 @@ private void initializeAvailableLocations() { updateSourceRegistrar.addSource(locationChangePoller = new LocationChangePoller(locationBuffer)); } else { locationProvider.refresh(); - maybeAddLocations(locationProvider.getTableLocationKeys()); + final Collection tableLocationKeys = new ArrayList<>(); + locationProvider.getTableLocationKeys(tableLocationKeys::add); + maybeAddLocations(tableLocationKeys); } }); locationsInitialized = true; @@ -161,8 +164,9 @@ private void maybeAddLocations(@NotNull final Collection { // Unconditionally manage all locations added to the column source manager - columnSourceManager.manage(lk); - columnSourceManager.addLocation(locationProvider.getTableLocation(lk)); + final TableLocation tableLocation = locationProvider.getTableLocation(lk.getKey()); + columnSourceManager.manage(tableLocation); + columnSourceManager.addLocation(tableLocation); }); } @@ -214,7 +218,8 @@ private LocationChangePoller(@NotNull final TableLocationSubscriptionBuffer loca @Override protected void instrumentedRefresh() { - try(final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending()) { + try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = + locationBuffer.processPending()) { maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index 6dbac0dcee0..e1f0df18266 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -3,12 +3,12 @@ // package io.deephaven.engine.table.impl.locations; -import io.deephaven.engine.liveness.LivenessManager; import io.deephaven.util.type.NamedImplementation; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.Collection; +import java.util.function.Consumer; import java.util.function.Predicate; /** @@ -131,30 +131,17 @@ default void handleTableLocationKeysUpdate( * Get this provider's currently known location keys. The locations specified by the keys returned may have null * size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is * guaranteed to succeed for all results. - * - * @return A collection of keys for locations available from this provider. */ - @NotNull - default Collection getTableLocationKeys() { - return getTableLocationKeys(key -> true); + default void getTableLocationKeys(Consumer consumer) { + getTableLocationKeys(consumer, key -> true); } /** - *

    - * Get the provider's currently known location keys which pass the supplied filter. The locations specified by the - * keys returned may have null size - that is, they may not "exist" for application purposes. - * {@link #getTableLocation(TableLocationKey)} is guaranteed to succeed for all results. - *

    - * - *

    - * This call also adds a management reference to the TLK from the provide {@link LivenessManager} - *

    - * - * @param filter A filter to apply to the location keys. - * @return A collection of keys for locations available from this provider. + * Get this provider's currently known location keys. The locations specified by the keys returned may have null + * size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is + * guaranteed to succeed for all results. */ - @NotNull - Collection getTableLocationKeys(final Predicate filter); + void getTableLocationKeys(Consumer consumer, Predicate filter); /** * Check if this provider knows the supplied location key. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java index 4d127d0ba06..81b23b06dbd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java @@ -5,15 +5,13 @@ import io.deephaven.base.verify.Assert; import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; -import org.jetbrains.annotations.NotNull; -import java.util.Set; import java.util.function.Consumer; /** * Sub-interface of {@link TableLocationKey} to mark immutable implementations. */ -public class TrackedTableLocationKey extends ReferenceCountedLivenessNode implements ImmutableTableLocationKey { +public class TrackedTableLocationKey extends ReferenceCountedLivenessNode { public static TrackedTableLocationKey[] ZERO_LENGTH_TRACKED_TABLE_LOCATION_KEY_ARRAY = new TrackedTableLocationKey[0]; @@ -66,25 +64,4 @@ protected void destroy() { } zeroCountConsumer.accept(this); } - - @Override - public > PARTITION_VALUE_TYPE getPartitionValue( - @NotNull String partitionKey) { - return locationKey.getPartitionValue(partitionKey); - } - - @Override - public Set getPartitionKeys() { - return locationKey.getPartitionKeys(); - } - - @Override - public int compareTo(@NotNull TableLocationKey o) { - return locationKey.compareTo(o); - } - - @Override - public boolean equals(Object o) { - return locationKey.equals(o); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 117c61a8eab..233ce1e0bf7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -4,8 +4,6 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.base.verify.Assert; -import io.deephaven.engine.liveness.DelegatingLivenessNode; -import io.deephaven.engine.liveness.LivenessNode; import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.*; @@ -16,8 +14,8 @@ import org.jetbrains.annotations.Nullable; import java.util.*; +import java.util.function.Consumer; import java.util.function.Predicate; -import java.util.stream.Collectors; /** * Partial {@link TableLocationProvider} implementation for standalone use or as part of a {@link TableDataService}. @@ -30,7 +28,7 @@ */ public abstract class AbstractTableLocationProvider extends SubscriptionAggregator - implements TableLocationProvider, DelegatingLivenessNode { + implements TableLocationProvider { private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet(); @@ -146,11 +144,6 @@ public final ImmutableTableKey getKey() { return tableKey; } - @Override - public LivenessNode asLivenessNode() { - return livenessNode; - } - // ------------------------------------------------------------------------------------------------------------------ // TableLocationProvider/SubscriptionAggregator implementation // ------------------------------------------------------------------------------------------------------------------ @@ -234,7 +227,7 @@ protected void endTransaction(@NotNull final Object token) { } // Release the keys that were removed only after we have delivered the notifications and the // subscribers have had a chance to process them - removedKeys.forEach(this::unmanage); + removedKeys.forEach(livenessNode::unmanage); } } } @@ -322,7 +315,7 @@ protected void handleTableLocationKeyRemoved( // Remove this from the live set and un-manage it. final TrackedTableLocationKey trackedKey = liveLocationKeys.get(locationKey); liveLocationKeys.remove(trackedKey); - unmanage(trackedKey); + livenessNode.unmanage(trackedKey); return; } @@ -340,7 +333,7 @@ protected void handleTableLocationKeyRemoved( true)) { onEmpty(); } - unmanage(trackedKey); + livenessNode.unmanage(trackedKey); } } } @@ -361,7 +354,7 @@ private Object observeInsert(@NotNull final TableLocationKey locationKey) { locationCreatedRecorder = true; final TrackedTableLocationKey trackedKey = toTrackedKey(locationKey); - manage(trackedKey); + livenessNode.manage(trackedKey); // Add this to the live set. liveLocationKeys.add(trackedKey); @@ -413,14 +406,12 @@ protected void doInitialization() { } @Override - @NotNull - public final Collection getTableLocationKeys( - final Predicate filter) { + public void getTableLocationKeys( + final Consumer consumer, + final Predicate filter) { // Lock the live set and deliver a copy to the listener after filtering. synchronized (liveLocationKeys) { - return liveLocationKeys.stream() - .filter(tk -> filter.test(tk.getKey())) - .collect(Collectors.toList()); + liveLocationKeys.stream().filter(ttlk -> filter.test(ttlk.getKey())).forEach(consumer); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index 115b33b852c..42d0541039a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -13,6 +13,7 @@ import org.jetbrains.annotations.Nullable; import java.util.*; +import java.util.function.Consumer; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -116,7 +117,7 @@ public void subscribe(@NotNull final Listener listener) { p.subscribe(listener); } else { p.refresh(); - p.getTableLocationKeys().forEach(listener::handleTableLocationKeyAdded); + p.getTableLocationKeys(listener::handleTableLocationKeyAdded); } }); } @@ -142,29 +143,30 @@ public TableLocationProvider ensureInitialized() { } @Override - public @NotNull Collection getTableLocationKeys(Predicate filter) { + public void getTableLocationKeys( + final Consumer consumer, + final Predicate filter) { final Set locationKeys = new KeyedObjectHashSet<>(KeyKeyDefinition.INSTANCE); try (final SafeCloseable ignored = CompositeTableDataServiceConsistencyMonitor.INSTANCE.start()) { - inputProviders.stream() - .map(TableLocationProvider::getTableLocationKeys) - .flatMap(Collection::stream) - .filter(tlk -> filter.test(tlk.getKey())) - .filter(x -> !locationKeys.add(x)) - .findFirst() - .ifPresent(duplicateLocationKey -> { - final String overlappingProviders = inputProviders.stream() - .filter(inputProvider -> inputProvider.hasTableLocationKey(duplicateLocationKey)) - .map(TableLocationProvider::getName) - .collect(Collectors.joining(",")); - throw new TableDataException( - "Data Routing Configuration error: TableDataService elements overlap at location " + - duplicateLocationKey + - " in providers " + overlappingProviders + - ". Full TableDataService configuration:\n" + - Formatter - .formatTableDataService(CompositeTableDataService.this.toString())); - }); - return Collections.unmodifiableCollection(locationKeys); + // Add all the location keys from the providers to the set, throw an exception if there are duplicates + inputProviders.forEach(p -> p.getTableLocationKeys(tlk -> { + if (filter.test(tlk.getKey()) && !locationKeys.add(tlk)) { + final String overlappingProviders = inputProviders.stream() + .filter(inputProvider -> inputProvider.hasTableLocationKey(tlk.getKey())) + .map(TableLocationProvider::getName) + .collect(Collectors.joining(",")); + throw new TableDataException( + "Data Routing Configuration error: TableDataService elements overlap at location " + + tlk + + " in providers " + overlappingProviders + + ". Full TableDataService configuration:\n" + + Formatter + .formatTableDataService(CompositeTableDataService.this.toString())); + + } + })); + // Consume all the location keys + locationKeys.forEach(consumer); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 9a6acd26200..fe9158a8488 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -12,8 +12,8 @@ import java.util.Collection; import java.util.Map; import java.util.WeakHashMap; +import java.util.function.Consumer; import java.util.function.Predicate; -import java.util.stream.Collectors; /** * {@link TableDataService} implementation with support to filter the provided {@link TableLocation}s. @@ -125,11 +125,10 @@ public TableLocationProvider ensureInitialized() { } @Override - public @NotNull Collection getTableLocationKeys(Predicate filter) { - return inputProvider.getTableLocationKeys().stream() - .filter(locationKeyFilter::accept) - .filter(filter) - .collect(Collectors.toList()); + public void getTableLocationKeys( + final Consumer consumer, + final Predicate filter) { + inputProvider.getTableLocationKeys(consumer, filter); } @Override @@ -182,7 +181,7 @@ public void handleTableLocationKeyAdded(@NotNull final TrackedTableLocationKey t final TableLocationProvider.Listener outputListener = getWrapped(); // We can't try to clean up null listeners here, the underlying implementation may not allow concurrent // unsubscribe operations. - if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { + if (outputListener != null && locationKeyFilter.accept(tableLocationKey.getKey())) { outputListener.handleTableLocationKeyAdded(tableLocationKey); } } @@ -190,7 +189,7 @@ public void handleTableLocationKeyAdded(@NotNull final TrackedTableLocationKey t @Override public void handleTableLocationKeyRemoved(@NotNull final TrackedTableLocationKey tableLocationKey) { final TableLocationProvider.Listener outputListener = getWrapped(); - if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { + if (outputListener != null && locationKeyFilter.accept(tableLocationKey.getKey())) { outputListener.handleTableLocationKeyRemoved(tableLocationKey); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java index cf27efb8392..2d9229422be 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java @@ -7,9 +7,7 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.util.Collection; -import java.util.Collections; -import java.util.List; +import java.util.function.Consumer; import java.util.function.Predicate; /** @@ -19,6 +17,7 @@ public final class SingleTableLocationProvider implements TableLocationProvider private static final String IMPLEMENTATION_NAME = SingleTableLocationProvider.class.getSimpleName(); + private final TrackedTableLocationKey trackedTableLocationKey; private final TableLocation tableLocation; /** @@ -26,6 +25,9 @@ public final class SingleTableLocationProvider implements TableLocationProvider */ public SingleTableLocationProvider(@NotNull final TableLocation tableLocation) { this.tableLocation = tableLocation; + trackedTableLocationKey = new TrackedTableLocationKey(tableLocation.getKey(), ttlk -> { + // no-op + }); } @Override @@ -62,11 +64,12 @@ public TableLocationProvider ensureInitialized() { } @Override - public @NotNull Collection getTableLocationKeys(Predicate filter) { - // TODO: should TableLocation#getKey() be a TrackedTableLocationKey? This is getting complicated. - return filter.test(tableLocation.getKey()) - ? Collections.singleton(tableLocation.getKey()) - : List.of(); + public void getTableLocationKeys( + final Consumer consumer, + final Predicate filter) { + if (filter.test(trackedTableLocationKey.getKey())) { + consumer.accept(trackedTableLocationKey); + } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index 0778e1ac245..d01f7d034ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -81,7 +81,9 @@ public synchronized LocationUpdate processPending() { // NB: Providers that don't support subscriptions don't tick - this single call to run is // sufficient. tableLocationProvider.refresh(); - handleTableLocationKeysUpdate(tableLocationProvider.getTableLocationKeys(), List.of()); + final Collection tableLocationKeys = new ArrayList<>(); + tableLocationProvider.getTableLocationKeys(tableLocationKeys::add); + handleTableLocationKeysUpdate(tableLocationKeys, List.of()); } subscribed = true; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 10b8244f1d0..a39291e322d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -152,6 +152,11 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im @NotNull final ColumnToCodecMappings codecMappings, @NotNull final List> columnDefinitions) { super(false); + // TODO: this seems to be required since the transition to ReferenceCountedLivenessNode (from LivenessArtifact) + // if (Liveness.REFERENCE_TRACKING_DISABLED) { + // return; + // } + LivenessScopeStack.peek().manage(this); this.isRefreshing = isRefreshing; this.columnDefinitions = columnDefinitions; diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index bafbcdff872..09287e066a1 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -88,7 +88,9 @@ public void update(Snapshot snapshot) { */ private void refreshSnapshot() { beginTransaction(this); - final Set missedKeys = new HashSet<>(getTableLocationKeys()); + final Set missedKeys = new HashSet<>(); + getTableLocationKeys(ttlk -> missedKeys.add(ttlk.getKey())); + locationKeyFinder.findKeys(tableLocationKey -> { missedKeys.remove(tableLocationKey); handleTableLocationKeyAdded(tableLocationKey, this); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index f2e5e8fb7e2..be0bae3a20e 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -96,7 +96,9 @@ public synchronized void update(final Snapshot snapshot) { */ private void refreshSnapshot() { beginTransaction(this); - final Set missedKeys = new HashSet<>(getTableLocationKeys()); + final Set missedKeys = new HashSet<>(); + getTableLocationKeys(ttlk -> missedKeys.add(ttlk.getKey())); + locationKeyFinder.findKeys(tlk -> { missedKeys.remove(tlk); handleTableLocationKeyAdded(tlk, this); From 944dac63adbaef3014c698c9c3181e5a7cf8b874 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 13 Sep 2024 09:50:46 -0700 Subject: [PATCH 31/56] Working through problems. --- .../deephaven/engine/table/impl/SourceTable.java | 1 + .../impl/locations/TrackedTableLocationKey.java | 13 +++++++++++++ .../impl/AbstractTableLocationProvider.java | 16 ++++++++++------ 3 files changed, 24 insertions(+), 6 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index a6c652a8510..b41ccaeba49 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -139,6 +139,7 @@ private void initializeAvailableLocations() { if (isRefreshing()) { final TableLocationSubscriptionBuffer locationBuffer = new TableLocationSubscriptionBuffer(locationProvider); + manage(locationBuffer); try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending()) { maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java index 81b23b06dbd..7515e8f43c5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java @@ -36,6 +36,19 @@ public ImmutableTableLocationKey getKey() { return locationKey; } + @Override + public int hashCode() { + return locationKey.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ImmutableTableLocationKey) { + return locationKey.equals(obj); + } + return (this == obj); + } + /** * This {@link TrackedTableLocationKey} should manage the given {@link TableLocation} and store a reference to it. * diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 233ce1e0bf7..45b337e136c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -4,6 +4,7 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.*; @@ -124,6 +125,7 @@ protected void destroy() { AbstractTableLocationProvider.this.destroy(); } }; + LivenessScopeStack.peek().manage(livenessNode); } /** @@ -431,15 +433,14 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t // The intent is to create a TableLocation exactly once to replace the TableLocationKey placeholder that was // added in handleTableLocationKey. if (!(current instanceof TableLocation)) { - final TableLocationKey immutableKey = (TableLocationKey) current; + final TrackedTableLocationKey trackedKey = (TrackedTableLocationKey) current; // noinspection SynchronizationOnLocalVariableOrMethodParameter - synchronized (immutableKey) { - current = tableLocations.get(immutableKey); - if (immutableKey == current) { + synchronized (trackedKey) { + current = tableLocations.get(trackedKey); + if (trackedKey == current) { // Make a new location, have the tracked key manage it, then replace the key with the // new location in the map and return it. Note, this may contend for the lock on tableLocations - final TrackedTableLocationKey trackedKey = (TrackedTableLocationKey) current; - final TableLocation newLocation = makeTableLocation(immutableKey); + final TableLocation newLocation = makeTableLocation(trackedKey.getKey()); trackedKey.manageTableLocation(newLocation); tableLocations.add(current = newLocation); } @@ -522,6 +523,9 @@ private static TableLocationKey toKey(@NotNull final Object keyOrLocation) { if (keyOrLocation instanceof TableLocation) { return ((TableLocation) keyOrLocation).getKey(); } + if (keyOrLocation instanceof TrackedTableLocationKey) { + return (((TrackedTableLocationKey) keyOrLocation).getKey()); + } if (keyOrLocation instanceof TableLocationKey) { return ((TableLocationKey) keyOrLocation); } From 912b9f2a535d193701b6378a483130e730558d91 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 13 Sep 2024 11:08:24 -0700 Subject: [PATCH 32/56] Cleanup and minor changes --- .../engine/table/impl/PartitionAwareSourceTable.java | 9 +++------ .../java/io/deephaven/engine/table/impl/SourceTable.java | 7 +------ .../locations/impl/AbstractTableLocationProvider.java | 1 + .../impl/locations/impl/SingleTableLocationProvider.java | 2 +- .../sources/regioned/RegionedColumnSourceManager.java | 2 ++ 5 files changed, 8 insertions(+), 13 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java index 44132aa1a49..9d9d6f87526 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java @@ -205,7 +205,6 @@ protected final Table redefine(TableDefinition newDefinitionExternal, TableDefin reference, null, viewColumns, null); } - private static final String LOCATION_KEY_COLUMN_NAME = "__PartitionAwareSourceTable_TableLocationKey__"; private static final String TRACKED_KEY_COLUMN_NAME = "__PartitionAwareSourceTable_TrackedTableLocationKey__"; private static ColumnSource makePartitionSource(@NotNull final ColumnDefinition columnDefinition, @@ -236,15 +235,13 @@ protected final Collection filterLocationKeys( // TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table final List partitionTableColumnNames = Stream.concat( partitioningColumnDefinitions.keySet().stream(), - Stream.of(LOCATION_KEY_COLUMN_NAME, TRACKED_KEY_COLUMN_NAME)).collect(Collectors.toList()); + Stream.of(TRACKED_KEY_COLUMN_NAME)).collect(Collectors.toList()); final List> partitionTableColumnSources = new ArrayList<>(partitioningColumnDefinitions.size() + 1); for (final ColumnDefinition columnDefinition : partitioningColumnDefinitions.values()) { partitionTableColumnSources.add(makePartitionSource(columnDefinition, immutableTableLocationKeys)); } - // Add the tracked and immutable keys to the table - partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource(immutableTableLocationKeys, - ImmutableTableLocationKey.class, null)); + // Add the tracked keys to the table partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource(foundLocationKeys, TrackedTableLocationKey.class, null)); @@ -255,7 +252,7 @@ protected final Collection filterLocationKeys( return foundLocationKeys; } - // Return the filtered tracked location keys + // Return the filtered keys final Iterable iterable = () -> filteredColumnPartitionTable.columnIterator(TRACKED_KEY_COLUMN_NAME); return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toList()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index b41ccaeba49..fa74ccce349 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -163,12 +163,7 @@ private void maybeAddLocations(@NotNull final Collection { - // Unconditionally manage all locations added to the column source manager - final TableLocation tableLocation = locationProvider.getTableLocation(lk.getKey()); - columnSourceManager.manage(tableLocation); - columnSourceManager.addLocation(tableLocation); - }); + .forEach(lk -> columnSourceManager.addLocation(locationProvider.getTableLocation(lk.getKey()))); } private TrackedTableLocationKey[] maybeRemoveLocations( diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 45b337e136c..4dc7f444636 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -125,6 +125,7 @@ protected void destroy() { AbstractTableLocationProvider.this.destroy(); } }; + // TODO: understand why this seems to be needed LivenessScopeStack.peek().manage(livenessNode); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java index 2d9229422be..e7c1bde1284 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java @@ -26,7 +26,7 @@ public final class SingleTableLocationProvider implements TableLocationProvider public SingleTableLocationProvider(@NotNull final TableLocation tableLocation) { this.tableLocation = tableLocation; trackedTableLocationKey = new TrackedTableLocationKey(tableLocation.getKey(), ttlk -> { - // no-op + // TODO: I don't think we need to do anything here, but need to think more about it }); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index a39291e322d..b22eeff9786 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -230,6 +230,8 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation) if (log.isDebugEnabled()) { log.debug().append("LOCATION_ADDED:").append(tableLocation.toString()).endl(); } + // Hold on to this table location. + manage(tableLocation); emptyTableLocations.add(new EmptyTableLocationEntry(tableLocation)); } else { // Duplicate location - not allowed From e2b6fd0178a7aa7f609327821c921f8a52c50fec Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 17 Sep 2024 17:36:39 -0700 Subject: [PATCH 33/56] Refactored ATLP --- .../table/impl/ColumnSourceManager.java | 9 +- .../table/impl/PartitionAwareSourceTable.java | 17 +- .../table/impl/SourcePartitionedTable.java | 7 +- .../engine/table/impl/SourceTable.java | 34 ++- .../table/impl/locations/TableLocation.java | 4 +- .../impl/locations/TableLocationKey.java | 5 + .../impl/locations/TableLocationProvider.java | 32 ++- .../locations/TrackedTableLocationKey.java | 80 ------ .../locations/impl/AbstractTableLocation.java | 3 + .../impl/AbstractTableLocationProvider.java | 246 +++++++++--------- .../impl/CompositeTableDataService.java | 25 +- .../impl/FilteredTableDataService.java | 17 +- .../impl/SingleTableLocationProvider.java | 31 ++- .../impl/TableLocationSubscriptionBuffer.java | 41 +-- .../regioned/RegionedColumnSourceManager.java | 15 +- .../engine/liveness/LiveSupplier.java | 12 + ...ebergAutoRefreshTableLocationProvider.java | 3 +- ...ergManualRefreshTableLocationProvider.java | 3 +- .../location/ParquetTableLocationKey.java | 6 + 19 files changed, 290 insertions(+), 300 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java create mode 100644 engine/updategraph/src/main/java/io/deephaven/engine/liveness/LiveSupplier.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java index 080e11be44d..a2f2aa45515 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java @@ -3,12 +3,12 @@ // package io.deephaven.engine.table.impl; -import io.deephaven.engine.liveness.LivenessNode; +import io.deephaven.engine.liveness.LivenessReferent; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.TrackingWritableRowSet; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableLocation; -import io.deephaven.engine.table.impl.locations.TrackedTableLocationKey; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -18,7 +18,7 @@ /** * Manager for ColumnSources in a Table. */ -public interface ColumnSourceManager extends LivenessNode { +public interface ColumnSourceManager extends LivenessReferent { /** * Get a map of name to {@link ColumnSource} for the column sources maintained by this manager. @@ -112,8 +112,7 @@ public interface ColumnSourceManager extends LivenessNode { /** * Remove a table location key from the sources. * - * @return true if the location key was actually removed * @param tableLocationKey the location key being removed */ - boolean removeLocationKey(@NotNull TrackedTableLocationKey tableLocationKey); + void removeLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java index 9d9d6f87526..c93c0d2f0c3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java @@ -6,8 +6,8 @@ import io.deephaven.api.Selectable; import io.deephaven.api.filter.Filter; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.locations.TrackedTableLocationKey; import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; @@ -222,14 +222,14 @@ private static ColumnSource makePartitionSource(@NotNull final Co } @Override - protected final Collection filterLocationKeys( - @NotNull final Collection foundLocationKeys) { + protected final Collection> filterLocationKeys( + @NotNull final Collection> foundLocationKeys) { if (partitioningColumnFilters.length == 0) { return foundLocationKeys; } final Collection immutableTableLocationKeys = foundLocationKeys.stream() - .map(TrackedTableLocationKey::getKey) + .map(LiveSupplier::get) .collect(Collectors.toList()); // TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table @@ -242,8 +242,11 @@ protected final Collection filterLocationKeys( partitionTableColumnSources.add(makePartitionSource(columnDefinition, immutableTableLocationKeys)); } // Add the tracked keys to the table - partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource(foundLocationKeys, - TrackedTableLocationKey.class, null)); + // TODO: figure out how to do this + // partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource( + // foundLocationKeys, + // LiveSupplier.class, + // null)); final Table filteredColumnPartitionTable = TableTools .newTable(foundLocationKeys.size(), partitionTableColumnNames, partitionTableColumnSources) @@ -253,7 +256,7 @@ protected final Collection filterLocationKeys( } // Return the filtered keys - final Iterable iterable = + final Iterable> iterable = () -> filteredColumnPartitionTable.columnIterator(TRACKED_KEY_COLUMN_NAME); return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toList()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index 35d080b85da..ae971702514 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -4,6 +4,7 @@ package io.deephaven.engine.table.impl; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; @@ -166,7 +167,7 @@ protected void instrumentedRefresh() { final Collection locations = new ArrayList<>(); tableLocationProvider.getTableLocationKeys(tlk -> { - locations.add(tableLocationProvider.getTableLocation(tlk.getKey())); + locations.add(tableLocationProvider.getTableLocation(tlk.get())); }, locationKeyMatcher); try (final RowSet added = sortAndAddLocations(locations.stream())) { resultRows.insert(added); @@ -250,7 +251,7 @@ private RowSet processAdditions(final TableLocationSubscriptionBuffer.LocationUp */ // TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table locationUpdate.getPendingAddedLocationKeys().stream() - .map(TrackedTableLocationKey::getKey) + .map(LiveSupplier::get) .filter(locationKeyMatcher) .map(tableLocationProvider::getTableLocation) .map(PendingLocationState::new) @@ -272,7 +273,7 @@ private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpd final Set relevantRemovedLocations = locationUpdate.getPendingRemovedLocationKeys() .stream() - .map(TrackedTableLocationKey::getKey) + .map(LiveSupplier::get) .filter(locationKeyMatcher) .collect(Collectors.toSet()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index fa74ccce349..f57481700e7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -5,6 +5,7 @@ import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; +import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.TrackingWritableRowSet; import io.deephaven.engine.table.TableDefinition; @@ -148,33 +149,40 @@ private void initializeAvailableLocations() { updateSourceRegistrar.addSource(locationChangePoller = new LocationChangePoller(locationBuffer)); } else { locationProvider.refresh(); - final Collection tableLocationKeys = new ArrayList<>(); - locationProvider.getTableLocationKeys(tableLocationKeys::add); - maybeAddLocations(tableLocationKeys); + final Collection> keySuppliers = new ArrayList<>(); + // Manage each of the location keys as we see them (since the TLP is not guaranteeing them outside + // the callback) + locationProvider.getTableLocationKeys(ttlk -> { + manage(ttlk); + keySuppliers.add(ttlk); + }); + maybeAddLocations(keySuppliers); + // Now we can un-manage the location keys + keySuppliers.forEach(this::unmanage); } }); locationsInitialized = true; } } - private void maybeAddLocations(@NotNull final Collection locationKeys) { + private void maybeAddLocations(@NotNull final Collection> locationKeys) { if (locationKeys.isEmpty()) { return; } filterLocationKeys(locationKeys) .parallelStream() - .forEach(lk -> columnSourceManager.addLocation(locationProvider.getTableLocation(lk.getKey()))); + .forEach(lk -> columnSourceManager.addLocation(locationProvider.getTableLocation(lk.get()))); } - private TrackedTableLocationKey[] maybeRemoveLocations( - @NotNull final Collection removedKeys) { + private void maybeRemoveLocations( + @NotNull final Collection> removedKeys) { if (removedKeys.isEmpty()) { - return TrackedTableLocationKey.ZERO_LENGTH_TRACKED_TABLE_LOCATION_KEY_ARRAY; + return; } - return filterLocationKeys(removedKeys).stream() - .filter(columnSourceManager::removeLocationKey) - .toArray(TrackedTableLocationKey[]::new); + filterLocationKeys(removedKeys).stream() + .map(LiveSupplier::get) + .forEach(columnSourceManager::removeLocationKey); } private void initializeLocationSizes() { @@ -255,8 +263,8 @@ protected void onRefreshError(@NotNull final Exception error) { * {@link TableLocationProvider}, but not yet incorporated into the table * @return A sub-collection of the input */ - protected Collection filterLocationKeys( - @NotNull final Collection foundLocationKeys) { + protected Collection> filterLocationKeys( + @NotNull final Collection> foundLocationKeys) { return foundLocationKeys; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java index a3d3ad99011..29aba5a011c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java @@ -6,7 +6,7 @@ import io.deephaven.api.SortColumn; import io.deephaven.base.log.LogOutput; import io.deephaven.base.log.LogOutputAppendable; -import io.deephaven.engine.liveness.LivenessNode; +import io.deephaven.engine.liveness.LivenessReferent; import io.deephaven.engine.table.BasicDataIndex; import io.deephaven.engine.table.Table; import io.deephaven.io.log.impl.LogOutputStringImpl; @@ -22,7 +22,7 @@ * location allows access to columns, size, and possibly other metadata for a single partition that may be included in a * source table. */ -public interface TableLocation extends NamedImplementation, LogOutputAppendable, TableLocationState, LivenessNode { +public interface TableLocation extends NamedImplementation, LogOutputAppendable, TableLocationState, LivenessReferent { /** * Listener interface for anything that wants to know about changes to a location. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationKey.java index 8b67fd1b6bd..e0fa3ae29ee 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationKey.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationKey.java @@ -56,4 +56,9 @@ > PARTITION_VALUE_ * @return An immutable version of this key */ ImmutableTableLocationKey makeImmutable(); + + /** + * Release any cached data associated with this key. This would only be called at EOL for this key. + */ + default void clear() {} } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index e1f0df18266..4314b092659 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.locations; +import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.util.type.NamedImplementation; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -43,24 +44,24 @@ interface Listener extends BasicTableDataListener { void endTransaction(@NotNull Object token); /** - * Notify the listener of a {@link TrackedTableLocationKey} encountered while initiating or maintaining the - * location subscription. This should occur at most once per location, but the order of delivery is not - * guaranteed. This addition is not part of any transaction, and is equivalent to + * Notify the listener of a {@link LiveSupplier} encountered while initiating or + * maintaining the location subscription. This should occur at most once per location, but the order of delivery + * is not guaranteed. This addition is not part of any transaction, and is equivalent to * {@code handleTableLocationKeyAdded(tableLocationKey, null);} by default. * * @param tableLocationKey The new table location key. */ - void handleTableLocationKeyAdded(@NotNull TrackedTableLocationKey tableLocationKey); + void handleTableLocationKeyAdded(@NotNull LiveSupplier tableLocationKey); /** - * Notify the listener of a {@link TrackedTableLocationKey} that has been removed. This removal is not part of - * any transaction, and is equivalent to {@code handleTableLocationKeyRemoved(tableLocationKey, null);} by - * default. + * Notify the listener of a {@link LiveSupplier} that has been removed. This removal + * is not part of any transaction, and is equivalent to + * {@code handleTableLocationKeyRemoved(tableLocationKey, null);} by default. * * @param tableLocationKey The table location key that was removed. */ @SuppressWarnings("unused") - void handleTableLocationKeyRemoved(@NotNull TrackedTableLocationKey tableLocationKey); + void handleTableLocationKeyRemoved(@NotNull LiveSupplier tableLocationKey); /** *

    @@ -73,8 +74,8 @@ interface Listener extends BasicTableDataListener { * @param removedKeys Collection of table location keys that were removed. */ default void handleTableLocationKeysUpdate( - @NotNull Collection addedKeys, - @NotNull Collection removedKeys) { + @NotNull Collection> addedKeys, + @NotNull Collection> removedKeys) { removedKeys.forEach(this::handleTableLocationKeyRemoved); addedKeys.forEach(this::handleTableLocationKeyAdded); } @@ -131,8 +132,10 @@ default void handleTableLocationKeysUpdate( * Get this provider's currently known location keys. The locations specified by the keys returned may have null * size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is * guaranteed to succeed for all results. + * + * @param consumer A consumer to receive the location keys */ - default void getTableLocationKeys(Consumer consumer) { + default void getTableLocationKeys(Consumer> consumer) { getTableLocationKeys(consumer, key -> true); } @@ -140,8 +143,13 @@ default void getTableLocationKeys(Consumer consumer) { * Get this provider's currently known location keys. The locations specified by the keys returned may have null * size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is * guaranteed to succeed for all results. + * + * @param consumer A consumer to receive the location keys + * @param filter A filter to apply to the location keys before the consumer is called */ - void getTableLocationKeys(Consumer consumer, Predicate filter); + void getTableLocationKeys( + Consumer> consumer, + Predicate filter); /** * Check if this provider knows the supplied location key. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java deleted file mode 100644 index 7515e8f43c5..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TrackedTableLocationKey.java +++ /dev/null @@ -1,80 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.engine.table.impl.locations; - -import io.deephaven.base.verify.Assert; -import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; - -import java.util.function.Consumer; - -/** - * Sub-interface of {@link TableLocationKey} to mark immutable implementations. - */ -public class TrackedTableLocationKey extends ReferenceCountedLivenessNode { - - public static TrackedTableLocationKey[] ZERO_LENGTH_TRACKED_TABLE_LOCATION_KEY_ARRAY = - new TrackedTableLocationKey[0]; - - final ImmutableTableLocationKey locationKey; - final Consumer zeroCountConsumer; - - private TableLocation tableLocation; - - public TrackedTableLocationKey( - final ImmutableTableLocationKey locationKey, - final Consumer zeroCountConsumer) { - super(false); - - this.locationKey = locationKey; - this.zeroCountConsumer = zeroCountConsumer; - - tableLocation = null; - } - - public ImmutableTableLocationKey getKey() { - return locationKey; - } - - @Override - public int hashCode() { - return locationKey.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof ImmutableTableLocationKey) { - return locationKey.equals(obj); - } - return (this == obj); - } - - /** - * This {@link TrackedTableLocationKey} should manage the given {@link TableLocation} and store a reference to it. - * - * @param tableLocation The {@link TableLocation} to manage. - */ - public void manageTableLocation(TableLocation tableLocation) { - Assert.eqNull(this.tableLocation, "this.tableLocation"); - this.tableLocation = tableLocation; - manage(tableLocation); - } - - /** - * Unmanage the {@link TableLocation} and the release the reference. - */ - public void unmanageTableLocation() { - Assert.neqNull(this.tableLocation, "this.tableLocation"); - unmanage(tableLocation); - tableLocation = null; - } - - @Override - protected void destroy() { - super.destroy(); - if (tableLocation != null) { - unmanageTableLocation(); - } - zeroCountConsumer.accept(this); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index efd8da0526e..39b952fe7a4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java @@ -254,5 +254,8 @@ public final BasicDataIndex getDataIndex(@NotNull final String... columns) { private void destroy() { handleUpdate(null, System.currentTimeMillis()); clearColumnLocations(); + + // The key may be holding resources that can be cleared. + tableLocationKey.clear(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 4dc7f444636..bec0ceda65e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -4,12 +4,12 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.hash.KeyedObjectHashMap; -import io.deephaven.hash.KeyedObjectHashSet; import io.deephaven.hash.KeyedObjectKey; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -17,6 +17,7 @@ import java.util.*; import java.util.function.Consumer; import java.util.function.Predicate; +import java.util.stream.Collectors; /** * Partial {@link TableLocationProvider} implementation for standalone use or as part of a {@link TableDataService}. @@ -75,6 +76,63 @@ synchronized void removeLocationKey(ImmutableTableLocationKey locationKey) { } } + private static class TrackedKeySupplier extends ReferenceCountedLivenessNode + implements LiveSupplier { + enum State { + /** The key is included in the live set for new subscribers. */ + ACTIVE, + /** The key was removed from the live list and will not be provided to new subscribers */ + INACTIVE + } + + private final ImmutableTableLocationKey key; + private final Consumer zeroCountConsumer; + + private TableLocation tableLocation; + private State state; + + TrackedKeySupplier( + final ImmutableTableLocationKey key, + final Consumer zeroCountConsumer) { + super(false); + this.key = key; + this.zeroCountConsumer = zeroCountConsumer; + state = State.ACTIVE; + } + + @Override + public ImmutableTableLocationKey get() { + return key; + } + + /** + * This {@link TrackedKeySupplier} should manage the given {@link TableLocation} and store a reference to it. + * + * @param tableLocation The {@link TableLocation} to manage. + */ + public synchronized void setTableLocation(final TableLocation tableLocation) { + Assert.eqNull(this.tableLocation, "this.tableLocation"); + manage(tableLocation); + this.tableLocation = tableLocation; + } + + /** + * Change the state of this object to {@link State#INACTIVE}, indicates that this key is not included in the + * live set for new subscribers. + */ + public synchronized void deactivate() { + this.state = State.INACTIVE; + } + + @Override + protected synchronized void destroy() { + super.destroy(); + tableLocation = null; + state = State.INACTIVE; + zeroCountConsumer.accept(this); + } + } + private final ImmutableTableKey tableKey; private final ReferenceCountedLivenessNode livenessNode; @@ -82,26 +140,14 @@ synchronized void removeLocationKey(ImmutableTableLocationKey locationKey) { private final Map transactions = Collections.synchronizedMap(new HashMap<>()); /** - * Map from {@link TableLocationKey} to itself, or to a {@link TableLocation}. The values are {@link TableLocation}s - * if: - *

      - *
    1. The location has been requested via {@link #getTableLocation(TableLocationKey)} or - * {@link #getTableLocationIfPresent(TableLocationKey)}
    2. - *
    3. The {@link TableLocationKey} is a {@link TableLocation}
    4. - *
    + * Map from {@link TableLocationKey} to a {@link TrackedKeySupplier}. * * These values will not be cleared until all references to the {@link TableLocation} have been released by its * managers (i.e. {@link TableLocationSubscriptionBuffer subscriptions} and * {@link io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSourceManager column source managers}). */ - private final KeyedObjectHashMap tableLocations = - new KeyedObjectHashMap<>(LocationKeyDefinition.INSTANCE); - - /** - * The set of active location keys that will be returned to new subscribers. - */ - private final KeyedObjectHashSet liveLocationKeys = - new KeyedObjectHashSet<>(ImmutableTableLocationKeyDefinition.INSTANCE); + private final KeyedObjectHashMap tableLocationKeyMap = + new KeyedObjectHashMap<>(TableLocationKeyDefinition.INSTANCE); private volatile boolean initialized; @@ -153,9 +199,13 @@ public final ImmutableTableKey getKey() { @Override protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) { - // Lock the live set and deliver a copy to the listener - synchronized (liveLocationKeys) { - listener.handleTableLocationKeysUpdate(liveLocationKeys, List.of()); + // Lock on the map and deliver a copy to the listener + synchronized (tableLocationKeyMap) { + final List> keySuppliers = + tableLocationKeyMap.values().stream() + .filter(iks -> iks.state == TrackedKeySupplier.State.ACTIVE) + .collect(Collectors.toList()); + listener.handleTableLocationKeysUpdate(keySuppliers, List.of()); } } @@ -187,33 +237,35 @@ protected void endTransaction(@NotNull final Object token) { } // Return early if there are no changes to process - if (transaction.locationsAdded == EMPTY_TABLE_LOCATION_KEYS - && transaction.locationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + if (transaction.locationsAdded.isEmpty() && transaction.locationsRemoved.isEmpty()) { return; } - final Collection addedKeys = + final Collection> addedKeys = new ArrayList<>(transaction.locationsAdded.size()); - final Collection removedKeys = + final Collection> removedKeys = new ArrayList<>(transaction.locationsRemoved.size()); - synchronized (tableLocations) { + synchronized (tableLocationKeyMap) { for (ImmutableTableLocationKey locationKey : transaction.locationsRemoved) { - // Remove this from the live set. - final TrackedTableLocationKey trackedKey = liveLocationKeys.get(locationKey); - Assert.neqNull(trackedKey, "trackedKey"); - liveLocationKeys.remove(trackedKey); + final TrackedKeySupplier trackedKey = tableLocationKeyMap.get(locationKey); + if (trackedKey == null) { + // Not an error to remove a key multiple times (or a keu that was never added) + continue; + } + trackedKey.deactivate(); + // Pass this removed key to the subscribers removedKeys.add(trackedKey); } for (ImmutableTableLocationKey locationKey : transaction.locationsAdded) { locationCreatedRecorder = false; - final Object result = tableLocations.putIfAbsent(locationKey, this::observeInsert); + final TrackedKeySupplier result = tableLocationKeyMap.putIfAbsent(locationKey, this::observeInsert); visitLocationKey(locationKey); if (locationCreatedRecorder) { verifyPartitionKeys(locationKey); - addedKeys.add((TrackedTableLocationKey) result); + addedKeys.add(result); } } } @@ -228,7 +280,7 @@ protected void endTransaction(@NotNull final Object token) { true)) { onEmpty(); } - // Release the keys that were removed only after we have delivered the notifications and the + // Release the keys that were removed after we have delivered the notifications and the // subscribers have had a chance to process them removedKeys.forEach(livenessNode::unmanage); } @@ -255,7 +307,6 @@ protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey protected final void handleTableLocationKeyAdded( @NotNull final TableLocationKey locationKey, @Nullable final Object transactionToken) { - if (transactionToken != null) { final Transaction transaction = transactions.get(transactionToken); if (transaction == null) { @@ -263,12 +314,12 @@ protected final void handleTableLocationKeyAdded( "No transaction with token " + transactionToken + " is currently open."); } // Store an immutable key - transaction.addLocationKey(toKeyImmutable(locationKey)); + transaction.addLocationKey(locationKey.makeImmutable()); return; } if (!supportsSubscriptions()) { - tableLocations.putIfAbsent(locationKey, this::observeInsert); + tableLocationKeyMap.putIfAbsent(locationKey, this::observeInsert); visitLocationKey(locationKey); return; } @@ -280,13 +331,13 @@ protected final void handleTableLocationKeyAdded( // as it minimizes lock duration for tableLocations, exemplifies correct use of putIfAbsent, and keeps // observeInsert out of the business of subscription processing. locationCreatedRecorder = false; - final Object result = tableLocations.putIfAbsent(locationKey, this::observeInsert); + final TrackedKeySupplier result = tableLocationKeyMap.putIfAbsent(locationKey, this::observeInsert); visitLocationKey(locationKey); if (locationCreatedRecorder) { - verifyPartitionKeys(locationKey); + verifyPartitionKeys(result.get()); if (subscriptions.deliverNotification( Listener::handleTableLocationKeyAdded, - (TrackedTableLocationKey) result, + (LiveSupplier) result, true)) { onEmpty(); } @@ -310,29 +361,25 @@ protected void handleTableLocationKeyRemoved( throw new IllegalStateException( "No transaction with token " + transactionToken + " is currently open."); } - transaction.removeLocationKey(toKeyImmutable(locationKey)); + transaction.removeLocationKey(locationKey.makeImmutable()); return; } if (!supportsSubscriptions()) { - // Remove this from the live set and un-manage it. - final TrackedTableLocationKey trackedKey = liveLocationKeys.get(locationKey); - liveLocationKeys.remove(trackedKey); + final TrackedKeySupplier trackedKey = tableLocationKeyMap.get(locationKey); + trackedKey.deactivate(); livenessNode.unmanage(trackedKey); return; } // If we're not in a transaction, we should push this key immediately. synchronized (subscriptions) { - final Object removedLocation = tableLocations.remove(locationKey); - if (removedLocation != null) { - final TrackedTableLocationKey trackedKey = liveLocationKeys.get(locationKey); - Assert.neqNull(trackedKey, "trackedKey"); - // Remove this from the live set and un-manage it. - liveLocationKeys.remove(trackedKey); + final TrackedKeySupplier trackedKey = tableLocationKeyMap.removeKey(locationKey); + if (trackedKey != null) { + trackedKey.deactivate(); if (subscriptions.deliverNotification( Listener::handleTableLocationKeyRemoved, - trackedKey, + (LiveSupplier) trackedKey, true)) { onEmpty(); } @@ -352,15 +399,13 @@ protected void handleTableLocationKeyRemoved( protected void visitLocationKey(@NotNull final TableLocationKey locationKey) {} @NotNull - private Object observeInsert(@NotNull final TableLocationKey locationKey) { + private TrackedKeySupplier observeInsert(@NotNull final TableLocationKey locationKey) { // NB: This must only be called while the lock on subscriptions is held. locationCreatedRecorder = true; - final TrackedTableLocationKey trackedKey = toTrackedKey(locationKey); + final TrackedKeySupplier trackedKey = toTrackedKey(locationKey); livenessNode.manage(trackedKey); - // Add this to the live set. - liveLocationKeys.add(trackedKey); return trackedKey; } @@ -410,51 +455,46 @@ protected void doInitialization() { @Override public void getTableLocationKeys( - final Consumer consumer, + final Consumer> consumer, final Predicate filter) { // Lock the live set and deliver a copy to the listener after filtering. - synchronized (liveLocationKeys) { - liveLocationKeys.stream().filter(ttlk -> filter.test(ttlk.getKey())).forEach(consumer); + synchronized (tableLocationKeyMap) { + tableLocationKeyMap.values().stream() + .filter(ttlk -> ttlk.state == TrackedKeySupplier.State.ACTIVE) + .filter(ttlk -> filter.test(ttlk.get())).forEach(consumer); } } @Override public final boolean hasTableLocationKey(@NotNull final TableLocationKey tableLocationKey) { - return tableLocations.containsKey(tableLocationKey); + return tableLocationKeyMap.containsKey(tableLocationKey); } @Override @Nullable public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey tableLocationKey) { - Object current = tableLocations.get(tableLocationKey); - if (current == null) { + TrackedKeySupplier trackedKey = tableLocationKeyMap.get(tableLocationKey); + if (trackedKey == null) { return null; } // See JavaDoc on tableLocations for background. - // The intent is to create a TableLocation exactly once to replace the TableLocationKey placeholder that was - // added in handleTableLocationKey. - if (!(current instanceof TableLocation)) { - final TrackedTableLocationKey trackedKey = (TrackedTableLocationKey) current; - // noinspection SynchronizationOnLocalVariableOrMethodParameter + if (trackedKey.tableLocation == null) { synchronized (trackedKey) { - current = tableLocations.get(trackedKey); - if (trackedKey == current) { - // Make a new location, have the tracked key manage it, then replace the key with the - // new location in the map and return it. Note, this may contend for the lock on tableLocations - final TableLocation newLocation = makeTableLocation(trackedKey.getKey()); - trackedKey.manageTableLocation(newLocation); - tableLocations.add(current = newLocation); + if (trackedKey.tableLocation == null) { + // Make a new location, have the tracked key manage it, then store the location in the tracked key. + final TableLocation newLocation = makeTableLocation(trackedKey.get()); + trackedKey.setTableLocation(newLocation); } } } - return (TableLocation) current; + return trackedKey.tableLocation; } /** * Remove a {@link TableLocationKey} and its corresponding {@link TableLocation} (if it was created). All * subscribers to this TableLocationProvider will be - * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(TrackedTableLocationKey) notified}. If the - * TableLocation was created, all of its subscribers will additionally be + * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(LiveSupplier) + * notified}. If the TableLocation was created, all of its subscribers will additionally be * {@link TableLocation.Listener#handleUpdate() notified} that it no longer exists. This TableLocationProvider will * continue to update other locations and will no longer provide or request information about the removed location. * @@ -489,60 +529,26 @@ private void verifyPartitionKeys(@NotNull final TableLocationKey locationKey) { } /** - * Key definition for {@link TableLocation} or {@link TableLocationKey} lookup by {@link TableLocationKey}. + * Key definition for {@link LiveSupplier} lookup by {@link ImmutableTableLocationKey}. */ - private static final class LocationKeyDefinition extends KeyedObjectKey.Basic { + private static final class TableLocationKeyDefinition + extends KeyedObjectKey.Basic { - private static final KeyedObjectKey INSTANCE = new LocationKeyDefinition(); + private static final KeyedObjectKey INSTANCE = + new TableLocationKeyDefinition(); - private LocationKeyDefinition() {} + private TableLocationKeyDefinition() {} @Override - public TableLocationKey getKey(@NotNull final Object keyOrLocation) { - return toKey(keyOrLocation); + public TableLocationKey getKey(TrackedKeySupplier immutableKeySupplier) { + return immutableKeySupplier.get(); } } - /** - * Key definition for {@link TrackedTableLocationKey} lookup by {@link ImmutableTableLocationKey}. - */ - private static final class ImmutableTableLocationKeyDefinition - extends KeyedObjectKey.Basic { - - private static final KeyedObjectKey INSTANCE = - new ImmutableTableLocationKeyDefinition(); - - private ImmutableTableLocationKeyDefinition() {} - - @Override - public ImmutableTableLocationKey getKey(TrackedTableLocationKey trackedTableLocationKey) { - return trackedTableLocationKey.getKey(); - } + private TrackedKeySupplier toTrackedKey(@NotNull final TableLocationKey locationKey) { + return new TrackedKeySupplier(locationKey.makeImmutable(), this::releaseLocationKey); } - private static TableLocationKey toKey(@NotNull final Object keyOrLocation) { - if (keyOrLocation instanceof TableLocation) { - return ((TableLocation) keyOrLocation).getKey(); - } - if (keyOrLocation instanceof TrackedTableLocationKey) { - return (((TrackedTableLocationKey) keyOrLocation).getKey()); - } - if (keyOrLocation instanceof TableLocationKey) { - return ((TableLocationKey) keyOrLocation); - } - throw new IllegalArgumentException( - "toKey expects a TableLocation or a TableLocationKey, instead received a " + keyOrLocation.getClass()); - } - - private static ImmutableTableLocationKey toKeyImmutable(@NotNull final Object keyOrLocation) { - return (ImmutableTableLocationKey) toKey(keyOrLocation); - } - - private TrackedTableLocationKey toTrackedKey(@NotNull final TableLocationKey locationKey) { - return new TrackedTableLocationKey(locationKey.makeImmutable(), this::releaseLocationKey); - } - - private static boolean equals(Collection c1, Collection c2) { final Iterator i2 = c2.iterator(); for (T t1 : c1) { @@ -557,13 +563,13 @@ private static boolean equals(Collection c1, Collection c2) { } /** - * Called when every reference to the {@link TrackedTableLocationKey key} has been released. + * Called when every reference to the {@link LiveSupplier key} has been released. * * @param locationKey the key to release */ - private void releaseLocationKey(@NotNull final TrackedTableLocationKey locationKey) { + private void releaseLocationKey(@NotNull final TrackedKeySupplier locationKey) { // We can now remove the key from the tableLocations map - tableLocations.remove(locationKey.getKey()); + tableLocationKeyMap.removeKey(locationKey.get()); } private void destroy() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index 42d0541039a..9551c2d72b8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -4,6 +4,7 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.base.verify.Require; +import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.util.Formatter; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.hash.KeyedObjectHashSet; @@ -144,15 +145,18 @@ public TableLocationProvider ensureInitialized() { @Override public void getTableLocationKeys( - final Consumer consumer, + final Consumer> consumer, final Predicate filter) { - final Set locationKeys = new KeyedObjectHashSet<>(KeyKeyDefinition.INSTANCE); + final Set> locationKeys = + new KeyedObjectHashSet<>(KeyKeyDefinition.INSTANCE); try (final SafeCloseable ignored = CompositeTableDataServiceConsistencyMonitor.INSTANCE.start()) { // Add all the location keys from the providers to the set, throw an exception if there are duplicates inputProviders.forEach(p -> p.getTableLocationKeys(tlk -> { - if (filter.test(tlk.getKey()) && !locationKeys.add(tlk)) { + if (!locationKeys.add(tlk)) { + // Consume the key immediately (while the key is still managed by the input provider) + consumer.accept(tlk); final String overlappingProviders = inputProviders.stream() - .filter(inputProvider -> inputProvider.hasTableLocationKey(tlk.getKey())) + .filter(inputProvider -> inputProvider.hasTableLocationKey(tlk.get())) .map(TableLocationProvider::getName) .collect(Collectors.joining(",")); throw new TableDataException( @@ -164,9 +168,7 @@ public void getTableLocationKeys( .formatTableDataService(CompositeTableDataService.this.toString())); } - })); - // Consume all the location keys - locationKeys.forEach(consumer); + }, filter)); } } @@ -228,16 +230,17 @@ public String describe() { // ------------------------------------------------------------------------------------------------------------------ private static final class KeyKeyDefinition - extends KeyedObjectKey.Basic { + extends KeyedObjectKey.Basic> { - private static final KeyedObjectKey INSTANCE = + private static final KeyedObjectKey> INSTANCE = new KeyKeyDefinition(); private KeyKeyDefinition() {} @Override - public ImmutableTableLocationKey getKey(@NotNull final TrackedTableLocationKey tableLocationKey) { - return tableLocationKey.getKey(); + public ImmutableTableLocationKey getKey( + @NotNull final LiveSupplier tableLocationKey) { + return tableLocationKey.get(); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index fe9158a8488..9c71d3ef67e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -5,6 +5,7 @@ import io.deephaven.base.reference.WeakReferenceWrapper; import io.deephaven.base.verify.Require; +import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.table.impl.locations.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -126,7 +127,7 @@ public TableLocationProvider ensureInitialized() { @Override public void getTableLocationKeys( - final Consumer consumer, + final Consumer> consumer, final Predicate filter) { inputProvider.getTableLocationKeys(consumer, filter); } @@ -177,27 +178,29 @@ public void endTransaction(@NotNull final Object token) { } @Override - public void handleTableLocationKeyAdded(@NotNull final TrackedTableLocationKey tableLocationKey) { + public void handleTableLocationKeyAdded( + @NotNull final LiveSupplier tableLocationKey) { final TableLocationProvider.Listener outputListener = getWrapped(); // We can't try to clean up null listeners here, the underlying implementation may not allow concurrent // unsubscribe operations. - if (outputListener != null && locationKeyFilter.accept(tableLocationKey.getKey())) { + if (outputListener != null && locationKeyFilter.accept(tableLocationKey.get())) { outputListener.handleTableLocationKeyAdded(tableLocationKey); } } @Override - public void handleTableLocationKeyRemoved(@NotNull final TrackedTableLocationKey tableLocationKey) { + public void handleTableLocationKeyRemoved( + @NotNull final LiveSupplier tableLocationKey) { final TableLocationProvider.Listener outputListener = getWrapped(); - if (outputListener != null && locationKeyFilter.accept(tableLocationKey.getKey())) { + if (outputListener != null && locationKeyFilter.accept(tableLocationKey.get())) { outputListener.handleTableLocationKeyRemoved(tableLocationKey); } } @Override public void handleTableLocationKeysUpdate( - @NotNull Collection addedKeys, - @NotNull Collection removedKeys) { + @NotNull Collection> addedKeys, + @NotNull Collection> removedKeys) { final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null) { outputListener.handleTableLocationKeysUpdate(addedKeys, removedKeys); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java index e7c1bde1284..f0a491ccb41 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java @@ -3,6 +3,8 @@ // package io.deephaven.engine.table.impl.locations.impl; +import io.deephaven.engine.liveness.LiveSupplier; +import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; import io.deephaven.engine.table.impl.locations.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -14,10 +16,24 @@ * A {@link TableLocationProvider} that provides access to exactly one, previously-known {@link TableLocation}. */ public final class SingleTableLocationProvider implements TableLocationProvider { - private static final String IMPLEMENTATION_NAME = SingleTableLocationProvider.class.getSimpleName(); - private final TrackedTableLocationKey trackedTableLocationKey; + private static class TrackedKeySupplier extends ReferenceCountedLivenessNode + implements LiveSupplier { + final ImmutableTableLocationKey key; + + protected TrackedKeySupplier(final ImmutableTableLocationKey key) { + super(false); + this.key = key; + } + + @Override + public ImmutableTableLocationKey get() { + return key; + } + } + + private final TrackedKeySupplier immutableKeySupplier; private final TableLocation tableLocation; /** @@ -25,9 +41,8 @@ public final class SingleTableLocationProvider implements TableLocationProvider */ public SingleTableLocationProvider(@NotNull final TableLocation tableLocation) { this.tableLocation = tableLocation; - trackedTableLocationKey = new TrackedTableLocationKey(tableLocation.getKey(), ttlk -> { - // TODO: I don't think we need to do anything here, but need to think more about it - }); + // TODO: it seems like we should manage this, but SingleTableLocationProvider isn't a LivenessManager. + immutableKeySupplier = new TrackedKeySupplier(tableLocation.getKey()); } @Override @@ -65,10 +80,10 @@ public TableLocationProvider ensureInitialized() { @Override public void getTableLocationKeys( - final Consumer consumer, + final Consumer> consumer, final Predicate filter) { - if (filter.test(trackedTableLocationKey.getKey())) { - consumer.accept(trackedTableLocationKey); + if (filter.test(immutableKeySupplier.get())) { + consumer.accept(immutableKeySupplier); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index d01f7d034ca..981cc11d720 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -4,8 +4,9 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.base.verify.Require; +import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; -import io.deephaven.engine.table.impl.locations.TrackedTableLocationKey; +import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocationProvider; import io.deephaven.util.SafeCloseable; @@ -20,7 +21,8 @@ public class TableLocationSubscriptionBuffer extends ReferenceCountedLivenessNode implements TableLocationProvider.Listener { - private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet(); + private static final Set> EMPTY_TABLE_LOCATION_KEYS = + Collections.emptySet(); private final TableLocationProvider tableLocationProvider; @@ -29,8 +31,8 @@ public class TableLocationSubscriptionBuffer extends ReferenceCountedLivenessNod private final Object updateLock = new Object(); // These sets represent adds and removes from completed transactions. - private Set pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; - private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + private Set> pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; + private Set> pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; private TableDataException pendingException = null; @@ -40,20 +42,21 @@ public TableLocationSubscriptionBuffer(@NotNull final TableLocationProvider tabl } public final class LocationUpdate implements SafeCloseable { - private final Collection pendingAddedLocationKeys; - private final Collection pendingRemovedLocations; + private final Collection> pendingAddedLocationKeys; + private final Collection> pendingRemovedLocations; - public LocationUpdate(@NotNull final Collection pendingAddedLocationKeys, - @NotNull final Collection pendingRemovedLocations) { + public LocationUpdate( + @NotNull final Collection> pendingAddedLocationKeys, + @NotNull final Collection> pendingRemovedLocations) { this.pendingAddedLocationKeys = pendingAddedLocationKeys; this.pendingRemovedLocations = pendingRemovedLocations; } - public Collection getPendingAddedLocationKeys() { + public Collection> getPendingAddedLocationKeys() { return pendingAddedLocationKeys; } - public Collection getPendingRemovedLocationKeys() { + public Collection> getPendingRemovedLocationKeys() { return pendingRemovedLocations; } @@ -81,14 +84,14 @@ public synchronized LocationUpdate processPending() { // NB: Providers that don't support subscriptions don't tick - this single call to run is // sufficient. tableLocationProvider.refresh(); - final Collection tableLocationKeys = new ArrayList<>(); + final Collection> tableLocationKeys = new ArrayList<>(); tableLocationProvider.getTableLocationKeys(tableLocationKeys::add); handleTableLocationKeysUpdate(tableLocationKeys, List.of()); } subscribed = true; } - final Collection resultLocationKeys; - final Collection resultLocationsRemoved; + final Collection> resultLocationKeys; + final Collection> resultLocationsRemoved; final TableDataException resultException; synchronized (updateLock) { resultLocationKeys = pendingLocationsAdded; @@ -140,7 +143,7 @@ public void endTransaction(@NotNull final Object token) { } @Override - public void handleTableLocationKeyAdded(@NotNull final TrackedTableLocationKey tableLocationKey) { + public void handleTableLocationKeyAdded(@NotNull final LiveSupplier tableLocationKey) { synchronized (updateLock) { // Need to verify that we don't have stacked adds (without intervening removes). if (pendingLocationsAdded.contains(tableLocationKey)) { @@ -156,7 +159,7 @@ public void handleTableLocationKeyAdded(@NotNull final TrackedTableLocationKey t } @Override - public void handleTableLocationKeyRemoved(@NotNull final TrackedTableLocationKey tableLocationKey) { + public void handleTableLocationKeyRemoved(@NotNull final LiveSupplier tableLocationKey) { synchronized (updateLock) { // If we have a pending add, it is being cancelled by this remove. if (pendingLocationsAdded.remove(tableLocationKey)) { @@ -177,11 +180,11 @@ public void handleTableLocationKeyRemoved(@NotNull final TrackedTableLocationKey @Override public void handleTableLocationKeysUpdate( - @Nullable Collection addedKeys, - @Nullable Collection removedKeys) { + @Nullable Collection> addedKeys, + @Nullable Collection> removedKeys) { synchronized (updateLock) { if (removedKeys != null) { - for (final TrackedTableLocationKey removedTableLocationKey : removedKeys) { + for (final LiveSupplier removedTableLocationKey : removedKeys) { // If we have a pending add, it is being cancelled by this remove. if (pendingLocationsAdded.remove(removedTableLocationKey)) { continue; @@ -199,7 +202,7 @@ public void handleTableLocationKeysUpdate( } } if (addedKeys != null) { - for (final TrackedTableLocationKey addedTableLocationKey : addedKeys) { + for (final LiveSupplier addedTableLocationKey : addedKeys) { // Need to verify that we don't have stacked adds. if (pendingLocationsAdded.contains(addedTableLocationKey)) { throw new IllegalStateException("TableLocationKey " + addedTableLocationKey diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index b22eeff9786..7820ec179ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -205,11 +205,11 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im invalidateCommitter = new UpdateCommitter<>(this, ExecutionContext.getContext().getUpdateGraph(), - (ignored) -> { - synchronized (this) { + (instance) -> { + synchronized (instance) { invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate); invalidatedLocations.clear(); - releasedLocations.forEach(this::unmanage); + releasedLocations.forEach(instance::unmanage); releasedLocations.clear(); } }); @@ -253,7 +253,7 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation) } @Override - public synchronized boolean removeLocationKey(final @NotNull TrackedTableLocationKey locationKey) { + public synchronized void removeLocationKey(final @NotNull ImmutableTableLocationKey locationKey) { final IncludedTableLocationEntry includedLocation = includedTableLocations.remove(locationKey); final EmptyTableLocationEntry emptyLocation = emptyTableLocations.remove(locationKey); @@ -272,10 +272,7 @@ public synchronized boolean removeLocationKey(final @NotNull TrackedTableLocatio releasedLocations.add((AbstractTableLocation) includedLocation.location); } invalidateCommitter.maybeActivate(); - return true; } - - return false; } @Override @@ -283,8 +280,8 @@ public synchronized TrackingWritableRowSet initialize() { Assert.assertion(includedLocationsTable.isEmpty(), "includedLocationsTable.isEmpty()"); // Do our first pass over the locations to include as many as possible and build the initial row set - // noinspection resource final TableUpdateImpl update = update(true); + // noinspection resource final TrackingWritableRowSet initialRowSet = update.added().writableCast().toTracking(); update.added = null; update.release(); @@ -360,7 +357,7 @@ private TableUpdateImpl update(final boolean initializing) { removedTableLocations.sort(Comparator.comparingInt(e -> e.regionIndex)); for (final IncludedTableLocationEntry removedLocation : removedTableLocations) { final long regionFirstKey = RegionedColumnSource.getFirstRowKey(removedLocation.regionIndex); - removedRowSetBuilder.appendRowSequenceWithOffset(removedLocation.location.getRowSet(), regionFirstKey); + removedRowSetBuilder.appendRowSequenceWithOffset(removedLocation.rowSetAtLastUpdate, regionFirstKey); removedRegionBuilder.appendKey(removedLocation.regionIndex); } diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LiveSupplier.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LiveSupplier.java new file mode 100644 index 00000000000..4343ce4de01 --- /dev/null +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LiveSupplier.java @@ -0,0 +1,12 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.liveness; + +import java.util.function.Supplier; + +/** + * Interface for a supplier that continues to "live" while retained by a {@link LivenessManager}. + */ +public interface LiveSupplier extends LivenessReferent, Supplier { +} diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index 09287e066a1..5d2b302b611 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -89,8 +89,7 @@ public void update(Snapshot snapshot) { private void refreshSnapshot() { beginTransaction(this); final Set missedKeys = new HashSet<>(); - getTableLocationKeys(ttlk -> missedKeys.add(ttlk.getKey())); - + getTableLocationKeys(ttlk -> missedKeys.add(ttlk.get())); locationKeyFinder.findKeys(tableLocationKey -> { missedKeys.remove(tableLocationKey); handleTableLocationKeyAdded(tableLocationKey, this); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index be0bae3a20e..68d558b4d7f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -97,8 +97,7 @@ public synchronized void update(final Snapshot snapshot) { private void refreshSnapshot() { beginTransaction(this); final Set missedKeys = new HashSet<>(); - getTableLocationKeys(ttlk -> missedKeys.add(ttlk.getKey())); - + getTableLocationKeys(ttlk -> missedKeys.add(ttlk.get())); locationKeyFinder.findKeys(tlk -> { missedKeys.remove(tlk); handleTableLocationKeyAdded(tlk, this); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java index cafff0a3210..e3002c98bf0 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java @@ -178,4 +178,10 @@ public synchronized int[] getRowGroupIndices() { public synchronized void setRowGroupIndices(final int[] rowGroupIndices) { this.rowGroupIndices = rowGroupIndices; } + + @Override + public void clear() { + metadata = null; + fileReader = null; + } } From 01e50feadd43270c2f4d2f2f9ee15eaabb7ef10f Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 18 Sep 2024 15:51:24 -0700 Subject: [PATCH 34/56] Updated but RCSM still not referenced properly. --- .../engine/table/impl/SourceTable.java | 1 + .../impl/AbstractTableLocationProvider.java | 32 +++---- .../regioned/RegionedColumnSourceManager.java | 6 +- .../engine/liveness/LivenessManager.java | 51 +++++++++++ .../engine/liveness/LivenessNode.java | 52 ----------- .../liveness/PermanentLivenessManager.java | 12 +++ .../liveness/SingletonLivenessManager.java | 13 +++ .../liveness/StandaloneLivenessManager.java | 90 +++++++++++++++++++ .../iceberg/util/IcebergToolsTest.java | 3 +- .../iceberg/util/IcebergTableAdapter.java | 3 +- 10 files changed, 185 insertions(+), 78 deletions(-) create mode 100644 engine/updategraph/src/main/java/io/deephaven/engine/liveness/StandaloneLivenessManager.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index f57481700e7..81e92a49eab 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -98,6 +98,7 @@ public abstract class SourceTable> exte definition.getColumns() // This is the *re-written* definition passed to the super-class constructor ); if (isRefreshing) { + // TODO: managing doesn't work here because columnSourceManager is at zero right now. manage(columnSourceManager); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index bec0ceda65e..71d2438df90 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -5,8 +5,8 @@ import io.deephaven.base.verify.Assert; import io.deephaven.engine.liveness.LiveSupplier; -import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; +import io.deephaven.engine.liveness.StandaloneLivenessManager; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.hash.KeyedObjectHashMap; @@ -134,7 +134,6 @@ protected synchronized void destroy() { } private final ImmutableTableKey tableKey; - private final ReferenceCountedLivenessNode livenessNode; // Open transactions that are being accumulated private final Map transactions = Collections.synchronizedMap(new HashMap<>()); @@ -149,6 +148,13 @@ protected synchronized void destroy() { private final KeyedObjectHashMap tableLocationKeyMap = new KeyedObjectHashMap<>(TableLocationKeyDefinition.INSTANCE); + /** + * TLP are (currently) outside of liveness scopes, but they need to manage liveness referents to prevent them from + * going out of scope. The {@link StandaloneLivenessManager manager} will maintain the references until GC'd or the + * referents are not needed byt the TLP. + */ + private final StandaloneLivenessManager livenessManager; + private volatile boolean initialized; private List partitionKeys; @@ -165,14 +171,7 @@ protected AbstractTableLocationProvider(@NotNull final TableKey tableKey, final this.tableKey = tableKey.makeImmutable(); this.partitionKeys = null; - livenessNode = new ReferenceCountedLivenessNode(false) { - @Override - protected void destroy() { - AbstractTableLocationProvider.this.destroy(); - } - }; - // TODO: understand why this seems to be needed - LivenessScopeStack.peek().manage(livenessNode); + livenessManager = new StandaloneLivenessManager(false); } /** @@ -282,7 +281,7 @@ protected void endTransaction(@NotNull final Object token) { } // Release the keys that were removed after we have delivered the notifications and the // subscribers have had a chance to process them - removedKeys.forEach(livenessNode::unmanage); + removedKeys.forEach(livenessManager::unmanage); } } } @@ -368,7 +367,7 @@ protected void handleTableLocationKeyRemoved( if (!supportsSubscriptions()) { final TrackedKeySupplier trackedKey = tableLocationKeyMap.get(locationKey); trackedKey.deactivate(); - livenessNode.unmanage(trackedKey); + livenessManager.unmanage(trackedKey); return; } @@ -383,7 +382,7 @@ protected void handleTableLocationKeyRemoved( true)) { onEmpty(); } - livenessNode.unmanage(trackedKey); + livenessManager.unmanage(trackedKey); } } } @@ -404,7 +403,7 @@ private TrackedKeySupplier observeInsert(@NotNull final TableLocationKey locatio locationCreatedRecorder = true; final TrackedKeySupplier trackedKey = toTrackedKey(locationKey); - livenessNode.manage(trackedKey); + livenessManager.manage(trackedKey); return trackedKey; } @@ -571,9 +570,4 @@ private void releaseLocationKey(@NotNull final TrackedKeySupplier locationKey) { // We can now remove the key from the tableLocations map tableLocationKeyMap.removeKey(locationKey.get()); } - - private void destroy() { - // TODO: release all the TTLK references. Or does that happen automatically? - throw new UnsupportedOperationException("Not yet implemented"); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 7820ec179ad..45dde30cb03 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -152,11 +152,6 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im @NotNull final ColumnToCodecMappings codecMappings, @NotNull final List> columnDefinitions) { super(false); - // TODO: this seems to be required since the transition to ReferenceCountedLivenessNode (from LivenessArtifact) - // if (Liveness.REFERENCE_TRACKING_DISABLED) { - // return; - // } - LivenessScopeStack.peek().manage(this); this.isRefreshing = isRefreshing; this.columnDefinitions = columnDefinitions; @@ -197,6 +192,7 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im }; if (isRefreshing) { rowSetModifiedColumnSet = includedLocationsTable.newModifiedColumnSet(ROWS_SET_COLUMN_NAME); + // TODO: managing doesn't work here because we are at zero right now. manage(includedLocationsTable); } else { rowSetModifiedColumnSet = null; diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessManager.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessManager.java index db7c336e985..177acd509ec 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessManager.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessManager.java @@ -6,6 +6,8 @@ import io.deephaven.util.annotations.FinalDefault; import io.deephaven.util.referencecounting.ReferenceCounted; import org.jetbrains.annotations.NotNull; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Interface for objects that retainReference {@link LivenessReferent}s until such time as they are no longer necessary. @@ -36,4 +38,53 @@ default void manage(@NotNull final LivenessReferent referent) { * @return Whether the referent was in fact added */ boolean tryManage(@NotNull LivenessReferent referent); + + /** + * If this manager manages {@code referent} one or more times, drop one such reference. If this manager is also a + * {@link LivenessReferent}, then it must also be live. + * + * @param referent The referent to drop + */ + @FinalDefault + default void unmanage(@NotNull LivenessReferent referent) { + if (!tryUnmanage(referent)) { + throw new LivenessStateException(this + " is no longer live and cannot unmanage " + + referent.getReferentDescription()); + } + } + + /** + * If this manager manages referent one or more times, drop one such reference. If this manager is also a + * {@link LivenessReferent}, then this method is a no-op if {@code this} is not live. + * + * @param referent The referent to drop + * @return Whether this node was live and thus in fact tried to drop a reference + */ + boolean tryUnmanage(@NotNull LivenessReferent referent); + + /** + * For each referent in {@code referent}, if this manager manages referent one or more times, drop one such + * reference. If this manager is also a {@link LivenessReferent}, then it must also be live. + * + * + * @param referents The referents to drop + */ + @SuppressWarnings("unused") + @FinalDefault + default void unmanage(@NotNull Stream referents) { + if (!tryUnmanage(referents)) { + throw new LivenessStateException(this + " is no longer live and cannot unmanage " + + referents.map(LivenessReferent::getReferentDescription).collect(Collectors.joining())); + } + } + + /** + * For each referent in referents, if this manager manages referent one or more times, drop one such reference. If + * this manager is also a {@link LivenessReferent}, then this method is a no-op if {@code this} is not live. + * + * @param referents The referents to drop + * @return Whether this node was live and thus in fact tried to drop a reference + */ + @SuppressWarnings("unused") + boolean tryUnmanage(@NotNull Stream referents); } diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessNode.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessNode.java index 5a7c52bbbfd..233608d3c81 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessNode.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessNode.java @@ -3,60 +3,8 @@ // package io.deephaven.engine.liveness; -import io.deephaven.util.annotations.FinalDefault; -import org.jetbrains.annotations.NotNull; - -import java.util.stream.Collectors; -import java.util.stream.Stream; - /** * A {@link LivenessReferent} that is also a {@link LivenessManager}, transitively enforcing liveness on its referents. */ public interface LivenessNode extends LivenessReferent, LivenessManager { - - /** - * If this node manages {@code referent} one or more times, drop one such reference. This node must be live. - * - * @param referent The referent to drop - */ - @FinalDefault - default void unmanage(@NotNull LivenessReferent referent) { - if (!tryUnmanage(referent)) { - throw new LivenessStateException(this + " is no longer live and cannot unmanage " + - referent.getReferentDescription()); - } - } - - /** - * If this node is still live and manages referent one or more times, drop one such reference. - * - * @param referent The referent to drop - * @return Whether this node was live and thus in fact tried to drop a reference - */ - boolean tryUnmanage(@NotNull LivenessReferent referent); - - /** - * For each referent in {@code referent}, if this node manages referent one or more times, drop one such reference. - * This node must be live. - * - * @param referents The referents to drop - */ - @SuppressWarnings("unused") - @FinalDefault - default void unmanage(@NotNull Stream referents) { - if (!tryUnmanage(referents)) { - throw new LivenessStateException(this + " is no longer live and cannot unmanage " + - referents.map(LivenessReferent::getReferentDescription).collect(Collectors.joining())); - } - } - - /** - * For each referent in referents, if this node is still live and manages referent one or more times, drop one such - * reference. - * - * @param referents The referents to drop - * @return Whether this node was live and thus in fact tried to drop a reference - */ - @SuppressWarnings("unused") - boolean tryUnmanage(@NotNull Stream referents); } diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/PermanentLivenessManager.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/PermanentLivenessManager.java index d4e5fca2986..16c61be5e40 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/PermanentLivenessManager.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/PermanentLivenessManager.java @@ -6,6 +6,8 @@ import io.deephaven.util.Utils; import org.jetbrains.annotations.NotNull; +import java.util.stream.Stream; + /** *

    * A {@link LivenessManager} implementation that will never release its referents. @@ -31,4 +33,14 @@ public boolean tryManage(@NotNull LivenessReferent referent) { } return true; } + + @Override + public boolean tryUnmanage(@NotNull LivenessReferent referent) { + throw new UnsupportedOperationException("PermanentLivenessManager cannot unmanage referents"); + } + + @Override + public boolean tryUnmanage(@NotNull Stream referents) { + throw new UnsupportedOperationException("PermanentLivenessManager cannot unmanage referents"); + } } diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/SingletonLivenessManager.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/SingletonLivenessManager.java index ed05a356008..bacbb972b97 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/SingletonLivenessManager.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/SingletonLivenessManager.java @@ -8,6 +8,7 @@ import java.lang.ref.WeakReference; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.stream.Stream; /** * {@link ReleasableLivenessManager} to manage exactly one object, passed at construction time or managed later. @@ -61,6 +62,18 @@ public final boolean tryManage(@NotNull LivenessReferent referent) { return true; } + @Override + public boolean tryUnmanage(@NotNull LivenessReferent referent) { + throw new UnsupportedOperationException( + "SingletonLivenessManager should call release() instead of tryUnmanage()"); + } + + @Override + public boolean tryUnmanage(@NotNull Stream referents) { + throw new UnsupportedOperationException( + "SingletonLivenessManager should call release() instead of tryUnmanage()"); + } + @Override public final void release() { if (Liveness.REFERENCE_TRACKING_DISABLED) { diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/StandaloneLivenessManager.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/StandaloneLivenessManager.java new file mode 100644 index 00000000000..c19dd0027ba --- /dev/null +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/StandaloneLivenessManager.java @@ -0,0 +1,90 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.liveness; + +import io.deephaven.base.log.LogOutput; +import io.deephaven.base.log.LogOutputAppendable; +import io.deephaven.util.Utils; +import io.deephaven.util.annotations.VisibleForTesting; +import org.jetbrains.annotations.NotNull; + +import java.util.stream.Stream; + +/** + * {@link LivenessNode} implementation that relies on reference counting to determine its liveness. + */ +public class StandaloneLivenessManager implements LivenessManager, LogOutputAppendable { + + final boolean enforceStrongReachability; + + transient RetainedReferenceTracker tracker; + + /** + * @param enforceStrongReachability Whether this {@link LivenessManager} should maintain strong references to its + * referents + */ + @SuppressWarnings("WeakerAccess") // Needed in order to deserialize Serializable subclass instances + public StandaloneLivenessManager(final boolean enforceStrongReachability) { + this.enforceStrongReachability = enforceStrongReachability; + initializeTransientFieldsForLiveness(); + } + + /** + * Package-private for {@link java.io.Serializable} sub-classes to use in readObject only. + * Public to allow unit tests in another package to work around mock issues where the constructor is never invoked. + */ + @VisibleForTesting + public final void initializeTransientFieldsForLiveness() { + if (Liveness.REFERENCE_TRACKING_DISABLED) { + return; + } + tracker = new RetainedReferenceTracker<>(this, enforceStrongReachability); + if (Liveness.DEBUG_MODE_ENABLED) { + Liveness.log.info().append("LivenessDebug: Created tracker ").append(Utils.REFERENT_FORMATTER, tracker) + .append(" for ").append(Utils.REFERENT_FORMATTER, this).endl(); + } + } + + @Override + public final boolean tryManage(@NotNull final LivenessReferent referent) { + if (Liveness.REFERENCE_TRACKING_DISABLED) { + return true; + } + if (Liveness.DEBUG_MODE_ENABLED) { + Liveness.log.info().append("LivenessDebug: ").append(this).append(" managing ") + .append(referent.getReferentDescription()).endl(); + } + if (!referent.tryRetainReference()) { + return false; + } + tracker.addReference(referent); + return true; + } + + @Override + public final boolean tryUnmanage(@NotNull final LivenessReferent referent) { + if (Liveness.REFERENCE_TRACKING_DISABLED) { + return true; + } + tracker.dropReference(referent); + return true; + } + + @Override + public final boolean tryUnmanage(@NotNull final Stream referents) { + if (Liveness.REFERENCE_TRACKING_DISABLED) { + return true; + } + tracker.dropReferences(referents); + return true; + } + + @Override + public LogOutput append(LogOutput logOutput) { + return logOutput + .append("StandaloneLivenessManager{enforceStrongReachability=") + .append(enforceStrongReachability) + .append("}"); + } +} diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 2e2a53fb02e..034ff1c0cd7 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -985,7 +985,8 @@ public void testAutoRefreshingTable() throws ExecutionException, InterruptedExce (IcebergTestTable) resourceCatalog.loadTable(tableId)); final DataInstructionsProviderLoader dataInstructionsProvider = DataInstructionsProviderLoader.create(Map.of()); - final IcebergTableAdapter tableAdapter = new IcebergTableAdapter(tableId, icebergTable, dataInstructionsProvider); + final IcebergTableAdapter tableAdapter = + new IcebergTableAdapter(tableId, icebergTable, dataInstructionsProvider); final IcebergInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java index 80c0fe941a1..0c23615a909 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -363,7 +363,8 @@ public IcebergTable table( keyFinder = new IcebergFlatLayout(this, snapshot, userInstructions, dataInstructionsProviderLoader); } else { // Create the partitioning column location key finder - keyFinder = new IcebergKeyValuePartitionedLayout(this, snapshot, partitionSpec, userInstructions, dataInstructionsProviderLoader); + keyFinder = new IcebergKeyValuePartitionedLayout(this, snapshot, partitionSpec, userInstructions, + dataInstructionsProviderLoader); } if (instructions == null From 81e88d80d402618c3126e95d08709468cc267027 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 20 Sep 2024 08:27:07 -0700 Subject: [PATCH 35/56] Refreshing tests still need work. --- .../table/impl/PartitionAwareSourceTable.java | 10 +- .../locations/impl/AbstractTableLocation.java | 10 +- .../regioned/RegionedColumnSourceManager.java | 48 ++++--- .../liveness/DelegatingLivenessReferent.java | 36 ++++++ .../iceberg/util/IcebergToolsTest.java | 118 ++++++++++-------- .../iceberg/util/IcebergTableAdapter.java | 26 +++- 6 files changed, 165 insertions(+), 83 deletions(-) create mode 100644 engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessReferent.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java index c93c0d2f0c3..db77d22e252 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java @@ -242,11 +242,11 @@ protected final Collection> filterLocati partitionTableColumnSources.add(makePartitionSource(columnDefinition, immutableTableLocationKeys)); } // Add the tracked keys to the table - // TODO: figure out how to do this - // partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource( - // foundLocationKeys, - // LiveSupplier.class, - // null)); + // noinspection unchecked,rawtypes + partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource( + (Collection) (Collection) foundLocationKeys, + LiveSupplier.class, + null)); final Table filteredColumnPartitionTable = TableTools .newTable(foundLocationKeys.size(), partitionTableColumnNames, partitionTableColumnSources) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index 39b952fe7a4..bb4f5e4260a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java @@ -27,7 +27,7 @@ */ public abstract class AbstractTableLocation extends SubscriptionAggregator - implements TableLocation, DelegatingLivenessNode { + implements TableLocation, DelegatingLivenessReferent { private final ImmutableTableKey tableKey; private final ImmutableTableLocationKey tableLocationKey; @@ -36,7 +36,7 @@ public abstract class AbstractTableLocation private final KeyedObjectHashMap columnLocations = new KeyedObjectHashMap<>(StringUtils.charSequenceKey()); - private final ReferenceCountedLivenessNode livenessNode; + private final ReferenceCountedLivenessReferent livenessReferent; @SuppressWarnings("rawtypes") private static final AtomicReferenceFieldUpdater CACHED_DATA_INDEXES_UPDATER = @@ -62,7 +62,7 @@ protected AbstractTableLocation(@NotNull final TableKey tableKey, this.tableKey = Require.neqNull(tableKey, "tableKey").makeImmutable(); this.tableLocationKey = Require.neqNull(tableLocationKey, "tableLocationKey").makeImmutable(); - livenessNode = new ReferenceCountedLivenessNode(false) { + livenessReferent = new ReferenceCountedLivenessNode(false) { @Override protected void destroy() { AbstractTableLocation.this.destroy(); @@ -76,8 +76,8 @@ public final String toString() { } @Override - public LivenessNode asLivenessNode() { - return livenessNode; + public LivenessReferent asLivenessReferent() { + return livenessReferent; } // ------------------------------------------------------------------------------------------------------------------ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 45dde30cb03..266707e1144 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -5,8 +5,7 @@ import io.deephaven.base.verify.Assert; import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.liveness.LivenessScopeStack; -import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; +import io.deephaven.engine.liveness.*; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.*; @@ -38,10 +37,15 @@ /** * Manage column sources made up of regions in their own row key address space. */ -public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode implements ColumnSourceManager { +public class RegionedColumnSourceManager implements ColumnSourceManager, DelegatingLivenessNode { private static final Logger log = LoggerFactory.getLogger(RegionedColumnSourceManager.class); + /** + * Whether this column source manager is serving a refreshing dynamic table. + */ + private final LivenessNode livenessNode; + /** * Whether this column source manager is serving a refreshing dynamic table. */ @@ -104,6 +108,11 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im @ReferentialIntegrity private final Collection retainedDataIndexes = new ArrayList<>(); + /** + * List of tracked location keys to release at the end of the cycle. + */ + private final List releasedLocations = new ArrayList<>(); + /** * A reference to a delayed error notifier for the {@link #includedLocationsTable}, if one is pending. */ @@ -129,11 +138,6 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im */ private List invalidatedLocations = new ArrayList<>(); - /** - * List of tracked location keys to release at the end of the cycle. - */ - private List releasedLocations = new ArrayList<>(); - /** * Will invalidate the locations at the end of the cycle after all downstream updates are complete. */ @@ -151,7 +155,6 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im @NotNull final RegionedTableComponentFactory componentFactory, @NotNull final ColumnToCodecMappings codecMappings, @NotNull final List> columnDefinitions) { - super(false); this.isRefreshing = isRefreshing; this.columnDefinitions = columnDefinitions; @@ -177,6 +180,15 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im ? SIMPLE_LOCATION_TABLE_DEFINITION : TableDefinition.inferFrom(columnSourceMap); + if (isRefreshing) { + livenessNode = new LivenessArtifact() {}; + } else { + // This RCSM wil be managing table locations to prevent them from being de-scoped but will not otherwise + // participate in the liveness management process. + livenessNode = new ReferenceCountedLivenessNode(false) {}; + livenessNode.retainReference(); + } + try (final SafeCloseable ignored = isRefreshing ? LivenessScopeStack.open() : null) { includedLocationsTable = new QueryTable( locationTableDefinition, @@ -191,9 +203,8 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im } }; if (isRefreshing) { + livenessNode.manage(includedLocationsTable); rowSetModifiedColumnSet = includedLocationsTable.newModifiedColumnSet(ROWS_SET_COLUMN_NAME); - // TODO: managing doesn't work here because we are at zero right now. - manage(includedLocationsTable); } else { rowSetModifiedColumnSet = null; } @@ -211,12 +222,6 @@ public class RegionedColumnSourceManager extends ReferenceCountedLivenessNode im }); } - @Override - public void destroy() { - super.destroy(); - // TODO: do I need to explicitly release the managed locations or does it happen automatically? - } - @Override public synchronized void addLocation(@NotNull final TableLocation tableLocation) { final IncludedTableLocationEntry includedLocation = includedTableLocations.get(tableLocation.getKey()); @@ -227,7 +232,7 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation) log.debug().append("LOCATION_ADDED:").append(tableLocation.toString()).endl(); } // Hold on to this table location. - manage(tableLocation); + livenessNode.manage(tableLocation); emptyTableLocations.add(new EmptyTableLocationEntry(tableLocation)); } else { // Duplicate location - not allowed @@ -289,7 +294,7 @@ public synchronized TrackingWritableRowSet initialize() { new PartitioningColumnDataIndex<>(cd.getName(), columnSources.get(cd.getName()), this); retainedDataIndexes.add(partitioningIndex); if (isRefreshing) { - manage(partitioningIndex); + livenessNode.manage(partitioningIndex); } DataIndexer.of(initialRowSet).addDataIndex(partitioningIndex); } @@ -511,6 +516,11 @@ public final synchronized boolean isEmpty() { return sharedColumnSources; } + @Override + public LivenessNode asLivenessNode() { + return livenessNode; + } + /** * State keeper for a table location and its subscription buffer if it hasn't been found to have a non-null, * non-zero size yet. diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessReferent.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessReferent.java new file mode 100644 index 00000000000..ba2540fd1e8 --- /dev/null +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessReferent.java @@ -0,0 +1,36 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.liveness; + +import java.lang.ref.WeakReference; + +/** + * Indicates that this class implements LivenessReferent via a member rather than implementing it directly. The real + * LivenessReferent is exposed via {@link #asLivenessReferent()}, all other methods delegate to this instance. + */ +public interface DelegatingLivenessReferent extends LivenessReferent { + /** + * Returns the "real" {@link LivenessReferent} instance. When implementing this, care should be taken to match + * lifecycle of the {@code DelegatingLivenessReferent} instance with this instance, as the returned + * {@code LivenessReferent} behaves as a proxy for {@code this}. + * + * @return a LivenessReferent to use to manage this object's liveness. + */ + LivenessReferent asLivenessReferent(); + + @Override + default boolean tryRetainReference() { + return asLivenessReferent().tryRetainReference(); + } + + @Override + default void dropReference() { + asLivenessReferent().dropReference(); + } + + @Override + default WeakReference getWeakReference() { + return asLivenessReferent().getWeakReference(); + } +} diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 034ff1c0cd7..6d97ff70c39 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -12,6 +12,7 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.testutil.ControlledUpdateGraph; import io.deephaven.engine.testutil.junit4.EngineCleanup; +import io.deephaven.engine.testutil.locations.DependentRegistrar; import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergRefreshingTestTable; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; @@ -273,14 +274,14 @@ public void testOpenTableA() throws ExecutionException, InterruptedException, Ti io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_PARTITIONED_DEFINITION); // Test the string versions of the methods table = adapter.readTable("sales.sales_partitioned", instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_PARTITIONED_DEFINITION); } @@ -295,14 +296,14 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // This table ends up with zero records - Assert.eq(table.size(), "table.size()", 0, "rows in the table"); + Assert.eq(table.size(), "table.size()", 0, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); // Test the string versions of the methods table = adapter.readTable("sales.sales_multi", instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 0, "rows in the table"); + Assert.eq(table.size(), "table.size()", 0, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } @@ -317,14 +318,14 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_SINGLE_DEFINITION); // Test the string versions of the methods table = adapter.readTable("sales.sales_single", instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_SINGLE_DEFINITION); } @@ -339,7 +340,7 @@ public void testOpenTableS3Only() throws ExecutionException, InterruptedExceptio final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_PARTITIONED_DEFINITION); } @@ -359,7 +360,7 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_PARTITIONED_DEFINITION); } @@ -429,7 +430,7 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", renamed); } @@ -458,7 +459,7 @@ public void testSkippedPartitioningColumn() throws ExecutionException, Interrupt final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", tableDef); } @@ -487,7 +488,7 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", tableDef); } @@ -507,7 +508,7 @@ public void testZeroPartitioningColumns() throws ExecutionException, Interrupted final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } @@ -597,7 +598,7 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); } @Test @@ -615,7 +616,7 @@ public void testOpenTableColumnLegalization() throws ExecutionException, Interru final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_RENAMED_DEFINITION); } @@ -644,7 +645,7 @@ public void testOpenTableColumnLegalizationRename() ColumnDefinition.ofTime("Order_Date")); // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", expected); } @@ -724,32 +725,32 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept // Verify we retrieved all the rows. final io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0).snapshotId(), instructions); - Assert.eq(table0.size(), "table0.size()", 18073, "rows in the table"); + Assert.eq(table0.size(), "table0.size()", 18073, "expected rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions); - Assert.eq(table1.size(), "table1.size()", 54433, "rows in the table"); + Assert.eq(table1.size(), "table1.size()", 54433, "expected rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions); - Assert.eq(table2.size(), "table2.size()", 72551, "rows in the table"); + Assert.eq(table2.size(), "table2.size()", 72551, "expected rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions); - Assert.eq(table3.size(), "table3.size()", 100_000, "rows in the table"); + Assert.eq(table3.size(), "table3.size()", 100_000, "expected rows in the table"); Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table4 = adapter.readTable(tableId, snapshots.get(4).snapshotId(), instructions); - Assert.eq(table4.size(), "table4.size()", 100_000, "rows in the table"); + Assert.eq(table4.size(), "table4.size()", 100_000, "expected rows in the table"); Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table5 = adapter.readTable(tableId, snapshots.get(5).snapshotId(), instructions); - Assert.eq(table5.size(), "table5.size()", 0, "rows in the table"); + Assert.eq(table5.size(), "table5.size()", 0, "expected rows in the table"); Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @@ -765,54 +766,54 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx // Verify we retrieved all the rows. io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions); - Assert.eq(table0.size(), "table0.size()", 18073, "rows in the table"); + Assert.eq(table0.size(), "table0.size()", 18073, "expected rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1), instructions); - Assert.eq(table1.size(), "table1.size()", 54433, "rows in the table"); + Assert.eq(table1.size(), "table1.size()", 54433, "expected rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2), instructions); - Assert.eq(table2.size(), "table2.size()", 72551, "rows in the table"); + Assert.eq(table2.size(), "table2.size()", 72551, "expected rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3), instructions); - Assert.eq(table3.size(), "table3.size()", 100_000, "rows in the table"); + Assert.eq(table3.size(), "table3.size()", 100_000, "expected rows in the table"); Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table4 = adapter.readTable(tableId, snapshots.get(4), instructions); - Assert.eq(table4.size(), "table4.size()", 100_000, "rows in the table"); + Assert.eq(table4.size(), "table4.size()", 100_000, "expected rows in the table"); Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); io.deephaven.engine.table.Table table5 = adapter.readTable(tableId, snapshots.get(5), instructions); - Assert.eq(table5.size(), "table5.size()", 0, "rows in the table"); + Assert.eq(table5.size(), "table5.size()", 0, "expected rows in the table"); Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); // Test the string versions of the methods // Verify we retrieved all the rows. table0 = adapter.readTable("sales.sales_multi", snapshots.get(0).snapshotId(), instructions); - Assert.eq(table0.size(), "table0.size()", 18073, "rows in the table"); + Assert.eq(table0.size(), "table0.size()", 18073, "expected rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); table1 = adapter.readTable("sales.sales_multi", snapshots.get(1).snapshotId(), instructions); - Assert.eq(table1.size(), "table1.size()", 54433, "rows in the table"); + Assert.eq(table1.size(), "table1.size()", 54433, "expected rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); table2 = adapter.readTable("sales.sales_multi", snapshots.get(2).snapshotId(), instructions); - Assert.eq(table2.size(), "table2.size()", 72551, "rows in the table"); + Assert.eq(table2.size(), "table2.size()", 72551, "expected rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); table3 = adapter.readTable("sales.sales_multi", snapshots.get(3).snapshotId(), instructions); - Assert.eq(table3.size(), "table3.size()", 100_000, "rows in the table"); + Assert.eq(table3.size(), "table3.size()", 100_000, "expected rows in the table"); Assert.equals(table3.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); table4 = adapter.readTable("sales.sales_multi", snapshots.get(4).snapshotId(), instructions); - Assert.eq(table4.size(), "table4.size()", 100_000, "rows in the table"); + Assert.eq(table4.size(), "table4.size()", 100_000, "expected rows in the table"); Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); table5 = adapter.readTable("sales.sales_multi", snapshots.get(5).snapshotId(), instructions); - Assert.eq(table5.size(), "table5.size()", 0, "rows in the table"); + Assert.eq(table5.size(), "table5.size()", 0, "expected rows in the table"); Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @@ -827,7 +828,7 @@ public void testOpenAllTypesTable() throws ExecutionException, InterruptedExcept // Verify we retrieved all the rows. final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions).select(); - Assert.eq(table.size(), "table.size()", 10, "rows in the table"); + Assert.eq(table.size(), "table.size()", 10, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", ALL_TYPES_DEF); } @@ -867,25 +868,25 @@ public void testTableDefinitionTable() { // Use string and current snapshot Table tableDefTable = adapter.getTableDefinitionTable("sales.sales_multi", null); - Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "rows in the table"); + Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "expected rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); // Use TableIdentifier and Snapshot tableDefTable = adapter.getTableDefinitionTable(tableId, null); - Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "rows in the table"); + Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "expected rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); // Use string and long snapshot ID tableDefTable = adapter.getTableDefinitionTable("sales.sales_multi", snapshots.get(0).snapshotId(), null); - Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "rows in the table"); + Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "expected rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); // Use TableIdentifier and Snapshot tableDefTable = adapter.getTableDefinitionTable(tableId, snapshots.get(0), null); - Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "rows in the table"); + Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "expected rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); } @@ -945,32 +946,37 @@ public void testManualRefreshingTable() throws ExecutionException, InterruptedEx final List snapshots = tableAdapter.listSnapshots(); + // final CapturingUpdateGraph updateGraph = new + // CapturingUpdateGraph(ExecutionContext.getContext().getUpdateGraph().cast()); final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + final DependentRegistrar registrar = new DependentRegistrar(); + final IcebergTable table = tableAdapter.table(snapshots.get(0).snapshotId(), localInstructions); - Assert.eq(table.size(), "table.size()", 18073, "rows in the table"); + Assert.eq(table.size(), "table.size()", 18073, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); updateGraph.runWithinUnitTestCycle(() -> { table.update(snapshots.get(1).snapshotId()); + registrar.run(); }); - Assert.eq(table.size(), "table.size()", 54433, "rows in the table"); + Assert.eq(table.size(), "table.size()", 54433, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(2).snapshotId()); - Assert.eq(table.size(), "table.size()", 72551, "rows in the table"); + Assert.eq(table.size(), "table.size()", 72551, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(3).snapshotId()); - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(4).snapshotId()); - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(5).snapshotId()); - Assert.eq(table.size(), "table.size()", 0, "rows in the table"); + Assert.eq(table.size(), "table.size()", 0, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } @@ -998,30 +1004,38 @@ public void testAutoRefreshingTable() throws ExecutionException, InterruptedExce final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); - final IcebergTable table = tableAdapter.table(snapshots.get(0).snapshotId(), localInstructions); - Assert.eq(table.size(), "table.size()", 18073, "rows in the table"); + final IcebergTable table = tableAdapter.table(localInstructions); + Assert.eq(table.size(), "table.size()", 18073, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); icebergTable.advanceSequenceNumber(); + Thread.sleep(500); updateGraph.runWithinUnitTestCycle(() -> { + Assert.eq(table.size(), "table.size()", 54433, "expected rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); + }); + + icebergTable.advanceSequenceNumber(); + Thread.sleep(500); + updateGraph.runWithinUnitTestCycle(() -> { + Assert.eq(table.size(), "table.size()", 72551, "expected rows in the table"); + Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); }); - Assert.eq(table.size(), "table.size()", 54433, "rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(2).snapshotId()); - Assert.eq(table.size(), "table.size()", 72551, "rows in the table"); + Assert.eq(table.size(), "table.size()", 72551, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(3).snapshotId()); - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(4).snapshotId()); - Assert.eq(table.size(), "table.size()", 100_000, "rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(5).snapshotId()); - Assert.eq(table.size(), "table.size()", 0, "rows in the table"); + Assert.eq(table.size(), "table.size()", 0, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java index 0c23615a909..87de7b07cd7 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -297,17 +297,40 @@ public Table definitionTable( return TableTools.metaTable(definition); } + /** + * Read the latest snapshot of an Iceberg table from the Iceberg catalog as a Deephaven {@link Table table}. + * + * @return The loaded table + */ + public IcebergTable table() { + return table(null, null); + } + /** * Read the latest snapshot of an Iceberg table from the Iceberg catalog as a Deephaven {@link Table table}. * * @param instructions The instructions for customizations while reading (or null for default instructions) * @return The loaded table */ - @SuppressWarnings("unused") public IcebergTable table(@Nullable final IcebergInstructions instructions) { return table(null, instructions); } + /** + * Read a snapshot of an Iceberg table from the Iceberg catalog. + * + * @param tableSnapshotId The snapshot id to load + * @return The loaded table + */ + public IcebergTable table(final long tableSnapshotId) { + // Find the snapshot with the given snapshot id + final Snapshot tableSnapshot = + snapshot(tableSnapshotId).orElseThrow(() -> new IllegalArgumentException( + "Snapshot with id " + tableSnapshotId + " not found for table " + tableIdentifier)); + + return table(tableSnapshot, null); + } + /** * Read a snapshot of an Iceberg table from the Iceberg catalog. * @@ -315,7 +338,6 @@ public IcebergTable table(@Nullable final IcebergInstructions instructions) { * @param instructions The instructions for customizations while reading (or null for default instructions) * @return The loaded table */ - @SuppressWarnings("unused") public IcebergTable table(final long tableSnapshotId, @Nullable final IcebergInstructions instructions) { // Find the snapshot with the given snapshot id final Snapshot tableSnapshot = From b37a04ef95ba4e2555f63dd3907cd36ef62af4e6 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 24 Sep 2024 20:13:38 -0700 Subject: [PATCH 36/56] Better tests and improved liveness management for the TableLocation keys and providers. --- .../table/impl/SourcePartitionedTable.java | 61 ++- .../impl/locations/TableLocationProvider.java | 13 + .../locations/impl/AbstractTableLocation.java | 4 +- .../impl/AbstractTableLocationProvider.java | 8 +- .../regioned/RegionedColumnSourceManager.java | 20 +- .../table/impl/AbstractTableLocationTest.java | 347 ++++++++++++++++++ .../impl/SourcePartitionedTableTest.java | 96 ++++- .../impl/TestPartitionAwareSourceTable.java | 39 +- .../liveness/DelegatingLivenessNode.java | 3 +- .../liveness/DelegatingLivenessReferent.java | 3 +- .../iceberg/util/IcebergToolsTest.java | 81 +--- .../IcebergRefreshingTestTable.java | 4 + 12 files changed, 570 insertions(+), 109 deletions(-) create mode 100644 engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index ae971702514..3bcd3376426 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -4,7 +4,7 @@ package io.deephaven.engine.table.impl; import io.deephaven.base.verify.Assert; -import io.deephaven.engine.liveness.LiveSupplier; +import io.deephaven.engine.liveness.*; import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; @@ -38,6 +38,26 @@ public class SourcePartitionedTable extends PartitionedTableImpl { private static final String KEY_COLUMN_NAME = "TableLocationKey"; private static final String CONSTITUENT_COLUMN_NAME = "LocationTable"; + /** + * Private constructor for a {@link SourcePartitionedTable}. + * + * @param table the locations table to use for this {@link SourcePartitionedTable} + * @param constituentDefinition The {@link TableDefinition} expected of constituent {@link Table tables} + * @param refreshLocations Whether the set of locations should be refreshed + */ + private SourcePartitionedTable( + @NotNull final Table table, + @NotNull final TableDefinition constituentDefinition, + final boolean refreshLocations) { + super(table, + Set.of(KEY_COLUMN_NAME), + true, + CONSTITUENT_COLUMN_NAME, + constituentDefinition, + refreshLocations, + false); + } + /** * Construct a {@link SourcePartitionedTable} from the supplied parameters. *

    @@ -52,29 +72,31 @@ public class SourcePartitionedTable extends PartitionedTableImpl { * @param refreshSizes Whether the locations found should be refreshed * @param locationKeyMatcher Function to filter desired location keys */ - public SourcePartitionedTable( + public static SourcePartitionedTable create( @NotNull final TableDefinition constituentDefinition, @NotNull final UnaryOperator applyTablePermissions, @NotNull final TableLocationProvider tableLocationProvider, final boolean refreshLocations, final boolean refreshSizes, @NotNull final Predicate locationKeyMatcher) { - super(new UnderlyingTableMaintainer( - constituentDefinition, + final UnderlyingTableMaintainer maintainer = new UnderlyingTableMaintainer(constituentDefinition, applyTablePermissions, tableLocationProvider, refreshLocations, refreshSizes, - locationKeyMatcher).result(), - Set.of(KEY_COLUMN_NAME), - true, - CONSTITUENT_COLUMN_NAME, + locationKeyMatcher); + + final SourcePartitionedTable sourcePartitionedTable = new SourcePartitionedTable( + maintainer.result(), constituentDefinition, - refreshLocations, - false); + refreshLocations); + + + maintainer.assignLivenessManager(sourcePartitionedTable); + return sourcePartitionedTable; } - private static final class UnderlyingTableMaintainer { + private static final class UnderlyingTableMaintainer extends ReferenceCountedLivenessNode { private final TableDefinition constituentDefinition; private final UnaryOperator
    applyTablePermissions; @@ -104,13 +126,17 @@ private UnderlyingTableMaintainer( final boolean refreshLocations, final boolean refreshSizes, @NotNull final Predicate locationKeyMatcher) { + super(false); + + // Increase the refcount of this liveness node to allow it to manage the subscription buffer. + retainReference(); + this.constituentDefinition = constituentDefinition; this.applyTablePermissions = applyTablePermissions; this.tableLocationProvider = tableLocationProvider; this.refreshSizes = refreshSizes; this.locationKeyMatcher = locationKeyMatcher; - // noinspection resource resultRows = RowSetFactory.empty().toTracking(); resultTableLocationKeys = ArrayBackedColumnSource.getMemoryColumnSource(TableLocationKey.class, null); resultLocationTables = ArrayBackedColumnSource.getMemoryColumnSource(Table.class, null); @@ -131,6 +157,7 @@ private UnderlyingTableMaintainer( if (needToRefreshLocations) { subscriptionBuffer = new TableLocationSubscriptionBuffer(tableLocationProvider); + manage(subscriptionBuffer); pendingLocationStates = new IntrusiveDoublyLinkedQueue<>( IntrusiveDoublyLinkedNode.Adapter.getInstance()); readyLocationStates = new IntrusiveDoublyLinkedQueue<>( @@ -155,7 +182,6 @@ protected void instrumentedRefresh() { removedConstituents.forEach(result::unmanage); removedConstituents = null; }); - processPendingLocations(false); } else { subscriptionBuffer = null; @@ -179,6 +205,15 @@ protected void instrumentedRefresh() { } } + /** + * If we have a {@link TableLocationSubscriptionBuffer}, manage it with the given liveness manager. + */ + private void assignLivenessManager(final LivenessManager manager) { + manager.manage(this); + dropReference(); + } + + private QueryTable result() { return result; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index 4314b092659..d1ca8624f49 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -8,7 +8,9 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.ArrayList; import java.util.Collection; +import java.util.List; import java.util.function.Consumer; import java.util.function.Predicate; @@ -128,6 +130,17 @@ default void handleTableLocationKeysUpdate( */ TableLocationProvider ensureInitialized(); + /** + * Get this provider's currently known location keys. The locations specified by the keys returned may have null + * size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is + * guaranteed to succeed for all results. + */ + default Collection getTableLocationKeys() { + final List keys = new ArrayList<>(); + getTableLocationKeys(trackedKey -> keys.add(trackedKey.get())); + return keys; + } + /** * Get this provider's currently known location keys. The locations specified by the keys returned may have null * size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index bb4f5e4260a..75eb8be3616 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java @@ -62,7 +62,7 @@ protected AbstractTableLocation(@NotNull final TableKey tableKey, this.tableKey = Require.neqNull(tableKey, "tableKey").makeImmutable(); this.tableLocationKey = Require.neqNull(tableLocationKey, "tableLocationKey").makeImmutable(); - livenessReferent = new ReferenceCountedLivenessNode(false) { + livenessReferent = new ReferenceCountedLivenessReferent() { @Override protected void destroy() { AbstractTableLocation.this.destroy(); @@ -251,7 +251,7 @@ public final BasicDataIndex getDataIndex(@NotNull final String... columns) { /** * The reference count has reached zero or we are being GC'd, we can clear this location and release any resources. */ - private void destroy() { + protected void destroy() { handleUpdate(null, System.currentTimeMillis()); clearColumnLocations(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 71d2438df90..53d4c080c60 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -279,11 +279,11 @@ protected void endTransaction(@NotNull final Object token) { true)) { onEmpty(); } - // Release the keys that were removed after we have delivered the notifications and the - // subscribers have had a chance to process them - removedKeys.forEach(livenessManager::unmanage); } } + // Release the keys that were removed after we have delivered the notifications and the + // subscribers have had a chance to process them + removedKeys.forEach(livenessManager::unmanage); } /** @@ -373,7 +373,7 @@ protected void handleTableLocationKeyRemoved( // If we're not in a transaction, we should push this key immediately. synchronized (subscriptions) { - final TrackedKeySupplier trackedKey = tableLocationKeyMap.removeKey(locationKey); + final TrackedKeySupplier trackedKey = tableLocationKeyMap.get(locationKey); if (trackedKey != null) { trackedKey.deactivate(); if (subscriptions.deliverNotification( diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 266707e1144..624d02ef820 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -212,14 +212,18 @@ public class RegionedColumnSourceManager implements ColumnSourceManager, Delegat invalidateCommitter = new UpdateCommitter<>(this, ExecutionContext.getContext().getUpdateGraph(), - (instance) -> { - synchronized (instance) { - invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate); - invalidatedLocations.clear(); - releasedLocations.forEach(instance::unmanage); - releasedLocations.clear(); - } - }); + RegionedColumnSourceManager::invalidateAndRelease); + } + + /** + * Activated by the invalidateCommitter to invalidate populated locations and to release all the managed locations + * that we no longer care about. + */ + private synchronized void invalidateAndRelease() { + invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate); + invalidatedLocations.clear(); + releasedLocations.forEach(this::unmanage); + releasedLocations.clear(); } @Override diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java new file mode 100644 index 00000000000..d297ec7b34d --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java @@ -0,0 +1,347 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl; + +import io.deephaven.api.SortColumn; +import io.deephaven.base.verify.Assert; +import io.deephaven.configuration.Configuration; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LiveSupplier; +import io.deephaven.engine.liveness.StandaloneLivenessManager; +import io.deephaven.engine.table.BasicDataIndex; +import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; +import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; +import io.deephaven.engine.table.impl.locations.local.URITableLocationKey; +import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; +import io.deephaven.io.logger.StreamLoggerImpl; +import io.deephaven.test.types.OutOfBandTest; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.process.ProcessEnvironment; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.net.URI; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +@Category(OutOfBandTest.class) +public class AbstractTableLocationTest extends RefreshingTableTestCase { + private CapturingUpdateGraph updateGraph; + private SafeCloseable contextCloseable; + + private static class TestTableLocationProvider extends AbstractTableLocationProvider { + + public TestTableLocationProvider(final boolean supportsSubscriptions) { + super(supportsSubscriptions); + } + + @Override + protected @NotNull TableLocation makeTableLocation(@NotNull TableLocationKey locationKey) { + return new TestTableLocation(getKey(), locationKey, false); + } + + @Override + public void refresh() { + Assert.statementNeverExecuted(); + } + + public void addKey(@NotNull TableLocationKey locationKey) { + handleTableLocationKeyAdded(locationKey, null); + } + + public void removeKey(@NotNull TableLocationKey locationKey) { + handleTableLocationKeyRemoved(locationKey, null); + } + + + public void beginTransaction(@NotNull Object token) { + super.beginTransaction(token); + } + + public void endTransaction(@NotNull Object token) { + super.endTransaction(token); + } + + public void addKey(@NotNull TableLocationKey locationKey, @NotNull Object token) { + handleTableLocationKeyAdded(locationKey, token); + } + + public void removeKey(@NotNull TableLocationKey locationKey, @NotNull Object token) { + handleTableLocationKeyRemoved(locationKey, token); + } + } + + private static class TestTableLocation extends AbstractTableLocation { + private boolean destroyed = false; + + protected TestTableLocation(@NotNull TableKey tableKey, @NotNull TableLocationKey tableLocationKey, + boolean supportsSubscriptions) { + super(tableKey, tableLocationKey, supportsSubscriptions); + } + + public boolean isDestroyed() { + return destroyed; + } + + @Override + protected void destroy() { + super.destroy(); + destroyed = true; + } + + @Override + protected @NotNull ColumnLocation makeColumnLocation(@NotNull String name) { + return null; + } + + @Override + public @Nullable BasicDataIndex loadDataIndex(@NotNull String... columns) { + return null; + } + + @Override + public void refresh() { + Assert.statementNeverExecuted(); + } + + @Override + public @NotNull List getSortedColumns() { + return List.of(); + } + + @Override + public @NotNull List getDataIndexColumns() { + return List.of(); + } + + @Override + public boolean hasDataIndex(@NotNull String... columns) { + return false; + } + } + + @Override + public void setUp() throws Exception { + if (null == ProcessEnvironment.tryGet()) { + ProcessEnvironment.basicServerInitialization(Configuration.getInstance(), + "SourcePartitionedTableTest", new StreamLoggerImpl()); + } + super.setUp(); + setExpectError(false); + + updateGraph = new CapturingUpdateGraph(ExecutionContext.getContext().getUpdateGraph().cast()); + contextCloseable = updateGraph.getContext().open(); + } + + @Override + public void tearDown() throws Exception { + contextCloseable.close(); + super.tearDown(); + } + + private List createKeys(final int count) { + final List keys = new ArrayList<>(); + for (int i = 0; i < count; i++) { + keys.add(new URITableLocationKey(URI.create("file:///tmp/" + i), i, null)); + } + return keys; + } + + /** + * Test the management and release of the liveness of table location keys is correct. + */ + @Test + public void testTableLocationKeyManagement() { + + // Create a test table location provider + final TestTableLocationProvider provider = new TestTableLocationProvider(false); + + // Create a set of table location keys + final List initialKeys = createKeys(5); + + // Add the keys to the table location provider + for (final TableLocationKey locationKey : initialKeys) { + provider.addKey(locationKey); + } + + // Verify all the initial keys are present for new listeners + Set keys = new HashSet<>(provider.getTableLocationKeys()); + for (final TableLocationKey tlk : initialKeys) { + Assert.eqTrue(keys.contains(tlk), "keys.contains(tlk)"); + } + + // Create a local liveness manager + final StandaloneLivenessManager manager = new StandaloneLivenessManager(false); + + // Get a list of the LiveSupplier<> keys + final List> initialTrackedKeys = new ArrayList<>(); + provider.getTableLocationKeys(initialTrackedKeys::add); + + // Resolve and create all the table locations + final List tableLocations = new ArrayList<>(); + for (final TableLocationKey key : initialKeys) { + final TestTableLocation tl = (TestTableLocation) provider.getTableLocation(key); + tableLocations.add(tl); + } + + // Externally manage keys 2 & 3 (simulate a TLSB) + manager.manage(initialTrackedKeys.get(2)); + manager.manage(initialTrackedKeys.get(3)); + + // Also manage the table locations for key 3 (simulate a filtered RCSM) + manager.manage(tableLocations.get(3)); + + // Drop the first 4 keys from the provider + for (int i = 0; i < 4; i++) { + final TableLocationKey removedKey = initialKeys.get(i); + provider.removeKey(removedKey); + } + + // Verify only the last key is present for new listeners + keys = new HashSet<>(provider.getTableLocationKeys()); + Assert.eqTrue(keys.contains(initialKeys.get(4)), "keys.contains(initialKeys.get(4))"); + Assert.eq(keys.size(), "keys.size()", 1); + + // Verify that we CAN'T retrieve the unmanaged locations from the provider (they were dropped) + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(0)), + "provider.hasTableLocationKey(initialKeys.get(0))"); + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(1)), + "provider.hasTableLocationKey(initialKeys.get(1))"); + + // Verify the tableLocations for the unmanaged keys are destroyed + Assert.eqTrue(tableLocations.get(0).isDestroyed(), "tableLocations.get(0).isDestroyed()"); + Assert.eqTrue(tableLocations.get(1).isDestroyed(), "tableLocations.get(1).isDestroyed()"); + + // Verify that we CAN retrieve the managed locations from the provider (they are still live) + Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(2)), + "provider.hasTableLocationKey(initialKeys.get(2))"); + Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(3)), + "provider.hasTableLocationKey(initialKeys.get(3))"); + + // Verify the tableLocations for the managed keys are NOT destroyed + Assert.eqFalse(tableLocations.get(2).isDestroyed(), "tableLocations.get(2).isDestroyed()"); + Assert.eqFalse(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); + + // Un-manage the two keys + manager.unmanage(initialTrackedKeys.get(2)); + manager.unmanage(initialTrackedKeys.get(3)); + + // location #2 should be destroyed, location #3 should not (because an RCSM is managing it) + Assert.eqTrue(tableLocations.get(2).isDestroyed(), "tableLocations.get(2).isDestroyed()"); + Assert.eqFalse(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); + + // Verify that we CAN'T retrieve the (now) unmanaged locations from the provider (they were dropped) + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(2)), + "provider.hasTableLocationKey(initialKeys.get(2))"); + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(3)), + "provider.hasTableLocationKey(initialKeys.get(3))"); + + // Release the table location being held by the RCSM + manager.unmanage(tableLocations.get(3)); + Assert.eqTrue(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); + } + + /** + * Test the management and release of the liveness of table location keys is correct when using transactions. + */ + @Test + public void testTableLocationKeyTransactionManagement() { + + // Create a test table location provider + final TestTableLocationProvider provider = new TestTableLocationProvider(true); + + // Create a set of table location keys + final List initialKeys = createKeys(5); + + // Add the keys to the table location provider + provider.beginTransaction(provider); + for (final TableLocationKey locationKey : initialKeys) { + provider.addKey(locationKey, provider); + } + provider.endTransaction(provider); + + // Verify all the initial keys are present for new listeners + Set keys = new HashSet<>(provider.getTableLocationKeys()); + for (final TableLocationKey tlk : initialKeys) { + Assert.eqTrue(keys.contains(tlk), "keys.contains(tlk)"); + } + + // Create a local liveness manager + final StandaloneLivenessManager manager = new StandaloneLivenessManager(false); + + // Get a list of the LiveSupplier<> keys + final List> initialTrackedKeys = new ArrayList<>(); + provider.getTableLocationKeys(initialTrackedKeys::add); + + // Resolve and create all the table locations + final List tableLocations = new ArrayList<>(); + for (final TableLocationKey key : initialKeys) { + final TestTableLocation tl = (TestTableLocation) provider.getTableLocation(key); + tableLocations.add(tl); + } + + // Externally manage keys 2 & 3 (simulate a TLSB) + manager.manage(initialTrackedKeys.get(2)); + manager.manage(initialTrackedKeys.get(3)); + + // Also manage the table locations for key 3 (simulate a filtered RCSM) + manager.manage(tableLocations.get(3)); + + // Drop the first 4 keys from the provider + provider.beginTransaction(provider); + for (int i = 0; i < 4; i++) { + final TableLocationKey removedKey = initialKeys.get(i); + provider.removeKey(removedKey, provider); + } + provider.endTransaction(provider); + + // Verify only the last key is present for new listeners + keys = new HashSet<>(provider.getTableLocationKeys()); + Assert.eqTrue(keys.contains(initialKeys.get(4)), "keys.contains(initialKeys.get(4))"); + Assert.eq(keys.size(), "keys.size()", 1); + + // Verify that we CAN'T retrieve the unmanaged locations from the provider (they were dropped) + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(0)), + "provider.hasTableLocationKey(initialKeys.get(0))"); + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(1)), + "provider.hasTableLocationKey(initialKeys.get(1))"); + + // Verify the tableLocations for the unmanaged keys are destroyed + Assert.eqTrue(tableLocations.get(0).isDestroyed(), "tableLocations.get(0).isDestroyed()"); + Assert.eqTrue(tableLocations.get(1).isDestroyed(), "tableLocations.get(1).isDestroyed()"); + + // Verify that we CAN retrieve the managed locations from the provider (they are still live) + Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(2)), + "provider.hasTableLocationKey(initialKeys.get(2))"); + Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(3)), + "provider.hasTableLocationKey(initialKeys.get(3))"); + + // Verify the tableLocations for the managed keys are NOT destroyed + Assert.eqFalse(tableLocations.get(2).isDestroyed(), "tableLocations.get(2).isDestroyed()"); + Assert.eqFalse(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); + + // Un-manage the two keys + manager.unmanage(initialTrackedKeys.get(2)); + manager.unmanage(initialTrackedKeys.get(3)); + + // location #2 should be destroyed, location #3 should not (because an RCSM is managing it) + Assert.eqTrue(tableLocations.get(2).isDestroyed(), "tableLocations.get(2).isDestroyed()"); + Assert.eqFalse(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); + + // Verify that we CAN'T retrieve the (now) unmanaged locations from the provider (they were dropped) + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(2)), + "provider.hasTableLocationKey(initialKeys.get(2))"); + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(3)), + "provider.hasTableLocationKey(initialKeys.get(3))"); + + // Release the table location being held by the RCSM and verify its destruction. + manager.unmanage(tableLocations.get(3)); + Assert.eqTrue(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java index ee43dbc5ba2..d45dae5bd7f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java @@ -6,11 +6,14 @@ import io.deephaven.base.verify.AssertionFailure; import io.deephaven.configuration.Configuration; import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LivenessScope; +import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.InvalidatedRegionException; +import io.deephaven.engine.table.impl.locations.TableLocation; import io.deephaven.engine.table.impl.locations.TableLocationRemovedException; import io.deephaven.engine.testutil.locations.DependentRegistrar; import io.deephaven.engine.testutil.locations.TableBackedTableLocationKey; @@ -25,6 +28,9 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import java.util.HashSet; +import java.util.Set; + import static io.deephaven.engine.testutil.TstUtils.*; import static io.deephaven.engine.util.TableTools.*; @@ -92,7 +98,7 @@ private SourcePartitionedTable setUpData() { true, p1, p2); - return new SourcePartitionedTable(p1.getDefinition(), + return SourcePartitionedTable.create(p1.getDefinition(), t -> t, tlp, true, @@ -117,6 +123,14 @@ public void testAddAndRemoveLocations() { tlp.removeTableLocationKey(tlks[0]); tlp.refresh(); + // We've removed location 0, should be gone from the location key list + Set activeSet = new HashSet<>(tlp.getTableLocationKeys()); + assertFalse(activeSet.contains(tlks[0])); + + // Since we haven't been through an update cycle, we should still be able to retrieve the location for this key. + assertTrue(tlp.hasTableLocationKey(tlks[0])); + assertNotNull(tlp.getTableLocation(tlks[0])); + allowingError(() -> updateGraph.getDelegate().runWithinUnitTestCycle(() -> { updateGraph.refreshSources(); updateGraph.markSourcesRefreshedForUnitTests(); @@ -130,8 +144,12 @@ public void testAddAndRemoveLocations() { assertTableEquals(tableIt.next(), p2); } + // After the update cycle, the update committer should have triggered cleanup of this key and location. + assertFalse(tlp.hasTableLocationKey(tlks[0])); + tlp.addPending(p3); tlp.refresh(); + updateGraph.getDelegate().runWithinUnitTestCycle(() -> { updateGraph.refreshSources(); registrar.run(); @@ -152,16 +170,84 @@ public void testAddAndRemoveLocations() { updateGraph.refreshSources(); updateGraph.markSourcesRefreshedForUnitTests(); registrar.run(); + + // flush the notifications and verify state + updateGraph.getDelegate().flushAllNormalNotificationsForUnitTests(); + assertEquals(2, partitionTable.size()); + try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { + assertTableEquals(tableIt.next(), p3); + assertTableEquals(tableIt.next(), p4); + } }, false), errors -> errors.size() == 1 && FindExceptionCause.isOrCausedBy(errors.get(0), TableLocationRemovedException.class).isPresent()); getUpdateErrors().clear(); - assertEquals(2, partitionTable.size()); - try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { - assertTableEquals(tableIt.next(), p3); - assertTableEquals(tableIt.next(), p4); + /** + * Set up a complicated table location management test, where we create a new table under a scope, add it to the + * SPT. Then drop the table and verify that the table location is destroyed only after the scope is released. + */ + final TableLocation location5; + try (final SafeCloseable ignored = LivenessScopeStack.open(new LivenessScope(), true)) { + final QueryTable p5 = testRefreshingTable(i(0, 1, 2, 3).toTracking(), + stringCol("Sym", "gg", "hh", "gg", "hh"), + intCol("intCol", 10000, 20000, 40000, 60000), + doubleCol("doubleCol", 0.1, 0.2, 0.4, 0.6)); + p5.setAttribute(Table.APPEND_ONLY_TABLE_ATTRIBUTE, true); + tlp.addPending(p5); + tlp.refresh(); + + updateGraph.getDelegate().runWithinUnitTestCycle(() -> { + updateGraph.refreshSources(); + registrar.run(); + }); + + assertEquals(3, partitionTable.size()); + try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { + assertTableEquals(tableIt.next(), p3); + assertTableEquals(tableIt.next(), p4); + assertTableEquals(tableIt.next(), p5); + } + + tlks = tlp.getTableLocationKeys().stream().sorted().toArray(ImmutableTableLocationKey[]::new); + final ImmutableTableLocationKey tlk_p5 = tlks[2]; + location5 = tlp.getTableLocation(tlk_p5); + assertTrue(location5.getRowSet() != null && location5.getRowSet().size() == 4); + + // remove the p5 key from the SPT + tlp.removeTableLocationKey(tlk_p5); + tlp.refresh(); + + // We've removed location 5, should be gone from the location key list + activeSet = new HashSet<>(tlp.getTableLocationKeys()); + assertFalse(activeSet.contains(tlk_p5)); + + // Since we haven't been through an update cycle, we can still retrieve the location for this key. + assertTrue(tlp.hasTableLocationKey(tlk_p5)); + assertNotNull(tlp.getTableLocation(tlk_p5)); + + updateGraph.getDelegate().runWithinUnitTestCycle(() -> { + updateGraph.refreshSources(); + registrar.run(); + + }); + + // After the cycle cleanup, this location should not be available + assertFalse(tlp.hasTableLocationKey(tlk_p5)); + + // The location associated with p5 should still be valid, because it is held by p5 RCSM and p5 is in scope + assertTrue(location5.getRowSet() != null && location5.getRowSet().size() == 4); + + assertEquals(2, partitionTable.size()); + try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { + assertTableEquals(tableIt.next(), p3); + assertTableEquals(tableIt.next(), p4); + } } + // The scope has been released, p5 should be dead so verify that the location associated with p5 has been + // cleaned up + assertNull(location5.getRowSet()); + // Prove that we propagate normal errors. This is a little tricky, we can't test for errors.size == 1 because // The TableBackedTableLocation has a copy() of the p3 table which is itself a leaf. Erroring P3 will // cause one error to come from the copied table, and one from the merged() table. We just need to validate diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java index 6bed409a747..af6c13fb0b5 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java @@ -6,6 +6,8 @@ import io.deephaven.base.Pair; import io.deephaven.base.verify.Assert; import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LiveSupplier; +import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.perf.PerformanceEntry; @@ -46,6 +48,37 @@ @SuppressWarnings({"AutoBoxing", "JUnit4AnnotatedMethodInJUnit3TestCase", "AnonymousInnerClassMayBeStatic"}) public class TestPartitionAwareSourceTable extends RefreshingTableTestCase { + private static class TestKeySupplier extends ReferenceCountedLivenessNode + implements LiveSupplier { + + private final ImmutableTableLocationKey key; + + private TableLocation tableLocation; + + TestKeySupplier( + final ImmutableTableLocationKey key) { + super(false); + this.key = key; + } + + @Override + public ImmutableTableLocationKey get() { + return key; + } + + public synchronized void setTableLocation(final TableLocation tableLocation) { + Assert.eqNull(this.tableLocation, "this.tableLocation"); + manage(tableLocation); + this.tableLocation = tableLocation; + } + + @Override + protected synchronized void destroy() { + super.destroy(); + tableLocation = null; + } + } + private static final int NUM_COLUMNS = 5; private static final ColumnDefinition PARTITIONING_COLUMN_DEFINITION = ColumnDefinition.ofString("Date").withPartitioning(); @@ -247,7 +280,8 @@ private void doInitializeCheck(final ImmutableTableLocationKey[] tableLocationKe @Override public Object invoke(Invocation invocation) { subscriptionBuffer = (TableLocationSubscriptionBuffer) invocation.getParameter(0); - Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded); + Arrays.stream(tableLocationKeys).map(TestKeySupplier::new) + .forEach(subscriptionBuffer::handleTableLocationKeyAdded); return null; } }); @@ -407,7 +441,8 @@ public Object invoke(Invocation invocation) { private void doAddLocationsRefreshCheck(final ImmutableTableLocationKey[] tableLocationKeys, final Set expectPassFilters) { - Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded); + Arrays.stream(tableLocationKeys).map(TestKeySupplier::new) + .forEach(subscriptionBuffer::handleTableLocationKeyAdded); expectPassFilters.forEach(tl -> checking(new Expectations() { { diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessNode.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessNode.java index c4173d8034b..5c539405edf 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessNode.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessNode.java @@ -49,6 +49,7 @@ default void dropReference() { @Override default WeakReference getWeakReference() { - return asLivenessNode().getWeakReference(); + // Must return a WeakReference to the DelegatingLivenessNode, not the underlying LivenessNode + return new WeakReference<>(this); } } diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessReferent.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessReferent.java index ba2540fd1e8..71a861f74be 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessReferent.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessReferent.java @@ -31,6 +31,7 @@ default void dropReference() { @Override default WeakReference getWeakReference() { - return asLivenessReferent().getWeakReference(); + // Must return a WeakReference to the DelegatingLivenessReferent, not the underlying LivenessReferent + return new WeakReference<>(this); } } diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 6d97ff70c39..2484a542010 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -12,7 +12,6 @@ import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.testutil.ControlledUpdateGraph; import io.deephaven.engine.testutil.junit4.EngineCleanup; -import io.deephaven.engine.testutil.locations.DependentRegistrar; import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergRefreshingTestTable; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; @@ -946,96 +945,32 @@ public void testManualRefreshingTable() throws ExecutionException, InterruptedEx final List snapshots = tableAdapter.listSnapshots(); - // final CapturingUpdateGraph updateGraph = new - // CapturingUpdateGraph(ExecutionContext.getContext().getUpdateGraph().cast()); final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); - final DependentRegistrar registrar = new DependentRegistrar(); + final IcebergTableImpl table = (IcebergTableImpl) tableAdapter.table(snapshots.get(0), localInstructions); - final IcebergTable table = tableAdapter.table(snapshots.get(0).snapshotId(), localInstructions); + // Initial size Assert.eq(table.size(), "table.size()", 18073, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - updateGraph.runWithinUnitTestCycle(() -> { - table.update(snapshots.get(1).snapshotId()); - registrar.run(); - }); + table.update(snapshots.get(1).snapshotId()); + updateGraph.runWithinUnitTestCycle(table::refresh); Assert.eq(table.size(), "table.size()", 54433, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(2).snapshotId()); + updateGraph.runWithinUnitTestCycle(table::refresh); Assert.eq(table.size(), "table.size()", 72551, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(3).snapshotId()); + updateGraph.runWithinUnitTestCycle(table::refresh); Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(4).snapshotId()); + updateGraph.runWithinUnitTestCycle(table::refresh); Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); table.update(snapshots.get(5).snapshotId()); + updateGraph.runWithinUnitTestCycle(table::refresh); Assert.eq(table.size(), "table.size()", 0, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - } - - @Test - public void testAutoRefreshingTable() throws ExecutionException, InterruptedException, TimeoutException { - uploadSalesMulti(); - - final TableIdentifier tableId = TableIdentifier.parse("sales.sales_multi"); - - // Create a custom table adapter on top of the refreshing Iceberg table. - final IcebergRefreshingTestTable icebergTable = IcebergRefreshingTestTable.fromTestTable( - (IcebergTestTable) resourceCatalog.loadTable(tableId)); - final DataInstructionsProviderLoader dataInstructionsProvider = - DataInstructionsProviderLoader.create(Map.of()); - final IcebergTableAdapter tableAdapter = - new IcebergTableAdapter(tableId, icebergTable, dataInstructionsProvider); - - final IcebergInstructions localInstructions = IcebergInstructions.builder() - .dataInstructions(instructions.dataInstructions().get()) - .updateMode(IcebergUpdateMode.autoRefreshingMode(1)) - .build(); - - - final List snapshots = tableAdapter.listSnapshots(); - - final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); - - final IcebergTable table = tableAdapter.table(localInstructions); - Assert.eq(table.size(), "table.size()", 18073, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - - icebergTable.advanceSequenceNumber(); - Thread.sleep(500); - updateGraph.runWithinUnitTestCycle(() -> { - Assert.eq(table.size(), "table.size()", 54433, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - }); - - icebergTable.advanceSequenceNumber(); - Thread.sleep(500); - updateGraph.runWithinUnitTestCycle(() -> { - Assert.eq(table.size(), "table.size()", 72551, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - }); - - table.update(snapshots.get(2).snapshotId()); - Assert.eq(table.size(), "table.size()", 72551, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - - table.update(snapshots.get(3).snapshotId()); - Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - - table.update(snapshots.get(4).snapshotId()); - Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - - table.update(snapshots.get(5).snapshotId()); - Assert.eq(table.size(), "table.size()", 0, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java index 76688e0b421..0c8595e1a4b 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java @@ -13,6 +13,10 @@ import java.util.List; import java.util.Map; +/** + * Test table wrapper that restricts available snapshots to a given sequence number which the user can advance using + * {@link #advanceSequenceNumber()}. + */ public class IcebergRefreshingTestTable implements Table { private final IcebergTestTable testTable; From dead9c460ebc8c478e26d72fe048333f5d441afa Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 25 Sep 2024 12:23:12 -0700 Subject: [PATCH 37/56] Added TLP state (add, append, static, refreshing) --- .../table/impl/SourcePartitionedTable.java | 5 +- .../engine/table/impl/SourceTable.java | 18 ++ .../impl/locations/TableLocationProvider.java | 12 ++ .../impl/CompositeTableDataService.java | 22 +++ .../impl/FilteredTableDataService.java | 8 + .../impl/PollingTableLocationProvider.java | 7 + .../impl/SingleTableLocationProvider.java | 7 + .../table/impl/AbstractTableLocationTest.java | 13 +- .../impl/SourcePartitionedTableTest.java | 162 +++++++++++++----- .../TableBackedTableLocationProvider.java | 6 + ...ebergAutoRefreshTableLocationProvider.java | 6 + ...ergManualRefreshTableLocationProvider.java | 6 + .../IcebergStaticTableLocationProvider.java | 6 + 13 files changed, 223 insertions(+), 55 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index 3bcd3376426..6a69e4f7fca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -91,7 +91,6 @@ public static SourcePartitionedTable create( constituentDefinition, refreshLocations); - maintainer.assignLivenessManager(sourcePartitionedTable); return sourcePartitionedTable; } @@ -206,14 +205,14 @@ protected void instrumentedRefresh() { } /** - * If we have a {@link TableLocationSubscriptionBuffer}, manage it with the given liveness manager. + * Manage ourselves with the provided liveness manager and cleanup the incremented reference count from the + * constructor. */ private void assignLivenessManager(final LivenessManager manager) { manager.manage(this); dropReference(); } - private QueryTable result() { return result; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index 81e92a49eab..eefa5b707fd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -8,6 +8,7 @@ import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.TrackingWritableRowSet; +import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.TableUpdateListener; @@ -104,6 +105,15 @@ public abstract class SourceTable> exte } setRefreshing(isRefreshing); + // Propagate the TLP attribute to the table + switch (locationProvider.getUpdateMode()) { + case ADD_ONLY: + setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); + break; + case APPEND_ONLY: + setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.FALSE); + break; + } } /** @@ -225,6 +235,14 @@ private LocationChangePoller(@NotNull final TableLocationSubscriptionBuffer loca protected void instrumentedRefresh() { try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending()) { + if (locationProvider.getUpdateMode() != TableLocationProvider.UPDATE_TYPE.REFRESHING + && !locationUpdate.getPendingRemovedLocationKeys().isEmpty()) { + // This TLP doesn't support removed locations, we need to throw an exception. + final ImmutableTableLocationKey[] keys = locationUpdate.getPendingRemovedLocationKeys().stream() + .map(LiveSupplier::get).toArray(ImmutableTableLocationKey[]::new); + throw new TableLocationRemovedException("Source table does not support removed locations", keys); + } + maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index d1ca8624f49..462d77a71bd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -18,6 +18,9 @@ * Discovery utility for {@link TableLocation}s for a given table. */ public interface TableLocationProvider extends NamedImplementation { + enum UPDATE_TYPE { + APPEND_ONLY, ADD_ONLY, STATIC, REFRESHING + } /** * Get the {@link TableKey} associated with this provider. @@ -26,6 +29,15 @@ public interface TableLocationProvider extends NamedImplementation { */ ImmutableTableKey getKey(); + /** + * Get the {@link UPDATE_TYPE} of this provider. + * + * @return The associated {@link TableKey} + */ + @NotNull + UPDATE_TYPE getUpdateMode(); + + /** * ShiftObliviousListener interface for anything that wants to know about new table location keys. */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index 5924aeb6769..7c15f8ca434 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -232,6 +232,28 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t } return location; } + + @Override + @NotNull + public UPDATE_TYPE getUpdateMode() { + // Composite TLP logic: + // If any providers are REFRESHING, the overall provider is REFRESHING + // If any providers are ADD_ONLY or APPEND_ONLY, the overall provider is ADD_ONLY + // If all providers are STATIC, the overall provider is STATIC + boolean anyAdditions = false; + for (final TableLocationProvider provider : inputProviders) { + if (provider.getUpdateMode() == UPDATE_TYPE.REFRESHING) { + return UPDATE_TYPE.REFRESHING; + } else if (provider.getUpdateMode() == UPDATE_TYPE.ADD_ONLY + || provider.getUpdateMode() == UPDATE_TYPE.APPEND_ONLY) { + anyAdditions = true; + } + } + if (anyAdditions) { + return UPDATE_TYPE.ADD_ONLY; + } + return UPDATE_TYPE.STATIC; + } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 7c68e71c3f6..008e877d20e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -161,6 +161,14 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t public String getName() { return FilteredTableDataService.this.getName(); } + + @Override + @NotNull + public UPDATE_TYPE getUpdateMode() { + // Assuming a dynamic filter, locations could be added or removed at any time from the perspective of + // this filtered provider. + return UPDATE_TYPE.REFRESHING; + } } private class FilteringListener extends WeakReferenceWrapper diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java index a350b93096e..bc243010f5c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java @@ -77,4 +77,11 @@ protected final void deactivateUnderlyingDataSource() { protected final boolean matchSubscriptionToken(final T token) { return token == subscriptionToken; } + + @Override + @NotNull + public UPDATE_TYPE getUpdateMode() { + // No removals are possible from this provider but newly discovered locations can be added + return UPDATE_TYPE.ADD_ONLY; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java index f0a491ccb41..5477c91cd22 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java @@ -97,4 +97,11 @@ public boolean hasTableLocationKey(@NotNull final TableLocationKey tableLocation public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey tableLocationKey) { return hasTableLocationKey(tableLocationKey) ? tableLocation : null; } + + @Override + @NotNull + public UPDATE_TYPE getUpdateMode() { + // No additions or removals are possible from this provider + return UPDATE_TYPE.STATIC; + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java index d297ec7b34d..2eef894e984 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java @@ -32,9 +32,6 @@ @Category(OutOfBandTest.class) public class AbstractTableLocationTest extends RefreshingTableTestCase { - private CapturingUpdateGraph updateGraph; - private SafeCloseable contextCloseable; - private static class TestTableLocationProvider extends AbstractTableLocationProvider { public TestTableLocationProvider(final boolean supportsSubscriptions) { @@ -75,6 +72,12 @@ public void addKey(@NotNull TableLocationKey locationKey, @NotNull Object token) public void removeKey(@NotNull TableLocationKey locationKey, @NotNull Object token) { handleTableLocationKeyRemoved(locationKey, token); } + + @Override + @NotNull + public UPDATE_TYPE getUpdateMode() { + return UPDATE_TYPE.REFRESHING; + } } private static class TestTableLocation extends AbstractTableLocation { @@ -134,14 +137,10 @@ public void setUp() throws Exception { } super.setUp(); setExpectError(false); - - updateGraph = new CapturingUpdateGraph(ExecutionContext.getContext().getUpdateGraph().cast()); - contextCloseable = updateGraph.getContext().open(); } @Override public void tearDown() throws Exception { - contextCloseable.close(); super.tearDown(); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java index d45dae5bd7f..46628f070d6 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java @@ -10,11 +10,13 @@ import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.InvalidatedRegionException; import io.deephaven.engine.table.impl.locations.TableLocation; import io.deephaven.engine.table.impl.locations.TableLocationRemovedException; +import io.deephaven.engine.table.iterators.ChunkedColumnIterator; import io.deephaven.engine.testutil.locations.DependentRegistrar; import io.deephaven.engine.testutil.locations.TableBackedTableLocationKey; import io.deephaven.engine.testutil.locations.TableBackedTableLocationProvider; @@ -106,54 +108,97 @@ private SourcePartitionedTable setUpData() { l -> true); } + private void verifyStringColumnContents(Table table, String columnName, String... expectedValues) { + final ColumnSource columnSource = table.getColumnSource(columnName); + final Set expectedSym = Set.of(expectedValues); + + final Set actualSym = new HashSet<>(); + try (final CloseableIterator symIterator = ChunkedColumnIterator.make( + columnSource, table.getRowSet(), 1024)) { + symIterator.forEachRemaining(actualSym::add); + } + assertEquals(expectedSym, actualSym); + } + @Test public void testAddAndRemoveLocations() { final SourcePartitionedTable spt = setUpData(); final Table partitionTable = spt.table(); + final Table ptSummary = spt.merge().selectDistinct("Sym"); + assertEquals(2, partitionTable.size()); try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { assertTableEquals(tableIt.next(), p1); assertTableEquals(tableIt.next(), p2); } + // Verify the contents of the downstream summary table + verifyStringColumnContents(ptSummary, "Sym", "aa", "bb", "cc", "dd"); + + //////////////////////////////////////////// + // Remove the p1 location + //////////////////////////////////////////// + ImmutableTableLocationKey[] tlks = tlp.getTableLocationKeys() .stream().sorted().toArray(ImmutableTableLocationKey[]::new); - tlp.removeTableLocationKey(tlks[0]); + final ImmutableTableLocationKey tlk0 = tlks[0]; + + tlp.removeTableLocationKey(tlk0); tlp.refresh(); // We've removed location 0, should be gone from the location key list - Set activeSet = new HashSet<>(tlp.getTableLocationKeys()); - assertFalse(activeSet.contains(tlks[0])); + assertFalse(new HashSet<>(tlp.getTableLocationKeys()).contains(tlk0)); // Since we haven't been through an update cycle, we should still be able to retrieve the location for this key. - assertTrue(tlp.hasTableLocationKey(tlks[0])); - assertNotNull(tlp.getTableLocation(tlks[0])); + assertTrue(tlp.hasTableLocationKey(tlk0)); + assertNotNull(tlp.getTableLocation(tlk0)); - allowingError(() -> updateGraph.getDelegate().runWithinUnitTestCycle(() -> { - updateGraph.refreshSources(); - updateGraph.markSourcesRefreshedForUnitTests(); - registrar.run(); - }, false), errors -> errors.size() == 1 && - FindExceptionCause.isOrCausedBy(errors.get(0), TableLocationRemovedException.class).isPresent()); - getUpdateErrors().clear(); + // Verify the contents of the downstream summary table haven't changed yet + verifyStringColumnContents(ptSummary, "Sym", "aa", "bb", "cc", "dd"); + + updateGraph.getDelegate().startCycleForUnitTests(false); + updateGraph.refreshSources(); + updateGraph.markSourcesRefreshedForUnitTests(); + registrar.run(); + + // Verify the contents of the downstream summary table haven't changed yet + verifyStringColumnContents(ptSummary, "Sym", "aa", "bb", "cc", "dd"); + + // flush the notifications and verify changes are now visible + updateGraph.getDelegate().flushAllNormalNotificationsForUnitTests(); + verifyStringColumnContents(ptSummary, "Sym", "cc", "dd"); + + // Finish the cycle and retest + updateGraph.getDelegate().completeCycleForUnitTests(); + assertFalse(tlp.hasTableLocationKey(tlk0)); assertEquals(1, partitionTable.size()); try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { assertTableEquals(tableIt.next(), p2); } - // After the update cycle, the update committer should have triggered cleanup of this key and location. - assertFalse(tlp.hasTableLocationKey(tlks[0])); + //////////////////////////////////////////// + // Add a new location (p3) + //////////////////////////////////////////// tlp.addPending(p3); tlp.refresh(); - updateGraph.getDelegate().runWithinUnitTestCycle(() -> { - updateGraph.refreshSources(); - registrar.run(); - }); + updateGraph.getDelegate().startCycleForUnitTests(false); + updateGraph.refreshSources(); + updateGraph.markSourcesRefreshedForUnitTests(); + registrar.run(); + // Verify the contents of the downstream summary table haven't changed yet + verifyStringColumnContents(ptSummary, "Sym", "cc", "dd"); + + // flush the notifications and verify changes are now visible + updateGraph.getDelegate().flushAllNormalNotificationsForUnitTests(); + verifyStringColumnContents(ptSummary, "Sym", "cc", "dd", "ee", "ff"); + + // Finish the cycle + updateGraph.getDelegate().completeCycleForUnitTests(); assertEquals(2, partitionTable.size()); try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { @@ -161,45 +206,62 @@ public void testAddAndRemoveLocations() { assertTableEquals(tableIt.next(), p3); } + //////////////////////////////////////////// + // Add a new location (p4) and remove p2 + //////////////////////////////////////////// + tlks = tlp.getTableLocationKeys().stream().sorted().toArray(ImmutableTableLocationKey[]::new); tlp.addPending(p4); tlp.removeTableLocationKey(tlks[0]); tlp.refresh(); - allowingError(() -> updateGraph.getDelegate().runWithinUnitTestCycle(() -> { - updateGraph.refreshSources(); - updateGraph.markSourcesRefreshedForUnitTests(); - registrar.run(); + updateGraph.getDelegate().startCycleForUnitTests(false); + updateGraph.refreshSources(); + updateGraph.markSourcesRefreshedForUnitTests(); + registrar.run(); + // Verify the contents of the downstream summary table haven't changed yet + verifyStringColumnContents(ptSummary, "Sym", "cc", "dd", "ee", "ff"); - // flush the notifications and verify state - updateGraph.getDelegate().flushAllNormalNotificationsForUnitTests(); - assertEquals(2, partitionTable.size()); - try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { - assertTableEquals(tableIt.next(), p3); - assertTableEquals(tableIt.next(), p4); - } - }, false), errors -> errors.size() == 1 && - FindExceptionCause.isOrCausedBy(errors.get(0), TableLocationRemovedException.class).isPresent()); - getUpdateErrors().clear(); + // flush the notifications and verify changes are now visible + updateGraph.getDelegate().flushAllNormalNotificationsForUnitTests(); + verifyStringColumnContents(ptSummary, "Sym", "ee", "ff", "gg", "hh"); - /** + // Finish the cycle + updateGraph.getDelegate().completeCycleForUnitTests(); + + assertEquals(2, partitionTable.size()); + try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { + assertTableEquals(tableIt.next(), p3); + assertTableEquals(tableIt.next(), p4); + } + + /* * Set up a complicated table location management test, where we create a new table under a scope, add it to the - * SPT. Then drop the table and verify that the table location is destroyed only after the scope is released. + * SPT, then drop the table and verify that the table location is destroyed only after the scope is released. */ final TableLocation location5; try (final SafeCloseable ignored = LivenessScopeStack.open(new LivenessScope(), true)) { final QueryTable p5 = testRefreshingTable(i(0, 1, 2, 3).toTracking(), - stringCol("Sym", "gg", "hh", "gg", "hh"), + stringCol("Sym", "ii", "jj", "ii", "jj"), intCol("intCol", 10000, 20000, 40000, 60000), doubleCol("doubleCol", 0.1, 0.2, 0.4, 0.6)); p5.setAttribute(Table.APPEND_ONLY_TABLE_ATTRIBUTE, true); tlp.addPending(p5); tlp.refresh(); - updateGraph.getDelegate().runWithinUnitTestCycle(() -> { - updateGraph.refreshSources(); - registrar.run(); - }); + updateGraph.getDelegate().startCycleForUnitTests(false); + updateGraph.refreshSources(); + updateGraph.markSourcesRefreshedForUnitTests(); + registrar.run(); + // Verify the contents of the downstream summary table haven't changed yet + verifyStringColumnContents(ptSummary, "Sym", "ee", "ff", "gg", "hh"); + + // flush the notifications and verify changes are now visible + updateGraph.getDelegate().flushAllNormalNotificationsForUnitTests(); + verifyStringColumnContents(ptSummary, "Sym", "ee", "ff", "gg", "hh", "ii", "jj"); + + // Finish the cycle + updateGraph.getDelegate().completeCycleForUnitTests(); assertEquals(3, partitionTable.size()); try (final CloseableIterator
    tableIt = partitionTable.columnIterator("LocationTable")) { @@ -213,23 +275,33 @@ public void testAddAndRemoveLocations() { location5 = tlp.getTableLocation(tlk_p5); assertTrue(location5.getRowSet() != null && location5.getRowSet().size() == 4); + //////////////////////////////////////////// // remove the p5 key from the SPT + //////////////////////////////////////////// + tlp.removeTableLocationKey(tlk_p5); tlp.refresh(); // We've removed location 5, should be gone from the location key list - activeSet = new HashSet<>(tlp.getTableLocationKeys()); - assertFalse(activeSet.contains(tlk_p5)); + assertFalse(new HashSet<>(tlp.getTableLocationKeys()).contains(tlk_p5)); // Since we haven't been through an update cycle, we can still retrieve the location for this key. assertTrue(tlp.hasTableLocationKey(tlk_p5)); assertNotNull(tlp.getTableLocation(tlk_p5)); - updateGraph.getDelegate().runWithinUnitTestCycle(() -> { - updateGraph.refreshSources(); - registrar.run(); + updateGraph.getDelegate().startCycleForUnitTests(false); + updateGraph.refreshSources(); + updateGraph.markSourcesRefreshedForUnitTests(); + registrar.run(); + // Verify the contents of the downstream summary table haven't changed yet + verifyStringColumnContents(ptSummary, "Sym", "ee", "ff", "gg", "hh", "ii", "jj"); + + // flush the notifications and verify changes are now visible + updateGraph.getDelegate().flushAllNormalNotificationsForUnitTests(); + verifyStringColumnContents(ptSummary, "Sym", "ee", "ff", "gg", "hh"); - }); + // Finish the cycle + updateGraph.getDelegate().completeCycleForUnitTests(); // After the cycle cleanup, this location should not be available assertFalse(tlp.hasTableLocationKey(tlk_p5)); diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java index 6d742706c2f..845edccf932 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java @@ -77,4 +77,10 @@ public void refresh() { protected @NotNull TableLocation makeTableLocation(@NotNull TableLocationKey locationKey) { return new TableBackedTableLocation(registrar, (TableBackedTableLocationKey) locationKey); } + + @Override + @NotNull + public UPDATE_TYPE getUpdateMode() { + return UPDATE_TYPE.REFRESHING; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index 5d2b302b611..f56112b7fbb 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -121,4 +121,10 @@ protected final void deactivateUnderlyingDataSource() { protected final boolean matchSubscriptionToken(final T token) { return token == subscriptionToken; } + + @Override + @NotNull + public UPDATE_TYPE getUpdateMode() { + return UPDATE_TYPE.REFRESHING; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index 68d558b4d7f..836ac7f659f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -129,4 +129,10 @@ protected void deactivateUnderlyingDataSource() { protected boolean matchSubscriptionToken(T token) { return token == this; } + + @Override + @NotNull + public UPDATE_TYPE getUpdateMode() { + return UPDATE_TYPE.REFRESHING; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index 53a5e9fc172..2047daa8712 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -81,4 +81,10 @@ protected void deactivateUnderlyingDataSource() { protected boolean matchSubscriptionToken(final T token) { return false; } + + @Override + @NotNull + public UPDATE_TYPE getUpdateMode() { + return UPDATE_TYPE.STATIC; + } } From e5d10e7d66eadfdad26f74b81423d82f6ab12454 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 27 Sep 2024 12:31:20 -0700 Subject: [PATCH 38/56] Added TLP state (add, append, static, refreshing) --- .../table/impl/SourcePartitionedTable.java | 2 +- .../engine/table/impl/SourceTable.java | 33 +++++---- .../impl/locations/TableLocationProvider.java | 17 +++-- .../impl/AbstractTableLocationProvider.java | 41 ++++++++++- .../impl/CompositeTableDataService.java | 72 ++++++++++++++----- .../impl/FilteredTableDataService.java | 14 ++-- .../impl/PollingTableLocationProvider.java | 13 ++-- .../impl/SingleTableLocationProvider.java | 18 +++-- ...=> AbstractTableLocationProviderTest.java} | 14 +--- .../impl/TestPartitionAwareSourceTable.java | 7 ++ .../table/impl/TestSimpleSourceTable.java | 33 +++++++-- .../locations/impl/TestTableDataService.java | 2 +- .../TestRegionedColumnSourceManager.java | 12 ++++ .../TableBackedTableLocationProvider.java | 10 +-- ...ebergAutoRefreshTableLocationProvider.java | 21 +++--- ...ergManualRefreshTableLocationProvider.java | 16 +++-- .../IcebergStaticTableLocationProvider.java | 17 ++--- .../IcebergTableLocationProviderBase.java | 6 +- .../deephaven/parquet/table/ParquetTools.java | 13 +++- 19 files changed, 249 insertions(+), 112 deletions(-) rename engine/table/src/test/java/io/deephaven/engine/table/impl/{AbstractTableLocationTest.java => AbstractTableLocationProviderTest.java} (97%) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index 6a69e4f7fca..8328e5a18c3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -242,7 +242,7 @@ private Table makeConstituentTable(@NotNull final TableLocation tableLocation) { constituentDefinition, "SingleLocationSourceTable-" + tableLocation, RegionedTableComponentFactoryImpl.INSTANCE, - new SingleTableLocationProvider(tableLocation), + new SingleTableLocationProvider(tableLocation, TableLocationProvider.UpdateMode.ADD_REMOVE), refreshSizes ? refreshCombiner : null); // Be careful to propagate the systemic attribute properly to child tables diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index eefa5b707fd..37fd9dc8172 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -99,20 +99,23 @@ public abstract class SourceTable> exte definition.getColumns() // This is the *re-written* definition passed to the super-class constructor ); if (isRefreshing) { - // TODO: managing doesn't work here because columnSourceManager is at zero right now. manage(columnSourceManager); } } - setRefreshing(isRefreshing); - // Propagate the TLP attribute to the table - switch (locationProvider.getUpdateMode()) { - case ADD_ONLY: + if (isRefreshing) { + setRefreshing(true); + if (locationProvider.getUpdateMode() == TableLocationProvider.UpdateMode.APPEND_ONLY + && locationProvider.getLocationUpdateMode() == TableLocationProvider.UpdateMode.STATIC) { + // This table is APPEND_ONLY IFF the set of locations is APPEND_ONLY + // and the location contents are STATIC + setAttribute(Table.APPEND_ONLY_TABLE_ATTRIBUTE, Boolean.FALSE); + } else if (locationProvider.getUpdateMode() != TableLocationProvider.UpdateMode.ADD_REMOVE + && locationProvider.getLocationUpdateMode() != TableLocationProvider.UpdateMode.ADD_REMOVE) { + // This table is ADD_ONLY IFF the set of locations is not allowed to remove locations (!ADD_REMOVE) + // and the locations contents are not allowed to remove rows (!ADD_REMOVE) setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); - break; - case APPEND_ONLY: - setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.FALSE); - break; + } } } @@ -164,12 +167,16 @@ private void initializeAvailableLocations() { // Manage each of the location keys as we see them (since the TLP is not guaranteeing them outside // the callback) locationProvider.getTableLocationKeys(ttlk -> { - manage(ttlk); + if (isRefreshing()) { + manage(ttlk); + } keySuppliers.add(ttlk); }); maybeAddLocations(keySuppliers); - // Now we can un-manage the location keys - keySuppliers.forEach(this::unmanage); + if (isRefreshing()) { + // Now we can un-manage the location keys + keySuppliers.forEach(this::unmanage); + } } }); locationsInitialized = true; @@ -235,7 +242,7 @@ private LocationChangePoller(@NotNull final TableLocationSubscriptionBuffer loca protected void instrumentedRefresh() { try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending()) { - if (locationProvider.getUpdateMode() != TableLocationProvider.UPDATE_TYPE.REFRESHING + if (locationProvider.getUpdateMode() != TableLocationProvider.UpdateMode.ADD_REMOVE && !locationUpdate.getPendingRemovedLocationKeys().isEmpty()) { // This TLP doesn't support removed locations, we need to throw an exception. final ImmutableTableLocationKey[] keys = locationUpdate.getPendingRemovedLocationKeys().stream() diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index 462d77a71bd..f2932be4781 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -18,8 +18,8 @@ * Discovery utility for {@link TableLocation}s for a given table. */ public interface TableLocationProvider extends NamedImplementation { - enum UPDATE_TYPE { - APPEND_ONLY, ADD_ONLY, STATIC, REFRESHING + enum UpdateMode { + STATIC, APPEND_ONLY, ADD_ONLY, ADD_REMOVE } /** @@ -30,13 +30,18 @@ enum UPDATE_TYPE { ImmutableTableKey getKey(); /** - * Get the {@link UPDATE_TYPE} of this provider. - * - * @return The associated {@link TableKey} + * Get the {@link UpdateMode update guarantees} of this provider describing how this provider will add or remove + * table locations. */ @NotNull - UPDATE_TYPE getUpdateMode(); + TableLocationProvider.UpdateMode getUpdateMode(); + /** + * Get the location {@link UpdateMode update guarantees} of this provider describing how individual locations will + * add or remove rows. + */ + @NotNull + TableLocationProvider.UpdateMode getLocationUpdateMode(); /** * ShiftObliviousListener interface for anything that wants to know about new table location keys. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 53d4c080c60..b80e49988c8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -155,6 +155,16 @@ protected synchronized void destroy() { */ private final StandaloneLivenessManager livenessManager; + /** + * Records how the set of locations for this TLP can update + */ + private final UpdateMode updateMode; + + /** + * Records how the individual locations for this TLP can update (whether row can be added or removed) + */ + private final UpdateMode locationUpdateMode; + private volatile boolean initialized; private List partitionKeys; @@ -165,11 +175,19 @@ protected synchronized void destroy() { * * @param tableKey A key that will be used by this provider * @param supportsSubscriptions Whether this provider should support subscriptions + * @param updateMode What updates to the set of locations are allowed + * @param locationUpdateMode What updates to the location rows are allowed */ - protected AbstractTableLocationProvider(@NotNull final TableKey tableKey, final boolean supportsSubscriptions) { + protected AbstractTableLocationProvider( + @NotNull final TableKey tableKey, + final boolean supportsSubscriptions, + final UpdateMode updateMode, + final UpdateMode locationUpdateMode) { super(supportsSubscriptions); this.tableKey = tableKey.makeImmutable(); this.partitionKeys = null; + this.updateMode = updateMode; + this.locationUpdateMode = locationUpdateMode; livenessManager = new StandaloneLivenessManager(false); } @@ -178,9 +196,14 @@ protected AbstractTableLocationProvider(@NotNull final TableKey tableKey, final * Construct a standalone provider. * * @param supportsSubscriptions Whether this provider should support subscriptions + * @param updateMode What updates to the set of locations are allowed + * @param locationUpdateMode What updates to the location rows are allowed */ - protected AbstractTableLocationProvider(final boolean supportsSubscriptions) { - this(StandaloneTableKey.getInstance(), supportsSubscriptions); + protected AbstractTableLocationProvider( + final boolean supportsSubscriptions, + final UpdateMode updateMode, + final UpdateMode locationUpdateMode) { + this(StandaloneTableKey.getInstance(), supportsSubscriptions, updateMode, locationUpdateMode); } @Override @@ -570,4 +593,16 @@ private void releaseLocationKey(@NotNull final TrackedKeySupplier locationKey) { // We can now remove the key from the tableLocations map tableLocationKeyMap.removeKey(locationKey.get()); } + + @Override + @NotNull + public TableLocationProvider.UpdateMode getUpdateMode() { + return updateMode; + } + + @Override + @NotNull + public TableLocationProvider.UpdateMode getLocationUpdateMode() { + return locationUpdateMode; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index 7c15f8ca434..2dc5d1e6c46 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -118,12 +118,58 @@ private class TableLocationProviderImpl implements TableLocationProvider { private final List inputProviders; private final String implementationName; + // What guarantees about added/removed locations can be made? + private final UpdateMode updateMode; + // What guarantees about added/removed rows within locations can be made? + private final UpdateMode locationUpdateMode; + private TableLocationProviderImpl(@NotNull final TableDataService[] inputServices, @NotNull final TableKey tableKey) { this.tableKey = tableKey.makeImmutable(); inputProviders = Arrays.stream(inputServices).map(s -> s.getTableLocationProvider(this.tableKey)) .collect(Collectors.toList()); implementationName = "Composite-" + inputProviders; + + // Analyze the update modes of the input providers to determine the update mode of the composite provider. + // 1) If any providers are REFRESHING, the overall provider is REFRESHING + // 2) If any providers are ADD_ONLY or APPEND_ONLY, the overall provider is ADD_ONLY + // 3) If all providers are STATIC, the overall provider is STATIC + boolean anyAdditions = false; + UpdateMode tmpMode = UpdateMode.STATIC; + for (final TableLocationProvider provider : inputProviders) { + if (provider.getUpdateMode() == UpdateMode.ADD_REMOVE) { + tmpMode = UpdateMode.ADD_REMOVE; + break; + } else if (provider.getUpdateMode() == UpdateMode.ADD_ONLY + || provider.getUpdateMode() == UpdateMode.APPEND_ONLY) { + anyAdditions = true; + } + } + if (anyAdditions) { + tmpMode = UpdateMode.ADD_ONLY; + } + updateMode = tmpMode; + + // Analyze the location update modes of the input providers to determine the location update mode + // of the composite provider. + // 1) If any provider locations are REFRESHING, the overall provider location mode is REFRESHING + // 2) If any provider locations are ADD_ONLY or APPEND_ONLY, the overall provider location mode is ADD_ONLY + // 3) If all provider locations are STATIC, the overall provider location mode is STATIC + anyAdditions = false; + tmpMode = UpdateMode.STATIC; + for (final TableLocationProvider provider : inputProviders) { + if (provider.getLocationUpdateMode() == UpdateMode.ADD_REMOVE) { + tmpMode = UpdateMode.ADD_REMOVE; + break; + } else if (provider.getLocationUpdateMode() == UpdateMode.ADD_ONLY + || provider.getLocationUpdateMode() == UpdateMode.APPEND_ONLY) { + anyAdditions = true; + } + } + if (anyAdditions) { + tmpMode = UpdateMode.ADD_ONLY; + } + locationUpdateMode = tmpMode; } @Override @@ -235,24 +281,14 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t @Override @NotNull - public UPDATE_TYPE getUpdateMode() { - // Composite TLP logic: - // If any providers are REFRESHING, the overall provider is REFRESHING - // If any providers are ADD_ONLY or APPEND_ONLY, the overall provider is ADD_ONLY - // If all providers are STATIC, the overall provider is STATIC - boolean anyAdditions = false; - for (final TableLocationProvider provider : inputProviders) { - if (provider.getUpdateMode() == UPDATE_TYPE.REFRESHING) { - return UPDATE_TYPE.REFRESHING; - } else if (provider.getUpdateMode() == UPDATE_TYPE.ADD_ONLY - || provider.getUpdateMode() == UPDATE_TYPE.APPEND_ONLY) { - anyAdditions = true; - } - } - if (anyAdditions) { - return UPDATE_TYPE.ADD_ONLY; - } - return UPDATE_TYPE.STATIC; + public TableLocationProvider.UpdateMode getUpdateMode() { + return updateMode; + } + + @Override + @NotNull + public TableLocationProvider.UpdateMode getLocationUpdateMode() { + return locationUpdateMode; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 008e877d20e..0b09a9ed23b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -164,10 +164,16 @@ public String getName() { @Override @NotNull - public UPDATE_TYPE getUpdateMode() { - // Assuming a dynamic filter, locations could be added or removed at any time from the perspective of - // this filtered provider. - return UPDATE_TYPE.REFRESHING; + public TableLocationProvider.UpdateMode getUpdateMode() { + // TODO: I think locations can be added or removed at any time (with a dynamic filter) + return UpdateMode.ADD_REMOVE; + } + + @Override + @NotNull + public TableLocationProvider.UpdateMode getLocationUpdateMode() { + // TODO: this is defensive, individual location contents might be static (e.g. Parquet files) + return UpdateMode.ADD_REMOVE; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java index bc243010f5c..97142c6357a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java @@ -27,8 +27,10 @@ public class PollingTableLocationProvider locationKeyFinder, @NotNull final TableLocationFactory locationFactory, - @Nullable final TableDataRefreshService refreshService) { - super(tableKey, refreshService != null); + @Nullable final TableDataRefreshService refreshService, + final UpdateMode updateMode, + final UpdateMode locationUpdateMode) { + super(tableKey, refreshService != null, updateMode, locationUpdateMode); this.locationKeyFinder = locationKeyFinder; this.locationFactory = locationFactory; this.refreshService = refreshService; @@ -77,11 +79,4 @@ protected final void deactivateUnderlyingDataSource() { protected final boolean matchSubscriptionToken(final T token) { return token == subscriptionToken; } - - @Override - @NotNull - public UPDATE_TYPE getUpdateMode() { - // No removals are possible from this provider but newly discovered locations can be added - return UPDATE_TYPE.ADD_ONLY; - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java index 5477c91cd22..81a3e9d16c3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java @@ -35,14 +35,18 @@ public ImmutableTableLocationKey get() { private final TrackedKeySupplier immutableKeySupplier; private final TableLocation tableLocation; + private final UpdateMode locationUpdateMode; /** * @param tableLocation The only table location that this provider will ever provide */ - public SingleTableLocationProvider(@NotNull final TableLocation tableLocation) { + public SingleTableLocationProvider( + @NotNull final TableLocation tableLocation, + final UpdateMode locationUpdateMode) { this.tableLocation = tableLocation; // TODO: it seems like we should manage this, but SingleTableLocationProvider isn't a LivenessManager. immutableKeySupplier = new TrackedKeySupplier(tableLocation.getKey()); + this.locationUpdateMode = locationUpdateMode; } @Override @@ -100,8 +104,14 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t @Override @NotNull - public UPDATE_TYPE getUpdateMode() { - // No additions or removals are possible from this provider - return UPDATE_TYPE.STATIC; + public TableLocationProvider.UpdateMode getUpdateMode() { + // No additions or removals are possible from this provider, it exists to serve + return UpdateMode.STATIC; + } + + @Override + @NotNull + public TableLocationProvider.UpdateMode getLocationUpdateMode() { + return locationUpdateMode; } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java similarity index 97% rename from engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java rename to engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java index 2eef894e984..10e42b5cc9a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java @@ -6,7 +6,6 @@ import io.deephaven.api.SortColumn; import io.deephaven.base.verify.Assert; import io.deephaven.configuration.Configuration; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.liveness.StandaloneLivenessManager; import io.deephaven.engine.table.BasicDataIndex; @@ -17,7 +16,6 @@ import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; import io.deephaven.io.logger.StreamLoggerImpl; import io.deephaven.test.types.OutOfBandTest; -import io.deephaven.util.SafeCloseable; import io.deephaven.util.process.ProcessEnvironment; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -31,11 +29,11 @@ import java.util.Set; @Category(OutOfBandTest.class) -public class AbstractTableLocationTest extends RefreshingTableTestCase { +public class AbstractTableLocationProviderTest extends RefreshingTableTestCase { private static class TestTableLocationProvider extends AbstractTableLocationProvider { public TestTableLocationProvider(final boolean supportsSubscriptions) { - super(supportsSubscriptions); + super(supportsSubscriptions, UpdateMode.ADD_REMOVE, UpdateMode.ADD_REMOVE); } @Override @@ -72,12 +70,6 @@ public void addKey(@NotNull TableLocationKey locationKey, @NotNull Object token) public void removeKey(@NotNull TableLocationKey locationKey, @NotNull Object token) { handleTableLocationKeyRemoved(locationKey, token); } - - @Override - @NotNull - public UPDATE_TYPE getUpdateMode() { - return UPDATE_TYPE.REFRESHING; - } } private static class TestTableLocation extends AbstractTableLocation { @@ -133,7 +125,7 @@ public boolean hasDataIndex(@NotNull String... columns) { public void setUp() throws Exception { if (null == ProcessEnvironment.tryGet()) { ProcessEnvironment.basicServerInitialization(Configuration.getInstance(), - "SourcePartitionedTableTest", new StreamLoggerImpl()); + "AbstractTableLocationProviderTest", new StreamLoggerImpl()); } super.setUp(); setExpectError(false); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java index af6c13fb0b5..55c8ab3bc4a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java @@ -140,6 +140,13 @@ public void setUp() throws Exception { return mocked; }).toArray(ColumnSource[]::new); locationProvider = mock(TableLocationProvider.class); + checking(new Expectations() { + { + allowing(locationProvider).getUpdateMode(); + will(returnValue(TableLocationProvider.UpdateMode.ADD_REMOVE)); + } + }); + tableLocationKeys = IntStream.range(0, 6).mapToObj(tlki -> { final Map> partitions = new LinkedHashMap<>(); partitions.put(PARTITIONING_COLUMN_DEFINITION.getName(), COLUMN_PARTITIONS[tlki]); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java index 435077fe280..0971c01af09 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java @@ -6,10 +6,9 @@ import io.deephaven.base.Pair; import io.deephaven.base.verify.Assert; import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.liveness.LiveSupplier; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocation; @@ -18,11 +17,15 @@ import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; +import org.jmock.api.Invocation; +import org.jmock.lib.action.CustomAction; import org.junit.After; import org.junit.Before; import org.junit.Test; +import java.lang.ref.WeakReference; import java.util.*; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -55,6 +58,7 @@ public class TestSimpleSourceTable extends RefreshingTableTestCase { private TableLocationProvider locationProvider; private TableLocation tableLocation; + private LiveSupplier keySupplier; private WritableRowSet expectedRowSet; @@ -88,12 +92,31 @@ public void setUp() throws Exception { }); return mocked; }).toArray(ColumnSource[]::new); + keySupplier = mock(LiveSupplier.class); locationProvider = mock(TableLocationProvider.class); tableLocation = mock(TableLocation.class); checking(new Expectations() { { + allowing(keySupplier).get(); + will(returnValue(StandaloneTableLocationKey.getInstance())); + allowing(keySupplier).tryRetainReference(); + will(returnValue(true)); + allowing(keySupplier).getWeakReference(); + will(returnValue(new WeakReference<>(keySupplier))); + allowing(keySupplier).dropReference(); + allowing(locationProvider).getTableLocationKeys(); will(returnValue(Collections.singleton(StandaloneTableLocationKey.getInstance()))); + allowing(locationProvider).getTableLocationKeys(with(any(Consumer.class))); + will(new CustomAction("check added") { + @Override + public Object invoke(Invocation invocation) { + final Consumer> consumer = + (Consumer>) invocation.getParameter(0); + consumer.accept(keySupplier); + return null; + } + }); allowing(locationProvider).getTableLocation(with(StandaloneTableLocationKey.getInstance())); will(returnValue(tableLocation)); allowing(tableLocation).supportsSubscriptions(); @@ -102,6 +125,8 @@ public void setUp() throws Exception { will(returnValue(StandaloneTableLocationKey.getInstance())); allowing(locationProvider).supportsSubscriptions(); will(returnValue(true)); + allowing(locationProvider).getUpdateMode(); + will(returnValue(TableLocationProvider.UpdateMode.ADD_REMOVE)); } }); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestTableDataService.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestTableDataService.java index 1687a46d5ed..1b362e61eb3 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestTableDataService.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestTableDataService.java @@ -56,7 +56,7 @@ private DummyTableDataService(@NotNull final String name, @NotNull final TableLo @Override @NotNull protected TableLocationProvider makeTableLocationProvider(@NotNull TableKey tableKey) { - return new SingleTableLocationProvider(tableLocation); + return new SingleTableLocationProvider(tableLocation, TableLocationProvider.UpdateMode.ADD_REMOVE); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java index ff44314501a..ca531a2ec2d 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java @@ -30,6 +30,7 @@ import org.junit.Before; import org.junit.Test; +import java.lang.ref.WeakReference; import java.util.*; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -150,6 +151,17 @@ public void setUp() throws Exception { tableLocation1A = tableLocations[1]; tableLocation0B = tableLocations[2]; tableLocation1B = tableLocations[3]; + checking(new Expectations() { + { + for (final TableLocation tl : tableLocations) { + allowing(tl).tryRetainReference(); + will(returnValue(true)); + allowing(tl).getWeakReference(); + will(returnValue(new WeakReference<>(tl))); + allowing(tl).dropReference(); + } + } + }); duplicateTableLocation0A = setUpTableLocation(0, "-duplicate"); diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java index 845edccf932..e603785539f 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java @@ -31,8 +31,10 @@ public final class TableBackedTableLocationProvider extends AbstractTableLocatio public TableBackedTableLocationProvider( @NotNull final UpdateSourceRegistrar registrar, final boolean supportsSubscriptions, + final UpdateMode updateMode, + final UpdateMode locationUpdateMode, @NotNull final Table... tables) { - super(StandaloneTableKey.getInstance(), supportsSubscriptions); + super(StandaloneTableKey.getInstance(), supportsSubscriptions, updateMode, locationUpdateMode); this.registrar = registrar; processPending(Arrays.stream(tables)); } @@ -77,10 +79,4 @@ public void refresh() { protected @NotNull TableLocation makeTableLocation(@NotNull TableLocationKey locationKey) { return new TableBackedTableLocation(registrar, (TableBackedTableLocationKey) locationKey); } - - @Override - @NotNull - public UPDATE_TYPE getUpdateMode() { - return UPDATE_TYPE.REFRESHING; - } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index f56112b7fbb..6b951db1309 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -41,7 +41,15 @@ public IcebergAutoRefreshTableLocationProvider( final long refreshIntervalMs, @NotNull final IcebergTableAdapter adapter, @NotNull final TableIdentifier tableIdentifier) { - super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier); + super(tableKey, + locationKeyFinder, + locationFactory, + true, + adapter, + tableIdentifier, + UpdateMode.ADD_REMOVE, // New locations can be added and removed + UpdateMode.STATIC // Individual locations cannot add or remove rows + ); Assert.neqNull(refreshService, "refreshService"); this.refreshService = refreshService; @@ -116,15 +124,4 @@ protected final void deactivateUnderlyingDataSource() { subscriptionToken = null; } } - - @Override - protected final boolean matchSubscriptionToken(final T token) { - return token == subscriptionToken; - } - - @Override - @NotNull - public UPDATE_TYPE getUpdateMode() { - return UPDATE_TYPE.REFRESHING; - } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index 836ac7f659f..63d5882a4c5 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -35,7 +35,15 @@ public IcebergManualRefreshTableLocationProvider( @NotNull final TableLocationFactory locationFactory, @NotNull final IcebergTableAdapter adapter, @NotNull final TableIdentifier tableIdentifier) { - super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier); + super(tableKey, + locationKeyFinder, + locationFactory, + true, + adapter, + tableIdentifier, + UpdateMode.ADD_REMOVE, // New locations can be added and removed + UpdateMode.STATIC // Individual locations cannot add or remove rows + ); } // ------------------------------------------------------------------------------------------------------------------ @@ -129,10 +137,4 @@ protected void deactivateUnderlyingDataSource() { protected boolean matchSubscriptionToken(T token) { return token == this; } - - @Override - @NotNull - public UPDATE_TYPE getUpdateMode() { - return UPDATE_TYPE.REFRESHING; - } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index 2047daa8712..4d43c51eb68 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -26,8 +26,15 @@ public IcebergStaticTableLocationProvider( @NotNull final IcebergBaseLayout locationKeyFinder, @NotNull final TableLocationFactory locationFactory, @NotNull final TableIdentifier tableIdentifier) { - super(tableKey, locationKeyFinder, locationFactory, false, null, tableIdentifier); - } + super(tableKey, + locationKeyFinder, + locationFactory, + false, + null, + tableIdentifier, + UpdateMode.STATIC, // The set of locations is static + UpdateMode.STATIC // Individual locations cannot add or remove rows + ); } // ------------------------------------------------------------------------------------------------------------------ // AbstractTableLocationProvider implementation @@ -81,10 +88,4 @@ protected void deactivateUnderlyingDataSource() { protected boolean matchSubscriptionToken(final T token) { return false; } - - @Override - @NotNull - public UPDATE_TYPE getUpdateMode() { - return UPDATE_TYPE.STATIC; - } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java index 9ec4eeb18b0..b72e3a5cce9 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java @@ -26,8 +26,10 @@ public IcebergTableLocationProviderBase( @NotNull final TableLocationFactory locationFactory, final boolean isRefreshing, @Nullable final IcebergTableAdapter adapter, - @NotNull final TableIdentifier tableIdentifier) { - super(tableKey, isRefreshing); + @NotNull final TableIdentifier tableIdentifier, + final UpdateMode updateMode, + final UpdateMode locationUpdateMode) { + super(tableKey, isRefreshing, updateMode, locationUpdateMode); this.locationKeyFinder = locationKeyFinder; this.locationFactory = locationFactory; this.adapter = adapter; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java index ab035bdd0c8..396f58e5580 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java @@ -829,7 +829,9 @@ private static Table readTable( StandaloneTableKey.getInstance(), new KnownLocationKeyFinder<>(tableLocationKey), new ParquetTableLocationFactory(readInstructions), - null); + null, + TableLocationProvider.UpdateMode.STATIC, // exactly one location here + TableLocationProvider.UpdateMode.STATIC); // parquet files are static return new SimpleSourceTable(tableDefinition.getWritable(), "Read single parquet file from " + tableLocationKey.getURI(), RegionedTableComponentFactoryImpl.INSTANCE, locationProvider, null); @@ -890,7 +892,14 @@ public static Table readTable( StandaloneTableKey.getInstance(), keyFinder, new ParquetTableLocationFactory(useInstructions), - refreshService), + refreshService, + // If refreshing, new locations can be discovered but they will be appended + // to the locations list + useInstructions.isRefreshing() + ? TableLocationProvider.UpdateMode.APPEND_ONLY + : TableLocationProvider.UpdateMode.STATIC, + TableLocationProvider.UpdateMode.STATIC // parquet files are static + ), updateSourceRegistrar); } From b30e240337ca5324ba407bd7eff62b42fc7e1592 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 30 Sep 2024 13:44:43 -0700 Subject: [PATCH 39/56] Addressed PR comments, some TODO remaining to address. --- .../table/impl/ColumnSourceManager.java | 4 ++ .../table/impl/PartitionAwareSourceTable.java | 8 +-- .../table/impl/SourcePartitionedTable.java | 72 +++++++------------ .../engine/table/impl/SourceTable.java | 51 ++++++------- .../engine/table/impl/TableUpdateMode.java | 34 +++++++++ .../impl/locations/TableLocationKey.java | 3 +- .../impl/locations/TableLocationProvider.java | 17 ++--- .../impl/AbstractTableLocationProvider.java | 17 ++--- .../impl/CompositeTableDataService.java | 31 ++++---- .../impl/FilteredTableDataService.java | 9 +-- .../impl/PollingTableLocationProvider.java | 5 +- .../impl/SingleTableLocationProvider.java | 11 +-- .../regioned/RegionedColumnSourceManager.java | 23 ++++++ .../AbstractTableLocationProviderTest.java | 2 +- .../impl/SourcePartitionedTableTest.java | 9 ++- .../impl/TestPartitionAwareSourceTable.java | 2 +- .../table/impl/TestPartitioningColumns.java | 6 +- .../table/impl/TestSimpleSourceTable.java | 2 +- .../locations/impl/TestTableDataService.java | 3 +- .../TableBackedTableLocationProvider.java | 5 +- .../liveness/DelegatingLivenessNode.java | 24 ++----- .../engine/liveness/LivenessManager.java | 12 ++-- .../liveness/SingletonLivenessManager.java | 22 ++++-- .../liveness/StandaloneLivenessManager.java | 2 +- ...ebergAutoRefreshTableLocationProvider.java | 5 +- ...ergManualRefreshTableLocationProvider.java | 5 +- .../IcebergStaticTableLocationProvider.java | 8 ++- .../IcebergTableLocationProviderBase.java | 5 +- .../deephaven/parquet/table/ParquetTools.java | 11 +-- .../location/ParquetTableLocationKey.java | 4 +- 30 files changed, 230 insertions(+), 182 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java index a2f2aa45515..1b8b7544439 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java @@ -115,4 +115,8 @@ public interface ColumnSourceManager extends LivenessReferent { * @param tableLocationKey the location key being removed */ void removeLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey); + + Map getTableAttributes( + @NotNull TableUpdateMode tableUpdateMode, + @NotNull TableUpdateMode tableLocationUpdateMode); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java index db77d22e252..b1933919672 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java @@ -205,7 +205,7 @@ protected final Table redefine(TableDefinition newDefinitionExternal, TableDefin reference, null, viewColumns, null); } - private static final String TRACKED_KEY_COLUMN_NAME = "__PartitionAwareSourceTable_TrackedTableLocationKey__"; + private static final String KEY_SUPPLIER_COLUMN_NAME = "__PartitionAwareSourceTable_KeySupplier__"; private static ColumnSource makePartitionSource(@NotNull final ColumnDefinition columnDefinition, @NotNull final Collection locationKeys) { @@ -235,13 +235,13 @@ protected final Collection> filterLocati // TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table final List partitionTableColumnNames = Stream.concat( partitioningColumnDefinitions.keySet().stream(), - Stream.of(TRACKED_KEY_COLUMN_NAME)).collect(Collectors.toList()); + Stream.of(KEY_SUPPLIER_COLUMN_NAME)).collect(Collectors.toList()); final List> partitionTableColumnSources = new ArrayList<>(partitioningColumnDefinitions.size() + 1); for (final ColumnDefinition columnDefinition : partitioningColumnDefinitions.values()) { partitionTableColumnSources.add(makePartitionSource(columnDefinition, immutableTableLocationKeys)); } - // Add the tracked keys to the table + // Add the key suppliers to the table // noinspection unchecked,rawtypes partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource( (Collection) (Collection) foundLocationKeys, @@ -257,7 +257,7 @@ protected final Collection> filterLocati // Return the filtered keys final Iterable> iterable = - () -> filteredColumnPartitionTable.columnIterator(TRACKED_KEY_COLUMN_NAME); + () -> filteredColumnPartitionTable.columnIterator(KEY_SUPPLIER_COLUMN_NAME); return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toList()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index 8328e5a18c3..6e5d732dcfd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -38,26 +38,6 @@ public class SourcePartitionedTable extends PartitionedTableImpl { private static final String KEY_COLUMN_NAME = "TableLocationKey"; private static final String CONSTITUENT_COLUMN_NAME = "LocationTable"; - /** - * Private constructor for a {@link SourcePartitionedTable}. - * - * @param table the locations table to use for this {@link SourcePartitionedTable} - * @param constituentDefinition The {@link TableDefinition} expected of constituent {@link Table tables} - * @param refreshLocations Whether the set of locations should be refreshed - */ - private SourcePartitionedTable( - @NotNull final Table table, - @NotNull final TableDefinition constituentDefinition, - final boolean refreshLocations) { - super(table, - Set.of(KEY_COLUMN_NAME), - true, - CONSTITUENT_COLUMN_NAME, - constituentDefinition, - refreshLocations, - false); - } - /** * Construct a {@link SourcePartitionedTable} from the supplied parameters. *

    @@ -72,27 +52,26 @@ private SourcePartitionedTable( * @param refreshSizes Whether the locations found should be refreshed * @param locationKeyMatcher Function to filter desired location keys */ - public static SourcePartitionedTable create( + public SourcePartitionedTable( @NotNull final TableDefinition constituentDefinition, @NotNull final UnaryOperator

    applyTablePermissions, @NotNull final TableLocationProvider tableLocationProvider, final boolean refreshLocations, final boolean refreshSizes, @NotNull final Predicate locationKeyMatcher) { - final UnderlyingTableMaintainer maintainer = new UnderlyingTableMaintainer(constituentDefinition, + super(new UnderlyingTableMaintainer( + constituentDefinition, applyTablePermissions, tableLocationProvider, refreshLocations, refreshSizes, - locationKeyMatcher); - - final SourcePartitionedTable sourcePartitionedTable = new SourcePartitionedTable( - maintainer.result(), + locationKeyMatcher).result(), + Set.of(KEY_COLUMN_NAME), + true, + CONSTITUENT_COLUMN_NAME, constituentDefinition, - refreshLocations); - - maintainer.assignLivenessManager(sourcePartitionedTable); - return sourcePartitionedTable; + refreshLocations, + false); } private static final class UnderlyingTableMaintainer extends ReferenceCountedLivenessNode { @@ -127,9 +106,6 @@ private UnderlyingTableMaintainer( @NotNull final Predicate locationKeyMatcher) { super(false); - // Increase the refcount of this liveness node to allow it to manage the subscription buffer. - retainReference(); - this.constituentDefinition = constituentDefinition; this.applyTablePermissions = applyTablePermissions; this.tableLocationProvider = tableLocationProvider; @@ -149,14 +125,19 @@ private UnderlyingTableMaintainer( if (needToRefreshLocations || refreshSizes) { result.setRefreshing(true); refreshCombiner = new UpdateSourceCombiner(result.getUpdateGraph()); - result.addParentReference(refreshCombiner); + manage(refreshCombiner); + result.addParentReference(this); } else { refreshCombiner = null; } if (needToRefreshLocations) { + resultTableLocationKeys.startTrackingPrevValues(); + resultLocationTables.startTrackingPrevValues(); + subscriptionBuffer = new TableLocationSubscriptionBuffer(tableLocationProvider); manage(subscriptionBuffer); + pendingLocationStates = new IntrusiveDoublyLinkedQueue<>( IntrusiveDoublyLinkedNode.Adapter.getInstance()); readyLocationStates = new IntrusiveDoublyLinkedQueue<>( @@ -170,7 +151,6 @@ protected void instrumentedRefresh() { processPendingLocations(true); } }; - result.addParentReference(processNewLocationsUpdateRoot); refreshCombiner.addSource(processNewLocationsUpdateRoot); this.removedLocationsComitter = new UpdateCommitter<>( @@ -191,9 +171,9 @@ protected void instrumentedRefresh() { tableLocationProvider.refresh(); final Collection locations = new ArrayList<>(); - tableLocationProvider.getTableLocationKeys(tlk -> { - locations.add(tableLocationProvider.getTableLocation(tlk.get())); - }, locationKeyMatcher); + tableLocationProvider.getTableLocationKeys( + tlk -> locations.add(tableLocationProvider.getTableLocation(tlk.get())), + locationKeyMatcher); try (final RowSet added = sortAndAddLocations(locations.stream())) { resultRows.insert(added); } @@ -204,15 +184,6 @@ protected void instrumentedRefresh() { } } - /** - * Manage ourselves with the provided liveness manager and cleanup the incremented reference count from the - * constructor. - */ - private void assignLivenessManager(final LivenessManager manager) { - manager.manage(this); - dropReference(); - } - private QueryTable result() { return result; } @@ -242,8 +213,11 @@ private Table makeConstituentTable(@NotNull final TableLocation tableLocation) { constituentDefinition, "SingleLocationSourceTable-" + tableLocation, RegionedTableComponentFactoryImpl.INSTANCE, - new SingleTableLocationProvider(tableLocation, TableLocationProvider.UpdateMode.ADD_REMOVE), + new SingleTableLocationProvider(tableLocation, refreshSizes + ? tableLocationProvider.getLocationUpdateMode() + : TableUpdateMode.STATIC), refreshSizes ? refreshCombiner : null); + // TODO: WE NEED TO TRANSFER TL REF FROM UTM TO CONSTITUENT RCSM // Be careful to propagate the systemic attribute properly to child tables constituent.setAttribute(Table.SYSTEMIC_TABLE_ATTRIBUTE, result.isSystemicObject()); @@ -288,6 +262,7 @@ private RowSet processAdditions(final TableLocationSubscriptionBuffer.LocationUp .map(LiveSupplier::get) .filter(locationKeyMatcher) .map(tableLocationProvider::getTableLocation) + .peek(this::manage) .map(PendingLocationState::new) .forEach(pendingLocationStates::offer); for (final Iterator iter = pendingLocationStates.iterator(); iter.hasNext();) { @@ -304,6 +279,7 @@ private RowSet processAdditions(final TableLocationSubscriptionBuffer.LocationUp } private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate) { + // TODO: I THINK WE HAVE A BUG. WE AREN'T REMOVING REMOVED LOCATIONS FROM pendingLocationStates final Set relevantRemovedLocations = locationUpdate.getPendingRemovedLocationKeys() .stream() diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index 37fd9dc8172..36f116d6fa1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -6,13 +6,16 @@ import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.engine.liveness.LiveSupplier; +import io.deephaven.engine.liveness.LivenessReferent; import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.TrackingWritableRowSet; -import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.table.TableUpdateListener; -import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.TableLocationProvider; +import io.deephaven.engine.table.impl.locations.TableLocationRemovedException; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; import io.deephaven.engine.table.impl.locations.impl.TableLocationSubscriptionBuffer; @@ -105,17 +108,11 @@ public abstract class SourceTable> exte if (isRefreshing) { setRefreshing(true); - if (locationProvider.getUpdateMode() == TableLocationProvider.UpdateMode.APPEND_ONLY - && locationProvider.getLocationUpdateMode() == TableLocationProvider.UpdateMode.STATIC) { - // This table is APPEND_ONLY IFF the set of locations is APPEND_ONLY - // and the location contents are STATIC - setAttribute(Table.APPEND_ONLY_TABLE_ATTRIBUTE, Boolean.FALSE); - } else if (locationProvider.getUpdateMode() != TableLocationProvider.UpdateMode.ADD_REMOVE - && locationProvider.getLocationUpdateMode() != TableLocationProvider.UpdateMode.ADD_REMOVE) { - // This table is ADD_ONLY IFF the set of locations is not allowed to remove locations (!ADD_REMOVE) - // and the locations contents are not allowed to remove rows (!ADD_REMOVE) - setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); - } + // Given the location provider's update modes, retreive applicable table attributes from the column + // source manager. + columnSourceManager.getTableAttributes( + locationProvider.getUpdateMode(), + locationProvider.getLocationUpdateMode()).forEach(this::setAttribute); } } @@ -164,18 +161,17 @@ private void initializeAvailableLocations() { } else { locationProvider.refresh(); final Collection> keySuppliers = new ArrayList<>(); - // Manage each of the location keys as we see them (since the TLP is not guaranteeing them outside - // the callback) - locationProvider.getTableLocationKeys(ttlk -> { - if (isRefreshing()) { - manage(ttlk); - } - keySuppliers.add(ttlk); - }); - maybeAddLocations(keySuppliers); - if (isRefreshing()) { - // Now we can un-manage the location keys - keySuppliers.forEach(this::unmanage); + try { + locationProvider.getTableLocationKeys(ttlk -> { + // Retain each of the location key suppliers as we see them (since the TLP is not guaranteed + // to retain them outside the callback). + ttlk.retainReference(); + keySuppliers.add(ttlk); + }); + maybeAddLocations(keySuppliers); + } finally { + // Now we can drop the location key supplier references. + keySuppliers.forEach(LivenessReferent::dropReference); } } }); @@ -192,8 +188,7 @@ private void maybeAddLocations(@NotNull final Collection columnSourceManager.addLocation(locationProvider.getTableLocation(lk.get()))); } - private void maybeRemoveLocations( - @NotNull final Collection> removedKeys) { + private void maybeRemoveLocations(@NotNull final Collection> removedKeys) { if (removedKeys.isEmpty()) { return; } @@ -242,7 +237,7 @@ private LocationChangePoller(@NotNull final TableLocationSubscriptionBuffer loca protected void instrumentedRefresh() { try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending()) { - if (locationProvider.getUpdateMode() != TableLocationProvider.UpdateMode.ADD_REMOVE + if (!locationProvider.getUpdateMode().removeAllowed() && !locationUpdate.getPendingRemovedLocationKeys().isEmpty()) { // This TLP doesn't support removed locations, we need to throw an exception. final ImmutableTableLocationKey[] keys = locationUpdate.getPendingRemovedLocationKeys().stream() diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java new file mode 100644 index 00000000000..e4cf21df63b --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java @@ -0,0 +1,34 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl; + +/** + * Records the guarantees that the table offers regarding addition and removal of data elements. This can apply to + * {@link SourceTable sourc table} location additions and removals or row additions and removals within a location. + */ +public enum TableUpdateMode { + STATIC, APPEND_ONLY, ADD_ONLY, ADD_REMOVE; + + public boolean addAllowed() { + switch (this) { + case APPEND_ONLY: + case ADD_ONLY: + case ADD_REMOVE: + return true; + + default: + return false; + } + } + + public boolean removeAllowed() { + switch (this) { + case ADD_REMOVE: + return true; + + default: + return false; + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationKey.java index e0fa3ae29ee..93000f385d9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationKey.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationKey.java @@ -58,7 +58,8 @@ > PARTITION_VALUE_ ImmutableTableLocationKey makeImmutable(); /** - * Release any cached data associated with this key. This would only be called at EOL for this key. + * Release any cached data associated with this key. Should only be called when this key is no longer associated + * with any live TableLocation. */ default void clear() {} } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index f2932be4781..18faf0f9eae 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -4,6 +4,7 @@ package io.deephaven.engine.table.impl.locations; import io.deephaven.engine.liveness.LiveSupplier; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.util.type.NamedImplementation; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -18,10 +19,6 @@ * Discovery utility for {@link TableLocation}s for a given table. */ public interface TableLocationProvider extends NamedImplementation { - enum UpdateMode { - STATIC, APPEND_ONLY, ADD_ONLY, ADD_REMOVE - } - /** * Get the {@link TableKey} associated with this provider. * @@ -30,18 +27,18 @@ enum UpdateMode { ImmutableTableKey getKey(); /** - * Get the {@link UpdateMode update guarantees} of this provider describing how this provider will add or remove - * table locations. + * Get the {@link TableUpdateMode update guarantees} of this provider describing how this provider will add or + * remove table locations. */ @NotNull - TableLocationProvider.UpdateMode getUpdateMode(); + TableUpdateMode getUpdateMode(); /** - * Get the location {@link UpdateMode update guarantees} of this provider describing how individual locations will - * add or remove rows. + * Get the location {@link TableUpdateMode update guarantees} of this provider describing how individual locations + * will add or remove rows. */ @NotNull - TableLocationProvider.UpdateMode getLocationUpdateMode(); + TableUpdateMode getLocationUpdateMode(); /** * ShiftObliviousListener interface for anything that wants to know about new table location keys. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index b80e49988c8..7cb5d82860a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -8,6 +8,7 @@ import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; import io.deephaven.engine.liveness.StandaloneLivenessManager; import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; @@ -158,12 +159,12 @@ protected synchronized void destroy() { /** * Records how the set of locations for this TLP can update */ - private final UpdateMode updateMode; + private final TableUpdateMode updateMode; /** * Records how the individual locations for this TLP can update (whether row can be added or removed) */ - private final UpdateMode locationUpdateMode; + private final TableUpdateMode locationUpdateMode; private volatile boolean initialized; @@ -181,8 +182,8 @@ protected synchronized void destroy() { protected AbstractTableLocationProvider( @NotNull final TableKey tableKey, final boolean supportsSubscriptions, - final UpdateMode updateMode, - final UpdateMode locationUpdateMode) { + final TableUpdateMode updateMode, + final TableUpdateMode locationUpdateMode) { super(supportsSubscriptions); this.tableKey = tableKey.makeImmutable(); this.partitionKeys = null; @@ -201,8 +202,8 @@ protected AbstractTableLocationProvider( */ protected AbstractTableLocationProvider( final boolean supportsSubscriptions, - final UpdateMode updateMode, - final UpdateMode locationUpdateMode) { + final TableUpdateMode updateMode, + final TableUpdateMode locationUpdateMode) { this(StandaloneTableKey.getInstance(), supportsSubscriptions, updateMode, locationUpdateMode); } @@ -596,13 +597,13 @@ private void releaseLocationKey(@NotNull final TrackedKeySupplier locationKey) { @Override @NotNull - public TableLocationProvider.UpdateMode getUpdateMode() { + public TableUpdateMode getUpdateMode() { return updateMode; } @Override @NotNull - public TableLocationProvider.UpdateMode getLocationUpdateMode() { + public TableUpdateMode getLocationUpdateMode() { return locationUpdateMode; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index 2dc5d1e6c46..1424fc55cbc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -5,6 +5,7 @@ import io.deephaven.base.verify.Require; import io.deephaven.engine.liveness.LiveSupplier; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.util.Formatter; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.hash.KeyedObjectHashSet; @@ -119,9 +120,9 @@ private class TableLocationProviderImpl implements TableLocationProvider { private final String implementationName; // What guarantees about added/removed locations can be made? - private final UpdateMode updateMode; + private final TableUpdateMode updateMode; // What guarantees about added/removed rows within locations can be made? - private final UpdateMode locationUpdateMode; + private final TableUpdateMode locationUpdateMode; private TableLocationProviderImpl(@NotNull final TableDataService[] inputServices, @NotNull final TableKey tableKey) { @@ -135,18 +136,17 @@ private TableLocationProviderImpl(@NotNull final TableDataService[] inputService // 2) If any providers are ADD_ONLY or APPEND_ONLY, the overall provider is ADD_ONLY // 3) If all providers are STATIC, the overall provider is STATIC boolean anyAdditions = false; - UpdateMode tmpMode = UpdateMode.STATIC; + TableUpdateMode tmpMode = TableUpdateMode.STATIC; for (final TableLocationProvider provider : inputProviders) { - if (provider.getUpdateMode() == UpdateMode.ADD_REMOVE) { - tmpMode = UpdateMode.ADD_REMOVE; + if (provider.getUpdateMode().removeAllowed()) { + tmpMode = TableUpdateMode.ADD_REMOVE; break; - } else if (provider.getUpdateMode() == UpdateMode.ADD_ONLY - || provider.getUpdateMode() == UpdateMode.APPEND_ONLY) { + } else if (provider.getUpdateMode().addAllowed()) { anyAdditions = true; } } if (anyAdditions) { - tmpMode = UpdateMode.ADD_ONLY; + tmpMode = TableUpdateMode.ADD_ONLY; } updateMode = tmpMode; @@ -156,18 +156,17 @@ private TableLocationProviderImpl(@NotNull final TableDataService[] inputService // 2) If any provider locations are ADD_ONLY or APPEND_ONLY, the overall provider location mode is ADD_ONLY // 3) If all provider locations are STATIC, the overall provider location mode is STATIC anyAdditions = false; - tmpMode = UpdateMode.STATIC; + tmpMode = TableUpdateMode.STATIC; for (final TableLocationProvider provider : inputProviders) { - if (provider.getLocationUpdateMode() == UpdateMode.ADD_REMOVE) { - tmpMode = UpdateMode.ADD_REMOVE; + if (provider.getLocationUpdateMode().removeAllowed()) { + tmpMode = TableUpdateMode.ADD_REMOVE; break; - } else if (provider.getLocationUpdateMode() == UpdateMode.ADD_ONLY - || provider.getLocationUpdateMode() == UpdateMode.APPEND_ONLY) { + } else if (provider.getLocationUpdateMode().addAllowed()) { anyAdditions = true; } } if (anyAdditions) { - tmpMode = UpdateMode.ADD_ONLY; + tmpMode = TableUpdateMode.ADD_ONLY; } locationUpdateMode = tmpMode; } @@ -281,13 +280,13 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t @Override @NotNull - public TableLocationProvider.UpdateMode getUpdateMode() { + public TableUpdateMode getUpdateMode() { return updateMode; } @Override @NotNull - public TableLocationProvider.UpdateMode getLocationUpdateMode() { + public TableUpdateMode getLocationUpdateMode() { return locationUpdateMode; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 0b09a9ed23b..add5f0b7db4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -6,6 +6,7 @@ import io.deephaven.base.reference.WeakReferenceWrapper; import io.deephaven.base.verify.Require; import io.deephaven.engine.liveness.LiveSupplier; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -164,16 +165,16 @@ public String getName() { @Override @NotNull - public TableLocationProvider.UpdateMode getUpdateMode() { + public TableUpdateMode getUpdateMode() { // TODO: I think locations can be added or removed at any time (with a dynamic filter) - return UpdateMode.ADD_REMOVE; + return TableUpdateMode.ADD_REMOVE; } @Override @NotNull - public TableLocationProvider.UpdateMode getLocationUpdateMode() { + public TableUpdateMode getLocationUpdateMode() { // TODO: this is defensive, individual location contents might be static (e.g. Parquet files) - return UpdateMode.ADD_REMOVE; + return TableUpdateMode.ADD_REMOVE; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java index 97142c6357a..40a9bf8e761 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.locations.impl; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import org.jetbrains.annotations.NotNull; @@ -28,8 +29,8 @@ public PollingTableLocationProvider(@NotNull final TK tableKey, @NotNull final TableLocationKeyFinder locationKeyFinder, @NotNull final TableLocationFactory locationFactory, @Nullable final TableDataRefreshService refreshService, - final UpdateMode updateMode, - final UpdateMode locationUpdateMode) { + final TableUpdateMode updateMode, + final TableUpdateMode locationUpdateMode) { super(tableKey, refreshService != null, updateMode, locationUpdateMode); this.locationKeyFinder = locationKeyFinder; this.locationFactory = locationFactory; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java index 81a3e9d16c3..79f50e1a9e1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java @@ -5,6 +5,7 @@ import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -35,14 +36,14 @@ public ImmutableTableLocationKey get() { private final TrackedKeySupplier immutableKeySupplier; private final TableLocation tableLocation; - private final UpdateMode locationUpdateMode; + private final TableUpdateMode locationUpdateMode; /** * @param tableLocation The only table location that this provider will ever provide */ public SingleTableLocationProvider( @NotNull final TableLocation tableLocation, - final UpdateMode locationUpdateMode) { + final TableUpdateMode locationUpdateMode) { this.tableLocation = tableLocation; // TODO: it seems like we should manage this, but SingleTableLocationProvider isn't a LivenessManager. immutableKeySupplier = new TrackedKeySupplier(tableLocation.getKey()); @@ -104,14 +105,14 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t @Override @NotNull - public TableLocationProvider.UpdateMode getUpdateMode() { + public TableUpdateMode getUpdateMode() { // No additions or removals are possible from this provider, it exists to serve - return UpdateMode.STATIC; + return TableUpdateMode.STATIC; } @Override @NotNull - public TableLocationProvider.UpdateMode getLocationUpdateMode() { + public TableUpdateMode getLocationUpdateMode() { return locationUpdateMode; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 624d02ef820..8ae4445ad41 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -582,6 +582,8 @@ private class IncludedTableLocationEntry implements Comparable> columnLocationStates = new ArrayList<>(); @@ -737,4 +739,25 @@ private void regionAllocated(final int regionIndex) { "source.addRegion((definition, location)"); } } + + public Map getTableAttributes( + @NotNull TableUpdateMode tableUpdateMode, + @NotNull TableUpdateMode tableLocationUpdateMode) { + final Map attributes = new LinkedHashMap<>(); + // NOTE: Current RegionedColumnSourceManager implementation appends new locations and does not reuse + // region indices. This is important for the following attributes to be correct. + + if (tableUpdateMode == TableUpdateMode.APPEND_ONLY + && tableLocationUpdateMode == TableUpdateMode.STATIC) { + // This table is APPEND_ONLY IFF the set of locations is APPEND_ONLY + // and the location contents are STATIC + attributes.put(Table.APPEND_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); + } else if (!tableUpdateMode.removeAllowed() + && !tableLocationUpdateMode.removeAllowed()) { + // This table is ADD_ONLY IFF the set of locations is not allowed to remove locations + // and the locations contents are not allowed to remove rows + attributes.put(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); + } + return attributes; + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java index 10e42b5cc9a..84956ee4ac7 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java @@ -33,7 +33,7 @@ public class AbstractTableLocationProviderTest extends RefreshingTableTestCase { private static class TestTableLocationProvider extends AbstractTableLocationProvider { public TestTableLocationProvider(final boolean supportsSubscriptions) { - super(supportsSubscriptions, UpdateMode.ADD_REMOVE, UpdateMode.ADD_REMOVE); + super(supportsSubscriptions, TableUpdateMode.ADD_REMOVE, TableUpdateMode.ADD_REMOVE); } @Override diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java index 46628f070d6..af416b73aad 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java @@ -12,10 +12,7 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; -import io.deephaven.engine.table.impl.locations.InvalidatedRegionException; -import io.deephaven.engine.table.impl.locations.TableLocation; -import io.deephaven.engine.table.impl.locations.TableLocationRemovedException; +import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.iterators.ChunkedColumnIterator; import io.deephaven.engine.testutil.locations.DependentRegistrar; import io.deephaven.engine.testutil.locations.TableBackedTableLocationKey; @@ -98,9 +95,11 @@ private SourcePartitionedTable setUpData() { tlp = new TableBackedTableLocationProvider( registrar, true, + TableUpdateMode.ADD_REMOVE, + TableUpdateMode.ADD_REMOVE, p1, p2); - return SourcePartitionedTable.create(p1.getDefinition(), + return new SourcePartitionedTable(p1.getDefinition(), t -> t, tlp, true, diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java index 55c8ab3bc4a..0e507df3c1e 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java @@ -143,7 +143,7 @@ public void setUp() throws Exception { checking(new Expectations() { { allowing(locationProvider).getUpdateMode(); - will(returnValue(TableLocationProvider.UpdateMode.ADD_REMOVE)); + will(returnValue(TableUpdateMode.ADD_REMOVE)); } }); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitioningColumns.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitioningColumns.java index 63cc10f6a7d..2da405b3c05 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitioningColumns.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitioningColumns.java @@ -10,6 +10,7 @@ import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.dataindex.DataIndexUtils; import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.table.impl.locations.TableLocationProvider; import io.deephaven.engine.table.impl.select.MatchFilter.MatchType; import io.deephaven.engine.table.iterators.ChunkedColumnIterator; import io.deephaven.engine.testutil.TstUtils; @@ -80,7 +81,9 @@ public void testEverything() { tl.handleUpdate(RowSetFactory.flat(1), 1L); return tl; }, - null), + null, + TableUpdateMode.STATIC, + TableUpdateMode.STATIC), null); for (String colName : partitionKeys) { @@ -113,7 +116,6 @@ public void testEverything() { Assert.eqTrue(fullRowSet.containsRange(rowKey, rowKey), "fullRowSet.containsRange(rowKey, rowKey)"); } } - } final Table expected = input.sort(input.getDefinition().getColumnNamesArray()); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java index 0971c01af09..c23c56f3e9c 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java @@ -126,7 +126,7 @@ public Object invoke(Invocation invocation) { allowing(locationProvider).supportsSubscriptions(); will(returnValue(true)); allowing(locationProvider).getUpdateMode(); - will(returnValue(TableLocationProvider.UpdateMode.ADD_REMOVE)); + will(returnValue(TableUpdateMode.ADD_REMOVE)); } }); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestTableDataService.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestTableDataService.java index 1b362e61eb3..100485541e7 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestTableDataService.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestTableDataService.java @@ -4,6 +4,7 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.engine.table.impl.DummyTableLocation; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableDataService; import io.deephaven.engine.table.impl.locations.TableKey; @@ -56,7 +57,7 @@ private DummyTableDataService(@NotNull final String name, @NotNull final TableLo @Override @NotNull protected TableLocationProvider makeTableLocationProvider(@NotNull TableKey tableKey) { - return new SingleTableLocationProvider(tableLocation, TableLocationProvider.UpdateMode.ADD_REMOVE); + return new SingleTableLocationProvider(tableLocation, TableUpdateMode.ADD_REMOVE); } } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java index e603785539f..65e196b6d46 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java @@ -6,6 +6,7 @@ import io.deephaven.base.verify.Assert; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.TableLocation; import io.deephaven.engine.table.impl.locations.TableLocationKey; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; @@ -31,8 +32,8 @@ public final class TableBackedTableLocationProvider extends AbstractTableLocatio public TableBackedTableLocationProvider( @NotNull final UpdateSourceRegistrar registrar, final boolean supportsSubscriptions, - final UpdateMode updateMode, - final UpdateMode locationUpdateMode, + final TableUpdateMode updateMode, + final TableUpdateMode locationUpdateMode, @NotNull final Table... tables) { super(StandaloneTableKey.getInstance(), supportsSubscriptions, updateMode, locationUpdateMode); this.registrar = registrar; diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessNode.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessNode.java index 5c539405edf..d11ed40c00b 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessNode.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/DelegatingLivenessNode.java @@ -5,14 +5,13 @@ import org.jetbrains.annotations.NotNull; -import java.lang.ref.WeakReference; import java.util.stream.Stream; /** * Indicates that this class implements LivenessNode via a member rather than implementing it directly. The real * LivenessNode is exposed via {@link #asLivenessNode()}, all other methods delegate to this instance. */ -public interface DelegatingLivenessNode extends LivenessNode { +public interface DelegatingLivenessNode extends DelegatingLivenessReferent, LivenessNode { /** * Returns the "real" {@link LivenessNode} instance. When implementing this, care should be taken to match lifecycle * of the {@code DelegatingLivenessNode} instance with this instance, as the returned {@code LivenessNode} behaves @@ -22,6 +21,11 @@ public interface DelegatingLivenessNode extends LivenessNode { */ LivenessNode asLivenessNode(); + @Override + default LivenessReferent asLivenessReferent() { + return asLivenessNode(); + } + @Override default boolean tryManage(@NotNull LivenessReferent referent) { return asLivenessNode().tryManage(referent); @@ -36,20 +40,4 @@ default boolean tryUnmanage(@NotNull LivenessReferent referent) { default boolean tryUnmanage(@NotNull Stream referents) { return asLivenessNode().tryUnmanage(referents); } - - @Override - default boolean tryRetainReference() { - return asLivenessNode().tryRetainReference(); - } - - @Override - default void dropReference() { - asLivenessNode().dropReference(); - } - - @Override - default WeakReference getWeakReference() { - // Must return a WeakReference to the DelegatingLivenessNode, not the underlying LivenessNode - return new WeakReference<>(this); - } } diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessManager.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessManager.java index 177acd509ec..4bcfdfc4af6 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessManager.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/LivenessManager.java @@ -48,8 +48,7 @@ default void manage(@NotNull final LivenessReferent referent) { @FinalDefault default void unmanage(@NotNull LivenessReferent referent) { if (!tryUnmanage(referent)) { - throw new LivenessStateException(this + " is no longer live and cannot unmanage " + - referent.getReferentDescription()); + throw new LivenessStateException(this + " cannot unmanage " + referent.getReferentDescription()); } } @@ -58,7 +57,9 @@ default void unmanage(@NotNull LivenessReferent referent) { * {@link LivenessReferent}, then this method is a no-op if {@code this} is not live. * * @param referent The referent to drop - * @return Whether this node was live and thus in fact tried to drop a reference + * @return If this node is also a {@link LivenessReferent}, whether this node was live and thus in fact tried to + * drop a reference. Else always returns {@code true} if dropping a reference via this method is supported + * by the implementation. */ boolean tryUnmanage(@NotNull LivenessReferent referent); @@ -66,7 +67,6 @@ default void unmanage(@NotNull LivenessReferent referent) { * For each referent in {@code referent}, if this manager manages referent one or more times, drop one such * reference. If this manager is also a {@link LivenessReferent}, then it must also be live. * - * * @param referents The referents to drop */ @SuppressWarnings("unused") @@ -83,7 +83,9 @@ default void unmanage(@NotNull Stream referents) { * this manager is also a {@link LivenessReferent}, then this method is a no-op if {@code this} is not live. * * @param referents The referents to drop - * @return Whether this node was live and thus in fact tried to drop a reference + * @return If this node is also a {@link LivenessReferent}, whether this node was live and thus in fact tried to + * drop the references. Else always returns {@code true} if dropping a reference via this method is + * supported by the implementation. */ @SuppressWarnings("unused") boolean tryUnmanage(@NotNull Stream referents); diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/SingletonLivenessManager.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/SingletonLivenessManager.java index bacbb972b97..d047b0c9339 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/SingletonLivenessManager.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/SingletonLivenessManager.java @@ -62,16 +62,30 @@ public final boolean tryManage(@NotNull LivenessReferent referent) { return true; } + /** + * @inheritDoc + * @implNote This is equivalent to {@link #release()} if {@code referent} is the one managed by this manager. + */ @Override public boolean tryUnmanage(@NotNull LivenessReferent referent) { - throw new UnsupportedOperationException( - "SingletonLivenessManager should call release() instead of tryUnmanage()"); + final WeakReference localRetainedReference; + if (!Liveness.REFERENCE_TRACKING_DISABLED + && (localRetainedReference = retainedReference) != null + && localRetainedReference.get() == referent) { + release(); + } + return true; } + /** + * @inheritDoc + * @implNote This is equivalent to {@link #release()} if any element of {@code referents} is the one managed by this + * manager. + */ @Override public boolean tryUnmanage(@NotNull Stream referents) { - throw new UnsupportedOperationException( - "SingletonLivenessManager should call release() instead of tryUnmanage()"); + referents.forEach(this::tryUnmanage); + return true; } @Override diff --git a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/StandaloneLivenessManager.java b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/StandaloneLivenessManager.java index c19dd0027ba..32aa80efb0a 100644 --- a/engine/updategraph/src/main/java/io/deephaven/engine/liveness/StandaloneLivenessManager.java +++ b/engine/updategraph/src/main/java/io/deephaven/engine/liveness/StandaloneLivenessManager.java @@ -12,7 +12,7 @@ import java.util.stream.Stream; /** - * {@link LivenessNode} implementation that relies on reference counting to determine its liveness. + * Simple {@link LivenessManager} implementation. */ public class StandaloneLivenessManager implements LivenessManager, LogOutputAppendable { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index 6b951db1309..2b7415fc144 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -4,6 +4,7 @@ package io.deephaven.iceberg.layout; import io.deephaven.base.verify.Assert; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; @@ -47,8 +48,8 @@ public IcebergAutoRefreshTableLocationProvider( true, adapter, tableIdentifier, - UpdateMode.ADD_REMOVE, // New locations can be added and removed - UpdateMode.STATIC // Individual locations cannot add or remove rows + TableUpdateMode.ADD_REMOVE, // New locations can be added and removed + TableUpdateMode.STATIC // Individual locations cannot add or remove rows ); Assert.neqNull(refreshService, "refreshService"); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index 63d5882a4c5..c05689d38da 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -3,6 +3,7 @@ // package io.deephaven.iceberg.layout; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; @@ -41,8 +42,8 @@ public IcebergManualRefreshTableLocationProvider( true, adapter, tableIdentifier, - UpdateMode.ADD_REMOVE, // New locations can be added and removed - UpdateMode.STATIC // Individual locations cannot add or remove rows + TableUpdateMode.ADD_REMOVE, // New locations can be added and removed + TableUpdateMode.STATIC // Individual locations cannot add or remove rows ); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index 4d43c51eb68..16c0f0a5d73 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -3,6 +3,7 @@ // package io.deephaven.iceberg.layout; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; @@ -32,9 +33,10 @@ public IcebergStaticTableLocationProvider( false, null, tableIdentifier, - UpdateMode.STATIC, // The set of locations is static - UpdateMode.STATIC // Individual locations cannot add or remove rows - ); } + TableUpdateMode.STATIC, // The set of locations is static + TableUpdateMode.STATIC // Individual locations cannot add or remove rows + ); + } // ------------------------------------------------------------------------------------------------------------------ // AbstractTableLocationProvider implementation diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java index b72e3a5cce9..6b646eec88e 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java @@ -3,6 +3,7 @@ // package io.deephaven.iceberg.layout; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory; @@ -27,8 +28,8 @@ public IcebergTableLocationProviderBase( final boolean isRefreshing, @Nullable final IcebergTableAdapter adapter, @NotNull final TableIdentifier tableIdentifier, - final UpdateMode updateMode, - final UpdateMode locationUpdateMode) { + final TableUpdateMode updateMode, + final TableUpdateMode locationUpdateMode) { super(tableKey, isRefreshing, updateMode, locationUpdateMode); this.locationKeyFinder = locationKeyFinder; this.locationFactory = locationFactory; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java index 396f58e5580..bfc41bd4fce 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.PartitionedTableFactory; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.util.PartitionFormatter; import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; @@ -830,8 +831,8 @@ private static Table readTable( new KnownLocationKeyFinder<>(tableLocationKey), new ParquetTableLocationFactory(readInstructions), null, - TableLocationProvider.UpdateMode.STATIC, // exactly one location here - TableLocationProvider.UpdateMode.STATIC); // parquet files are static + TableUpdateMode.STATIC, // exactly one location here + TableUpdateMode.STATIC); // parquet files are static return new SimpleSourceTable(tableDefinition.getWritable(), "Read single parquet file from " + tableLocationKey.getURI(), RegionedTableComponentFactoryImpl.INSTANCE, locationProvider, null); @@ -896,9 +897,9 @@ public static Table readTable( // If refreshing, new locations can be discovered but they will be appended // to the locations list useInstructions.isRefreshing() - ? TableLocationProvider.UpdateMode.APPEND_ONLY - : TableLocationProvider.UpdateMode.STATIC, - TableLocationProvider.UpdateMode.STATIC // parquet files are static + ? TableUpdateMode.APPEND_ONLY + : TableUpdateMode.STATIC, + TableUpdateMode.STATIC // parquet files are static ), updateSourceRegistrar); } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java index e3002c98bf0..1681c0fc339 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocationKey.java @@ -180,8 +180,10 @@ public synchronized void setRowGroupIndices(final int[] rowGroupIndices) { } @Override - public void clear() { + public synchronized void clear() { metadata = null; fileReader = null; + rowGroupIndices = null; + channelsProvider = null; } } From fa9d1545e6fc54f0079f09dfc6bfbeb381c3cce4 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 1 Oct 2024 11:21:32 -0700 Subject: [PATCH 40/56] Improved table location management in SourcePartitionedTable --- .../engine/table/impl/ColumnSourceManager.java | 11 +++++++++-- .../table/impl/SourcePartitionedTable.java | 17 +++++++++++++++-- .../engine/table/impl/SourceTable.java | 3 +-- 3 files changed, 25 insertions(+), 6 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java index 1b8b7544439..091f61eff85 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java @@ -3,7 +3,7 @@ // package io.deephaven.engine.table.impl; -import io.deephaven.engine.liveness.LivenessReferent; +import io.deephaven.engine.liveness.LivenessNode; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.TrackingWritableRowSet; import io.deephaven.engine.table.*; @@ -18,7 +18,7 @@ /** * Manager for ColumnSources in a Table. */ -public interface ColumnSourceManager extends LivenessReferent { +public interface ColumnSourceManager extends LivenessNode { /** * Get a map of name to {@link ColumnSource} for the column sources maintained by this manager. @@ -116,6 +116,13 @@ public interface ColumnSourceManager extends LivenessReferent { */ void removeLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey); + /** + * Get a list of Table attributes that can be applied to the output source table, given the update modes of the + * underlying table location provider. + * + * @param tableUpdateMode The update mode of the table location set + * @param tableLocationUpdateMode The update mode of the table location rows + */ Map getTableAttributes( @NotNull TableUpdateMode tableUpdateMode, @NotNull TableUpdateMode tableLocationUpdateMode); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index 6e5d732dcfd..20089506487 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -217,7 +217,11 @@ private Table makeConstituentTable(@NotNull final TableLocation tableLocation) { ? tableLocationProvider.getLocationUpdateMode() : TableUpdateMode.STATIC), refreshSizes ? refreshCombiner : null); - // TODO: WE NEED TO TRANSFER TL REF FROM UTM TO CONSTITUENT RCSM + + // Transfer management to the constituent CSM. NOTE: this is likely to end up double-managed + // after the CSM adds the location to the table, but that's acceptable. + constituent.columnSourceManager.manage(tableLocation); + unmanage(tableLocation); // Be careful to propagate the systemic attribute properly to child tables constituent.setAttribute(Table.SYSTEMIC_TABLE_ATTRIBUTE, result.isSystemicObject()); @@ -279,7 +283,6 @@ private RowSet processAdditions(final TableLocationSubscriptionBuffer.LocationUp } private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate) { - // TODO: I THINK WE HAVE A BUG. WE AREN'T REMOVING REMOVED LOCATIONS FROM pendingLocationStates final Set relevantRemovedLocations = locationUpdate.getPendingRemovedLocationKeys() .stream() @@ -291,6 +294,16 @@ private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpd return RowSetFactory.empty(); } + // Iterate through the pending locations and remove any that are in the removed set. + for (final Iterator iter = pendingLocationStates.iterator(); iter.hasNext();) { + final PendingLocationState pendingLocationState = iter.next(); + if (relevantRemovedLocations.contains(pendingLocationState.location.getKey())) { + iter.remove(); + // Release the state and unmanage the location + unmanage(pendingLocationState.release()); + } + } + // At the end of the cycle we need to make sure we unmanage any removed constituents. this.removedConstituents = new ArrayList<>(relevantRemovedLocations.size()); final RowSetBuilderSequential deleteBuilder = RowSetFactory.builderSequential(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index 36f116d6fa1..c04ac6033fd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -108,8 +108,7 @@ public abstract class SourceTable> exte if (isRefreshing) { setRefreshing(true); - // Given the location provider's update modes, retreive applicable table attributes from the column - // source manager. + // Given the location provider's update modes, retrieve applicable table attributes from the CSM columnSourceManager.getTableAttributes( locationProvider.getUpdateMode(), locationProvider.getLocationUpdateMode()).forEach(this::setAttribute); From db2b03142b998d845ff1e32e1fd24e1face92347 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 2 Oct 2024 13:20:01 -0700 Subject: [PATCH 41/56] Post-merge cleanup --- .../iceberg/layout/IcebergBaseLayout.java | 2 ++ .../io/deephaven/iceberg/PyIceberg1Test.java | 17 ++++++++++++----- .../TestCatalog/IcebergRefreshingTestTable.java | 2 +- 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index 15127457c9b..d5905587235 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -15,6 +15,7 @@ import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import org.apache.iceberg.*; +import org.apache.iceberg.io.FileIO; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -126,6 +127,7 @@ public IcebergBaseLayout( @NotNull private URI dataFileUri(@NotNull DataFile df) { String path = df.path().toString(); + final FileIO fileIO = tableAdapter.icebergTable().io(); if (fileIO instanceof RelativeFileIO) { path = ((RelativeFileIO) fileIO).absoluteLocation(path); } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java index c85b78eba43..927089710d1 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/PyIceberg1Test.java @@ -10,6 +10,7 @@ import io.deephaven.engine.util.TableTools; import io.deephaven.iceberg.sqlite.DbResource; import io.deephaven.iceberg.util.IcebergCatalogAdapter; +import io.deephaven.iceberg.util.IcebergTableAdapter; import io.deephaven.iceberg.util.IcebergTools; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Namespace; @@ -57,7 +58,9 @@ void setUp() throws URISyntaxException { void catalogInfo() { assertThat(catalogAdapter.listNamespaces()).containsExactly(NAMESPACE); assertThat(catalogAdapter.listTables(NAMESPACE)).containsExactly(CITIES_ID); - final List snapshots = catalogAdapter.listSnapshots(CITIES_ID); + + final IcebergTableAdapter tableAdapter = catalogAdapter.loadTable(CITIES_ID); + final List snapshots = tableAdapter.listSnapshots(); assertThat(snapshots).hasSize(2); { final Snapshot snapshot = snapshots.get(0); @@ -79,9 +82,11 @@ void catalogInfo() { void cities1() { final Table cities1; { - final TableDefinition td = catalogAdapter.getTableDefinition(CITIES_ID.toString(), SNAPSHOT_1_ID, null); + final IcebergTableAdapter tableAdapter = catalogAdapter.loadTable(CITIES_ID); + final TableDefinition td = tableAdapter.definition(SNAPSHOT_1_ID, null); assertThat(td).isEqualTo(CITIES_1_TD); - cities1 = catalogAdapter.readTable(CITIES_ID, SNAPSHOT_1_ID); + + cities1 = tableAdapter.table(SNAPSHOT_1_ID); assertThat(cities1.getDefinition()).isEqualTo(CITIES_1_TD); } final Table expectedCities1 = TableTools.newTable(CITIES_1_TD, @@ -95,9 +100,11 @@ void cities1() { void cities2() { final Table cities2; { - final TableDefinition td = catalogAdapter.getTableDefinition(CITIES_ID.toString(), SNAPSHOT_2_ID, null); + final IcebergTableAdapter tableAdapter = catalogAdapter.loadTable(CITIES_ID); + final TableDefinition td = tableAdapter.definition(SNAPSHOT_2_ID, null); assertThat(td).isEqualTo(CITIES_2_TD); - cities2 = catalogAdapter.readTable(CITIES_ID, SNAPSHOT_2_ID); + + cities2 = tableAdapter.table(SNAPSHOT_2_ID); assertThat(cities2.getDefinition()).isEqualTo(CITIES_2_TD); } // TODO(deephaven-core#6118): Iceberg column rename handling diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java index 0c8595e1a4b..8f29da976d3 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java @@ -3,11 +3,11 @@ // package io.deephaven.iceberg.TestCatalog; +import org.apache.commons.lang3.NotImplementedException; import org.apache.iceberg.*; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; -import org.testcontainers.shaded.org.apache.commons.lang3.NotImplementedException; import java.util.ArrayList; import java.util.List; From 746b343e06835d802296dbb0092436d78b3d7bc0 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 2 Oct 2024 14:21:41 -0700 Subject: [PATCH 42/56] Post-merge cleanup and test updating. --- .../iceberg/util/IcebergToolsTest.java | 276 ++++++------------ .../iceberg/util/IcebergCatalogAdapter.java | 20 +- .../AppendOnlyFixedSizePageRegionTest.java | 3 +- 3 files changed, 108 insertions(+), 191 deletions(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 2484a542010..8c454000eee 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -13,10 +13,7 @@ import io.deephaven.engine.testutil.ControlledUpdateGraph; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.extensions.s3.S3Instructions; -import io.deephaven.iceberg.TestCatalog.IcebergRefreshingTestTable; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; -import io.deephaven.iceberg.TestCatalog.IcebergTestTable; -import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import io.deephaven.test.types.OutOfBandTest; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; @@ -234,14 +231,30 @@ public void testListTables() { Assert.equals(table.getDefinition(), "table.getDefinition()", TABLES_DEFINITION); } + @Test + public void testGetTableAdapter() { + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); + + // Test the overloads of the load() method. + final IcebergTableAdapter tableAdapter0 = adapter.loadTable("sales.sales_single"); + final IcebergTableAdapter tableAdapter1 = adapter.loadTable(TableIdentifier.of("sales", "sales_single")); + + Assert.eq(tableAdapter0.listSnapshots().size(), "tableAdapter0.listSnapshots().size()", + tableAdapter1.listSnapshots().size(), "tableAdapter1.listSnapshots().size()"); + + Assert.eq(tableAdapter0.currentSnapshot().timestampMillis(), + "tableAdapter0.currentSnapshot().timestampMillis()", + tableAdapter1.currentSnapshot().timestampMillis(), "tableAdapter1.currentSnapshot().timestampMillis()"); + } + @Test public void testListSnapshots() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); final TLongArrayList snapshotIds = new TLongArrayList(); - final TableIdentifier tableIdentifier = TableIdentifier.of("sales", "sales_multi"); - adapter.listSnapshots(tableIdentifier) - .forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); + + tableAdapter.listSnapshots().forEach(snapshot -> snapshotIds.add(snapshot.snapshotId())); Assert.eq(snapshotIds.size(), "snapshots.size()", 6, "snapshots for sales/sales_multi"); @@ -252,12 +265,7 @@ public void testListSnapshots() { Assert.eqTrue(snapshotIds.contains(1277776933184906785L), "snapshots.contains(1277776933184906785L)"); Assert.eqTrue(snapshotIds.contains(3825168261540020388L), "snapshots.contains(3825168261540020388L)"); - Table table = adapter.snapshots(tableIdentifier); - Assert.eq(table.size(), "table.size()", 6, "snapshots for sales/sales_multi"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION); - - // Test the string versions of the methods - table = adapter.snapshots("sales.sales_multi"); + Table table = tableAdapter.snapshots(); Assert.eq(table.size(), "table.size()", 6, "snapshots for sales/sales_multi"); Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION); } @@ -267,17 +275,8 @@ public void testOpenTableA() throws ExecutionException, InterruptedException, Ti uploadSalesPartitioned(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); - - // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_PARTITIONED_DEFINITION); - - // Test the string versions of the methods - table = adapter.readTable("sales.sales_partitioned", instructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); + final io.deephaven.engine.table.Table table = tableAdapter.table(instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -289,21 +288,12 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); + final io.deephaven.engine.table.Table table = tableAdapter.table(instructions); // This table ends up with zero records Assert.eq(table.size(), "table.size()", 0, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); - - // Test the string versions of the methods - table = adapter.readTable("sales.sales_multi", instructions); - - // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 0, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION); } @Test @@ -311,17 +301,8 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti uploadSalesSingle(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_single"); - io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); - - // Verify we retrieved all the rows. - Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); - Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_SINGLE_DEFINITION); - - // Test the string versions of the methods - table = adapter.readTable("sales.sales_single", instructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_single"); + final io.deephaven.engine.table.Table table = tableAdapter.table(instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -333,10 +314,8 @@ public void testOpenTableS3Only() throws ExecutionException, InterruptedExceptio uploadSalesPartitioned(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); + final io.deephaven.engine.table.Table table = tableAdapter.table(instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -353,10 +332,8 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); + final io.deephaven.engine.table.Table table = tableAdapter.table(instructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -380,14 +357,12 @@ public void testOpenTablePartitionTypeException() { .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); for (Runnable runnable : Arrays.asList( - () -> adapter.readTable(tableId, localInstructions), - () -> adapter.getTableDefinition(tableId, localInstructions), - () -> adapter.getTableDefinitionTable(tableId, localInstructions))) { + () -> tableAdapter.table(localInstructions), + () -> tableAdapter.definition(localInstructions), + () -> tableAdapter.definitionTable(localInstructions))) { try { runnable.run(); Assert.statementNeverExecuted("Expected an exception for missing columns"); @@ -423,10 +398,8 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); + final io.deephaven.engine.table.Table table = tableAdapter.table(localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -452,10 +425,8 @@ public void testSkippedPartitioningColumn() throws ExecutionException, Interrupt .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); + final io.deephaven.engine.table.Table table = tableAdapter.table(localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -481,10 +452,8 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); + final io.deephaven.engine.table.Table table = tableAdapter.table(localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -501,10 +470,8 @@ public void testZeroPartitioningColumns() throws ExecutionException, Interrupted .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); + final io.deephaven.engine.table.Table table = tableAdapter.table(localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -528,14 +495,12 @@ public void testIncorrectPartitioningColumns() throws ExecutionException, Interr .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); for (Runnable runnable : Arrays.asList( - () -> adapter.readTable(tableId, localInstructions), - () -> adapter.getTableDefinition(tableId, localInstructions), - () -> adapter.getTableDefinitionTable(tableId, localInstructions))) { + () -> tableAdapter.table(localInstructions), + () -> tableAdapter.definition(localInstructions), + () -> tableAdapter.definitionTable(localInstructions))) { try { runnable.run(); Assert.statementNeverExecuted("Expected an exception for missing columns"); @@ -563,14 +528,12 @@ public void testMissingPartitioningColumns() { .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); for (Runnable runnable : Arrays.asList( - () -> adapter.readTable(tableId, localInstructions), - () -> adapter.getTableDefinition(tableId, localInstructions), - () -> adapter.getTableDefinitionTable(tableId, localInstructions))) { + () -> tableAdapter.table(localInstructions), + () -> tableAdapter.definition(localInstructions), + () -> tableAdapter.definitionTable(localInstructions))) { try { runnable.run(); Assert.statementNeverExecuted("Expected an exception for missing columns"); @@ -591,10 +554,8 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); + final io.deephaven.engine.table.Table table = tableAdapter.table(localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -609,10 +570,8 @@ public void testOpenTableColumnLegalization() throws ExecutionException, Interru .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_renamed"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_renamed"); + final io.deephaven.engine.table.Table table = tableAdapter.table(localInstructions); // Verify we retrieved all the rows. Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); @@ -630,11 +589,10 @@ public void testOpenTableColumnLegalizationRename() .putColumnRenames("Units/Sold", "Units_Sold") .build(); - final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_renamed"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_renamed"); + final io.deephaven.engine.table.Table table = tableAdapter.table(localInstructions); final TableDefinition expected = TableDefinition.of( ColumnDefinition.ofString("Region_Name"), @@ -662,14 +620,12 @@ public void testOpenTableColumnLegalizationPartitionException() { .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); for (Runnable runnable : Arrays.asList( - () -> adapter.readTable(tableId, localInstructions), - () -> adapter.getTableDefinition(tableId, localInstructions), - () -> adapter.getTableDefinitionTable(tableId, localInstructions))) { + () -> tableAdapter.table(localInstructions), + () -> tableAdapter.definition(localInstructions), + () -> tableAdapter.definitionTable(localInstructions))) { try { runnable.run(); Assert.statementNeverExecuted("Expected an exception for missing columns"); @@ -692,10 +648,8 @@ public void testOpenTableColumnRenamePartitioningColumns() .build(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned"); - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, localInstructions); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_partitioned"); + final io.deephaven.engine.table.Table table = tableAdapter.table(localInstructions); final TableDefinition expected = TableDefinition.of( ColumnDefinition.ofString("Region"), @@ -716,39 +670,37 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final List snapshots = adapter.listSnapshots(tableId); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); + final List snapshots = tableAdapter.listSnapshots(); // Verify we retrieved all the rows. final io.deephaven.engine.table.Table table0 = - adapter.readTable(tableId, snapshots.get(0).snapshotId(), instructions); + tableAdapter.table(snapshots.get(0).snapshotId(), instructions); Assert.eq(table0.size(), "table0.size()", 18073, "expected rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table1 = - adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions); + tableAdapter.table(snapshots.get(1).snapshotId(), instructions); Assert.eq(table1.size(), "table1.size()", 54433, "expected rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table2 = - adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions); + tableAdapter.table(snapshots.get(2).snapshotId(), instructions); Assert.eq(table2.size(), "table2.size()", 72551, "expected rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table3 = - adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions); + tableAdapter.table(snapshots.get(3).snapshotId(), instructions); Assert.eq(table3.size(), "table3.size()", 100_000, "expected rows in the table"); Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table4 = - adapter.readTable(tableId, snapshots.get(4).snapshotId(), instructions); + tableAdapter.table(snapshots.get(4).snapshotId(), instructions); Assert.eq(table4.size(), "table4.size()", 100_000, "expected rows in the table"); Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); final io.deephaven.engine.table.Table table5 = - adapter.readTable(tableId, snapshots.get(5).snapshotId(), instructions); + tableAdapter.table(snapshots.get(5).snapshotId(), instructions); Assert.eq(table5.size(), "table5.size()", 0, "expected rows in the table"); Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @@ -758,60 +710,31 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx uploadSalesMulti(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final List snapshots = adapter.listSnapshots(tableId); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); + final List snapshots = tableAdapter.listSnapshots(); // Verify we retrieved all the rows. - io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions); + io.deephaven.engine.table.Table table0 = tableAdapter.table(snapshots.get(0), instructions); Assert.eq(table0.size(), "table0.size()", 18073, "expected rows in the table"); Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); - io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1), instructions); + io.deephaven.engine.table.Table table1 = tableAdapter.table(snapshots.get(1), instructions); Assert.eq(table1.size(), "table1.size()", 54433, "expected rows in the table"); Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); - io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2), instructions); + io.deephaven.engine.table.Table table2 = tableAdapter.table(snapshots.get(2), instructions); Assert.eq(table2.size(), "table2.size()", 72551, "expected rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); - io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3), instructions); + io.deephaven.engine.table.Table table3 = tableAdapter.table(snapshots.get(3), instructions); Assert.eq(table3.size(), "table3.size()", 100_000, "expected rows in the table"); Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION); - io.deephaven.engine.table.Table table4 = adapter.readTable(tableId, snapshots.get(4), instructions); + io.deephaven.engine.table.Table table4 = tableAdapter.table(snapshots.get(4), instructions); Assert.eq(table4.size(), "table4.size()", 100_000, "expected rows in the table"); Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); - io.deephaven.engine.table.Table table5 = adapter.readTable(tableId, snapshots.get(5), instructions); - Assert.eq(table5.size(), "table5.size()", 0, "expected rows in the table"); - Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); - - // Test the string versions of the methods - - // Verify we retrieved all the rows. - table0 = adapter.readTable("sales.sales_multi", snapshots.get(0).snapshotId(), instructions); - Assert.eq(table0.size(), "table0.size()", 18073, "expected rows in the table"); - Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); - - table1 = adapter.readTable("sales.sales_multi", snapshots.get(1).snapshotId(), instructions); - Assert.eq(table1.size(), "table1.size()", 54433, "expected rows in the table"); - Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION); - - table2 = adapter.readTable("sales.sales_multi", snapshots.get(2).snapshotId(), instructions); - Assert.eq(table2.size(), "table2.size()", 72551, "expected rows in the table"); - Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); - - table3 = adapter.readTable("sales.sales_multi", snapshots.get(3).snapshotId(), instructions); - Assert.eq(table3.size(), "table3.size()", 100_000, "expected rows in the table"); - Assert.equals(table3.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION); - - table4 = adapter.readTable("sales.sales_multi", snapshots.get(4).snapshotId(), instructions); - Assert.eq(table4.size(), "table4.size()", 100_000, "expected rows in the table"); - Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION); - - table5 = adapter.readTable("sales.sales_multi", snapshots.get(5).snapshotId(), instructions); + io.deephaven.engine.table.Table table5 = tableAdapter.table(snapshots.get(5), instructions); Assert.eq(table5.size(), "table5.size()", 0, "expected rows in the table"); Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @@ -821,12 +744,10 @@ public void testOpenAllTypesTable() throws ExecutionException, InterruptedExcept uploadAllTypes(); final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sample"); - final TableIdentifier tableId = TableIdentifier.of(ns, "all_types"); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sample.all_types"); + final io.deephaven.engine.table.Table table = tableAdapter.table(instructions).select(); // Verify we retrieved all the rows. - final io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions).select(); Assert.eq(table.size(), "table.size()", 10, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", ALL_TYPES_DEF); } @@ -834,56 +755,52 @@ public void testOpenAllTypesTable() throws ExecutionException, InterruptedExcept @Test public void testTableDefinition() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final List snapshots = adapter.listSnapshots(tableId); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); + final List snapshots = tableAdapter.listSnapshots(); // Use string and current snapshot - TableDefinition tableDef = adapter.getTableDefinition("sales.sales_multi", null); + TableDefinition tableDef = tableAdapter.definition(); Assert.equals(tableDef, "tableDef", SALES_MULTI_DEFINITION); // Use TableIdentifier and Snapshot - tableDef = adapter.getTableDefinition(tableId, null); + tableDef = tableAdapter.definition(instructions); Assert.equals(tableDef, "tableDef", SALES_MULTI_DEFINITION); // Use string and long snapshot ID - tableDef = adapter.getTableDefinition("sales.sales_multi", snapshots.get(0).snapshotId(), null); + tableDef = tableAdapter.definition(snapshots.get(0).snapshotId(), null); Assert.equals(tableDef, "tableDef", SALES_MULTI_DEFINITION); // Use TableIdentifier and Snapshot - tableDef = adapter.getTableDefinition(tableId, snapshots.get(0), null); + tableDef = tableAdapter.definition(snapshots.get(0), null); Assert.equals(tableDef, "tableDef", SALES_MULTI_DEFINITION); } @Test public void testTableDefinitionTable() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); - - final Namespace ns = Namespace.of("sales"); - final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi"); - final List snapshots = adapter.listSnapshots(tableId); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); + final List snapshots = tableAdapter.listSnapshots(); // Use string and current snapshot - Table tableDefTable = adapter.getTableDefinitionTable("sales.sales_multi", null); + Table tableDefTable = tableAdapter.definitionTable(); Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "expected rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); // Use TableIdentifier and Snapshot - tableDefTable = adapter.getTableDefinitionTable(tableId, null); + tableDefTable = tableAdapter.definitionTable(instructions); Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "expected rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); // Use string and long snapshot ID - tableDefTable = adapter.getTableDefinitionTable("sales.sales_multi", snapshots.get(0).snapshotId(), null); + tableDefTable = tableAdapter.definitionTable(snapshots.get(0).snapshotId(), null); Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "expected rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); // Use TableIdentifier and Snapshot - tableDefTable = adapter.getTableDefinitionTable(tableId, snapshots.get(0), null); + tableDefTable = tableAdapter.definitionTable(snapshots.get(0), null); Assert.eq(tableDefTable.size(), "tableDefTable.size()", 5, "expected rows in the table"); Assert.equals(tableDefTable.getDefinition(), "tableDefTable.getDefinition()", META_DEF); @@ -892,6 +809,7 @@ public void testTableDefinitionTable() { @Test public void testTableDefinitionWithInstructions() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); IcebergInstructions localInstructions = IcebergInstructions.builder() .dataInstructions(instructions.dataInstructions().get()) @@ -910,7 +828,7 @@ public void testTableDefinitionWithInstructions() { ColumnDefinition.ofTime("OrderDate")); // Use string and current snapshot - TableDefinition tableDef = adapter.getTableDefinition("sales.sales_multi", localInstructions); + TableDefinition tableDef = tableAdapter.definition(localInstructions); Assert.equals(tableDef, "tableDef", renamed); ///////////////////////////////////////////////////// @@ -926,7 +844,7 @@ public void testTableDefinitionWithInstructions() { .build(); // Use string and current snapshot - tableDef = adapter.getTableDefinition("sales.sales_multi", localInstructions); + tableDef = tableAdapter.definition(localInstructions); Assert.equals(tableDef, "tableDef", userTableDef); } @@ -942,11 +860,9 @@ public void testManualRefreshingTable() throws ExecutionException, InterruptedEx .build(); final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); - final List snapshots = tableAdapter.listSnapshots(); final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); - final IcebergTableImpl table = (IcebergTableImpl) tableAdapter.table(snapshots.get(0), localInstructions); // Initial size diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index 9c1e659c504..134a16a8671 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -189,6 +189,16 @@ public Table tables(@NotNull final String... namespace) { return tables(Namespace.of(namespace)); } + /** + * Load an Iceberg table from the catalog. + * + * @param tableIdentifier The identifier of the table to load. + * @return The {@link IcebergTableAdapter table adapter} for the Iceberg table. + */ + public IcebergTableAdapter loadTable(final String tableIdentifier) { + return loadTable(TableIdentifier.parse(tableIdentifier)); + } + /** * Load an Iceberg table from the catalog. * @@ -503,16 +513,6 @@ public IcebergTable readTable( return tableAdapter.table(tableSnapshot, instructions); } - /** - * Load an Iceberg table from the catalog. - * - * @param tableIdentifier The identifier of the table to load. - * @return The {@link IcebergTableAdapter table adapter} for the Iceberg table. - */ - public IcebergTableAdapter loadTable(final String tableIdentifier) { - return loadTable(TableIdentifier.parse(tableIdentifier)); - } - /** * Returns the underlying Iceberg {@link Catalog catalog} used by this adapter. */ diff --git a/extensions/source-support/src/test/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionTest.java b/extensions/source-support/src/test/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionTest.java index 6e23ee92523..83ee282bc25 100644 --- a/extensions/source-support/src/test/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionTest.java +++ b/extensions/source-support/src/test/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionTest.java @@ -97,7 +97,8 @@ private static Table makeRegioned( constituents[0].getDefinition(), "Test SimpleSourceTable", RegionedTableComponentFactoryImpl.INSTANCE, - new TableBackedTableLocationProvider(registrar, false, constituents), + new TableBackedTableLocationProvider(registrar, false, TableUpdateMode.STATIC, TableUpdateMode.STATIC, + constituents), registrar).coalesce(); } From d69ddcd39d829b8e78f444b3d801bd6ef5c39663 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 4 Oct 2024 14:34:42 -0700 Subject: [PATCH 43/56] Addressed PR comments and test failures. --- .../table/impl/ColumnSourceManager.java | 2 +- .../table/impl/SourcePartitionedTable.java | 21 ++- .../engine/table/impl/SourceTable.java | 14 +- .../engine/table/impl/TableUpdateMode.java | 2 +- .../impl/locations/TableLocationProvider.java | 17 +- .../locations/impl/AbstractTableLocation.java | 2 +- .../impl/AbstractTableLocationProvider.java | 172 +++++++++--------- .../impl/CompositeTableDataService.java | 72 ++++---- .../impl/FilteredTableDataService.java | 36 ++-- .../impl/SingleTableLocationProvider.java | 25 ++- .../impl/TableLocationSubscriptionBuffer.java | 10 - .../util/ExecutorTableDataRefreshService.java | 4 +- .../regioned/RegionedColumnSourceManager.java | 5 +- .../impl/TestPartitionAwareSourceTable.java | 6 + .../table/impl/TestSimpleSourceTable.java | 12 +- ...ebergAutoRefreshTableLocationProvider.java | 5 + .../IcebergStaticTableLocationProvider.java | 5 - 17 files changed, 210 insertions(+), 200 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java index 091f61eff85..3a03dd69a0c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java @@ -117,7 +117,7 @@ public interface ColumnSourceManager extends LivenessNode { void removeLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey); /** - * Get a list of Table attributes that can be applied to the output source table, given the update modes of the + * Get a map of Table attributes that can be applied to the output source table, given the update modes of the * underlying table location provider. * * @param tableUpdateMode The update mode of the table location set diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index 20089506487..b329d110bdf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -16,7 +16,9 @@ import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; import io.deephaven.engine.table.iterators.ChunkedObjectColumnIterator; +import io.deephaven.engine.updategraph.NotificationQueue; import io.deephaven.engine.updategraph.UpdateCommitter; +import io.deephaven.engine.updategraph.UpdateGraph; import io.deephaven.engine.updategraph.UpdateSourceCombiner; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode; import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedQueue; @@ -74,7 +76,8 @@ public SourcePartitionedTable( false); } - private static final class UnderlyingTableMaintainer extends ReferenceCountedLivenessNode { + private static final class UnderlyingTableMaintainer extends ReferenceCountedLivenessNode + implements NotificationQueue.Dependency { private final TableDefinition constituentDefinition; private final UnaryOperator
    applyTablePermissions; @@ -340,6 +343,22 @@ private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpd resultLocationTables.setNull(deletedRows); return deletedRows; } + + @Override + public boolean satisfied(final long step) { + if (refreshCombiner == null) { + throw new UnsupportedOperationException("This method should not be called when result is static"); + } + return refreshCombiner.satisfied(step); + } + + @Override + public UpdateGraph getUpdateGraph() { + if (refreshCombiner == null) { + throw new UnsupportedOperationException("This method should not be called when result is static"); + } + return refreshCombiner.getUpdateGraph(); + } } private static final class PendingLocationState extends IntrusiveDoublyLinkedNode.Impl { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index c04ac6033fd..28d8f75822d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -106,13 +106,11 @@ public abstract class SourceTable> exte } } - if (isRefreshing) { - setRefreshing(true); - // Given the location provider's update modes, retrieve applicable table attributes from the CSM - columnSourceManager.getTableAttributes( - locationProvider.getUpdateMode(), - locationProvider.getLocationUpdateMode()).forEach(this::setAttribute); - } + setRefreshing(isRefreshing); + // Given the location provider's update modes, retrieve and set applicable table attributes from the CSM + columnSourceManager.getTableAttributes( + locationProvider.getUpdateMode(), + locationProvider.getLocationUpdateMode()).forEach(this::setAttribute); } /** @@ -261,7 +259,7 @@ protected void instrumentedRefresh() { return; } - Assert.eqTrue(update.shifted().empty(), "update.shifted().empty()"); + Assert.assertion(update.shifted().empty(), "update.shifted().empty()"); rowSet.remove(update.removed()); rowSet.insert(update.added()); notifyListeners(update); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java index e4cf21df63b..eb3529c2ff3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java @@ -5,7 +5,7 @@ /** * Records the guarantees that the table offers regarding addition and removal of data elements. This can apply to - * {@link SourceTable sourc table} location additions and removals or row additions and removals within a location. + * {@link SourceTable source table} location additions and removals or row additions and removals within a location. */ public enum TableUpdateMode { STATIC, APPEND_ONLY, ADD_ONLY, ADD_REMOVE; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java index 18faf0f9eae..cda4c542264 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java @@ -5,6 +5,7 @@ import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.table.impl.TableUpdateMode; +import io.deephaven.util.annotations.TestUseOnly; import io.deephaven.util.type.NamedImplementation; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -44,21 +45,6 @@ public interface TableLocationProvider extends NamedImplementation { * ShiftObliviousListener interface for anything that wants to know about new table location keys. */ interface Listener extends BasicTableDataListener { - - /** - * Begin a transaction that collects location key additions and removals to be processed atomically. - * - * @param token A token to identify the transaction. - */ - void beginTransaction(@NotNull Object token); - - /** - * End the transaction and process the location changes. - * - * @param token A token to identify the transaction. - */ - void endTransaction(@NotNull Object token); - /** * Notify the listener of a {@link LiveSupplier} encountered while initiating or * maintaining the location subscription. This should occur at most once per location, but the order of delivery @@ -149,6 +135,7 @@ default void handleTableLocationKeysUpdate( * size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is * guaranteed to succeed for all results. */ + @TestUseOnly default Collection getTableLocationKeys() { final List keys = new ArrayList<>(); getTableLocationKeys(trackedKey -> keys.add(trackedKey.get())); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index 75eb8be3616..524dd721096 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java @@ -249,7 +249,7 @@ public final BasicDataIndex getDataIndex(@NotNull final String... columns) { // ------------------------------------------------------------------------------------------------------------------ /** - * The reference count has reached zero or we are being GC'd, we can clear this location and release any resources. + * The reference count has reached zero, we can clear this location and release any resources. */ protected void destroy() { handleUpdate(null, System.currentTimeMillis()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 7cb5d82860a..32bddcb1c92 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.locations.impl; +import io.deephaven.base.Pair; import io.deephaven.base.verify.Assert; import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; @@ -18,6 +19,7 @@ import java.util.*; import java.util.function.Consumer; import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.stream.Collectors; /** @@ -77,28 +79,18 @@ synchronized void removeLocationKey(ImmutableTableLocationKey locationKey) { } } - private static class TrackedKeySupplier extends ReferenceCountedLivenessNode + private class TrackedKeySupplier extends ReferenceCountedLivenessNode implements LiveSupplier { - enum State { - /** The key is included in the live set for new subscribers. */ - ACTIVE, - /** The key was removed from the live list and will not be provided to new subscribers */ - INACTIVE - } private final ImmutableTableLocationKey key; - private final Consumer zeroCountConsumer; - private TableLocation tableLocation; - private State state; + private volatile TableLocation tableLocation; + private volatile boolean active; - TrackedKeySupplier( - final ImmutableTableLocationKey key, - final Consumer zeroCountConsumer) { + TrackedKeySupplier(@NotNull final ImmutableTableLocationKey key) { super(false); this.key = key; - this.zeroCountConsumer = zeroCountConsumer; - state = State.ACTIVE; + active = true; } @Override @@ -107,30 +99,40 @@ public ImmutableTableLocationKey get() { } /** - * This {@link TrackedKeySupplier} should manage the given {@link TableLocation} and store a reference to it. - * - * @param tableLocation The {@link TableLocation} to manage. + * Create the {@link TableLocation} for this key, if it has not already been created, and return it. */ - public synchronized void setTableLocation(final TableLocation tableLocation) { - Assert.eqNull(this.tableLocation, "this.tableLocation"); - manage(tableLocation); - this.tableLocation = tableLocation; + private TableLocation getTableLocation() { + TableLocation localTableLocation; + if ((localTableLocation = tableLocation) == null) { + synchronized (this) { + if ((localTableLocation = tableLocation) == null) { + // Make a new location, have the tracked key manage it, then store the location in the tracked + // key. + tableLocation = localTableLocation = makeTableLocation(key); + manage(tableLocation); + } + } + } + return localTableLocation; } /** - * Change the state of this object to {@link State#INACTIVE}, indicates that this key is not included in the - * live set for new subscribers. + * Mark this supplier inactive. Indicates that this key is not included in the live set for new subscribers. */ - public synchronized void deactivate() { - this.state = State.INACTIVE; + private void deactivate() { + active = false; + } + + private boolean active() { + return active; } @Override - protected synchronized void destroy() { + protected void destroy() { super.destroy(); + Assert.assertion(!active, "!active"); tableLocation = null; - state = State.INACTIVE; - zeroCountConsumer.accept(this); + releaseLocationKey(this); } } @@ -141,7 +143,7 @@ protected synchronized void destroy() { /** * Map from {@link TableLocationKey} to a {@link TrackedKeySupplier}. - * + *

    * These values will not be cleared until all references to the {@link TableLocation} have been released by its * managers (i.e. {@link TableLocationSubscriptionBuffer subscriptions} and * {@link io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSourceManager column source managers}). @@ -150,9 +152,9 @@ protected synchronized void destroy() { new KeyedObjectHashMap<>(TableLocationKeyDefinition.INSTANCE); /** - * TLP are (currently) outside of liveness scopes, but they need to manage liveness referents to prevent them from - * going out of scope. The {@link StandaloneLivenessManager manager} will maintain the references until GC'd or the - * referents are not needed byt the TLP. + * TLPs are (currently) not LivenessArtifacts, hence they are not managed by the enclosing LivenessScope, but they + * still need to manage liveness referents to control their lifecycles. The {@link StandaloneLivenessManager + * manager} will maintain the references until GC'd or the referents are not needed by the TLP. */ private final StandaloneLivenessManager livenessManager; @@ -226,7 +228,7 @@ protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider synchronized (tableLocationKeyMap) { final List> keySuppliers = tableLocationKeyMap.values().stream() - .filter(iks -> iks.state == TrackedKeySupplier.State.ACTIVE) + .filter(TrackedKeySupplier::active) .collect(Collectors.toList()); listener.handleTableLocationKeysUpdate(keySuppliers, List.of()); } @@ -264,37 +266,49 @@ protected void endTransaction(@NotNull final Object token) { return; } - final Collection> addedKeys = - new ArrayList<>(transaction.locationsAdded.size()); - final Collection> removedKeys = - new ArrayList<>(transaction.locationsRemoved.size()); - - synchronized (tableLocationKeyMap) { - for (ImmutableTableLocationKey locationKey : transaction.locationsRemoved) { - final TrackedKeySupplier trackedKey = tableLocationKeyMap.get(locationKey); - if (trackedKey == null) { - // Not an error to remove a key multiple times (or a keu that was never added) - continue; - } - trackedKey.deactivate(); - - // Pass this removed key to the subscribers - removedKeys.add(trackedKey); - } - - for (ImmutableTableLocationKey locationKey : transaction.locationsAdded) { - locationCreatedRecorder = false; - final TrackedKeySupplier result = tableLocationKeyMap.putIfAbsent(locationKey, this::observeInsert); - visitLocationKey(locationKey); - if (locationCreatedRecorder) { - verifyPartitionKeys(locationKey); - addedKeys.add(result); - } - } - } - - if (subscriptions != null && (!addedKeys.isEmpty() || !removedKeys.isEmpty())) { + // This may need to run this under the subscriptions lock. + final Supplier>, Collection>>> applyTransaction = + () -> { + final Collection> addedKeys = + new ArrayList<>(transaction.locationsAdded.size()); + final Collection> removedKeys = + new ArrayList<>(transaction.locationsRemoved.size()); + + synchronized (tableLocationKeyMap) { + for (ImmutableTableLocationKey locationKey : transaction.locationsRemoved) { + final TrackedKeySupplier trackedKey = tableLocationKeyMap.get(locationKey); + if (trackedKey == null) { + // Not an error to remove a key multiple times (or a key that was never added) + continue; + } + trackedKey.deactivate(); + + // Pass this removed key to the subscribers + removedKeys.add(trackedKey); + } + + for (ImmutableTableLocationKey locationKey : transaction.locationsAdded) { + locationCreatedRecorder = false; + final TrackedKeySupplier result = + tableLocationKeyMap.putIfAbsent(locationKey, this::observeInsert); + visitLocationKey(locationKey); + if (locationCreatedRecorder) { + verifyPartitionKeys(locationKey); + addedKeys.add(result); + } + } + } + return new Pair<>(addedKeys, removedKeys); + }; + + final Collection> removedKeys; + if (supportsSubscriptions()) { synchronized (subscriptions) { + final Collection> addedKeys; + final Pair>, Collection>> result = + applyTransaction.get(); + addedKeys = result.getFirst(); + removedKeys = result.getSecond(); // Push the notifications to the subscribers if (subscriptions.deliverNotification( Listener::handleTableLocationKeysUpdate, @@ -304,6 +318,8 @@ protected void endTransaction(@NotNull final Object token) { onEmpty(); } } + } else { + removedKeys = applyTransaction.get().getSecond(); } // Release the keys that were removed after we have delivered the notifications and the // subscribers have had a chance to process them @@ -360,7 +376,7 @@ protected final void handleTableLocationKeyAdded( verifyPartitionKeys(result.get()); if (subscriptions.deliverNotification( Listener::handleTableLocationKeyAdded, - (LiveSupplier) result, + result, true)) { onEmpty(); } @@ -402,7 +418,7 @@ protected void handleTableLocationKeyRemoved( trackedKey.deactivate(); if (subscriptions.deliverNotification( Listener::handleTableLocationKeyRemoved, - (LiveSupplier) trackedKey, + trackedKey, true)) { onEmpty(); } @@ -419,6 +435,7 @@ protected void handleTableLocationKeyRemoved( * * @param locationKey The {@link TableLocationKey} that was visited. */ + @SuppressWarnings("unused") protected void visitLocationKey(@NotNull final TableLocationKey locationKey) {} @NotNull @@ -483,8 +500,9 @@ public void getTableLocationKeys( // Lock the live set and deliver a copy to the listener after filtering. synchronized (tableLocationKeyMap) { tableLocationKeyMap.values().stream() - .filter(ttlk -> ttlk.state == TrackedKeySupplier.State.ACTIVE) - .filter(ttlk -> filter.test(ttlk.get())).forEach(consumer); + .filter(TrackedKeySupplier::active) + .filter(ttlk -> filter.test(ttlk.get())) + .forEach(consumer); } } @@ -496,21 +514,11 @@ public final boolean hasTableLocationKey(@NotNull final TableLocationKey tableLo @Override @Nullable public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey tableLocationKey) { - TrackedKeySupplier trackedKey = tableLocationKeyMap.get(tableLocationKey); + final TrackedKeySupplier trackedKey = tableLocationKeyMap.get(tableLocationKey); if (trackedKey == null) { return null; } - // See JavaDoc on tableLocations for background. - if (trackedKey.tableLocation == null) { - synchronized (trackedKey) { - if (trackedKey.tableLocation == null) { - // Make a new location, have the tracked key manage it, then store the location in the tracked key. - final TableLocation newLocation = makeTableLocation(trackedKey.get()); - trackedKey.setTableLocation(newLocation); - } - } - } - return trackedKey.tableLocation; + return trackedKey.getTableLocation(); } /** @@ -563,13 +571,13 @@ private static final class TableLocationKeyDefinition private TableLocationKeyDefinition() {} @Override - public TableLocationKey getKey(TrackedKeySupplier immutableKeySupplier) { + public TableLocationKey getKey(@NotNull final TrackedKeySupplier immutableKeySupplier) { return immutableKeySupplier.get(); } } private TrackedKeySupplier toTrackedKey(@NotNull final TableLocationKey locationKey) { - return new TrackedKeySupplier(locationKey.makeImmutable(), this::releaseLocationKey); + return new TrackedKeySupplier(locationKey.makeImmutable()); } private static boolean equals(Collection c1, Collection c2) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index 1424fc55cbc..d824961cadc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -132,43 +132,39 @@ private TableLocationProviderImpl(@NotNull final TableDataService[] inputService implementationName = "Composite-" + inputProviders; // Analyze the update modes of the input providers to determine the update mode of the composite provider. - // 1) If any providers are REFRESHING, the overall provider is REFRESHING - // 2) If any providers are ADD_ONLY or APPEND_ONLY, the overall provider is ADD_ONLY - // 3) If all providers are STATIC, the overall provider is STATIC + // The resultant mode is the most permissive mode of the input providers, with the exception that we will + // never return APPEND_ONLY. + boolean anyRemoves = false; boolean anyAdditions = false; - TableUpdateMode tmpMode = TableUpdateMode.STATIC; + boolean anyAppends = false; for (final TableLocationProvider provider : inputProviders) { if (provider.getUpdateMode().removeAllowed()) { - tmpMode = TableUpdateMode.ADD_REMOVE; - break; - } else if (provider.getUpdateMode().addAllowed()) { + anyRemoves = true; + } else if (provider.getLocationUpdateMode() == TableUpdateMode.ADD_ONLY) { anyAdditions = true; + } else if (provider.getLocationUpdateMode() == TableUpdateMode.APPEND_ONLY) { + anyAppends = true; } } - if (anyAdditions) { - tmpMode = TableUpdateMode.ADD_ONLY; - } - updateMode = tmpMode; + updateMode = anyRemoves ? TableUpdateMode.ADD_REMOVE + : (anyAdditions || anyAppends) ? TableUpdateMode.ADD_ONLY : TableUpdateMode.STATIC; // Analyze the location update modes of the input providers to determine the location update mode - // of the composite provider. - // 1) If any provider locations are REFRESHING, the overall provider location mode is REFRESHING - // 2) If any provider locations are ADD_ONLY or APPEND_ONLY, the overall provider location mode is ADD_ONLY - // 3) If all provider locations are STATIC, the overall provider location mode is STATIC - anyAdditions = false; - tmpMode = TableUpdateMode.STATIC; + // of the composite provider. The resultant mode is the most permissive mode of the input provider + // locations. + anyRemoves = anyAdditions = anyAppends = false; for (final TableLocationProvider provider : inputProviders) { if (provider.getLocationUpdateMode().removeAllowed()) { - tmpMode = TableUpdateMode.ADD_REMOVE; - break; - } else if (provider.getLocationUpdateMode().addAllowed()) { + anyRemoves = true; + } else if (provider.getLocationUpdateMode() == TableUpdateMode.ADD_ONLY) { anyAdditions = true; + } else if (provider.getLocationUpdateMode() == TableUpdateMode.APPEND_ONLY) { + anyAppends = true; } } - if (anyAdditions) { - tmpMode = TableUpdateMode.ADD_ONLY; - } - locationUpdateMode = tmpMode; + locationUpdateMode = anyRemoves ? TableUpdateMode.ADD_REMOVE + : anyAdditions ? TableUpdateMode.ADD_ONLY + : anyAppends ? TableUpdateMode.APPEND_ONLY : TableUpdateMode.STATIC; } @Override @@ -227,22 +223,24 @@ public void getTableLocationKeys( try (final SafeCloseable ignored = CompositeTableDataServiceConsistencyMonitor.INSTANCE.start()) { // Add all the location keys from the providers to the set, throw an exception if there are duplicates inputProviders.forEach(p -> p.getTableLocationKeys(tlk -> { - if (!locationKeys.add(tlk)) { + if (locationKeys.add(tlk)) { // Consume the key immediately (while the key is still managed by the input provider) consumer.accept(tlk); - final String overlappingProviders = inputProviders.stream() - .filter(inputProvider -> inputProvider.hasTableLocationKey(tlk.get())) - .map(TableLocationProvider::getName) - .collect(Collectors.joining(",")); - throw new TableDataException( - "Data Routing Configuration error: TableDataService elements overlap at location " + - tlk + - " in providers " + overlappingProviders + - ". Full TableDataService configuration:\n" + - Formatter - .formatTableDataService(CompositeTableDataService.this.toString())); - + return; } + // We have a duplicate key, throw a detailed exception + final String overlappingProviders = inputProviders.stream() + .filter(inputProvider -> inputProvider.hasTableLocationKey(tlk.get())) + .map(TableLocationProvider::getName) + .collect(Collectors.joining(",")); + throw new TableDataException( + "Data Routing Configuration error: TableDataService elements overlap at location " + + tlk + + " in providers " + overlappingProviders + + ". Full TableDataService configuration:\n" + + Formatter + .formatTableDataService(CompositeTableDataService.this.toString())); + }, filter)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index add5f0b7db4..62226c8c98a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -16,6 +16,7 @@ import java.util.WeakHashMap; import java.util.function.Consumer; import java.util.function.Predicate; +import java.util.stream.Collectors; /** * {@link TableDataService} implementation with support to filter the provided {@link TableLocation}s. @@ -166,15 +167,13 @@ public String getName() { @Override @NotNull public TableUpdateMode getUpdateMode() { - // TODO: I think locations can be added or removed at any time (with a dynamic filter) - return TableUpdateMode.ADD_REMOVE; + return inputProvider.getUpdateMode(); } @Override @NotNull public TableUpdateMode getLocationUpdateMode() { - // TODO: this is defensive, individual location contents might be static (e.g. Parquet files) - return TableUpdateMode.ADD_REMOVE; + return inputProvider.getLocationUpdateMode(); } } @@ -185,24 +184,6 @@ private FilteringListener(@NotNull final TableLocationProvider.Listener outputLi super(outputListener); } - @Override - public void beginTransaction(@NotNull final Object token) { - // Delegate to the wrapped listener. - final TableLocationProvider.Listener outputListener = getWrapped(); - if (outputListener != null) { - outputListener.beginTransaction(token); - } - } - - @Override - public void endTransaction(@NotNull final Object token) { - // Delegate to the wrapped listener. - final TableLocationProvider.Listener outputListener = getWrapped(); - if (outputListener != null) { - outputListener.endTransaction(token); - } - } - @Override public void handleTableLocationKeyAdded( @NotNull final LiveSupplier tableLocationKey) { @@ -225,11 +206,16 @@ public void handleTableLocationKeyRemoved( @Override public void handleTableLocationKeysUpdate( - @NotNull Collection> addedKeys, - @NotNull Collection> removedKeys) { + @NotNull final Collection> addedKeys, + @NotNull final Collection> removedKeys) { final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null) { - outputListener.handleTableLocationKeysUpdate(addedKeys, removedKeys); + // Produce filtered lists of added and removed keys. + final Collection> filteredAddedKeys = addedKeys.stream() + .filter(key -> locationKeyFilter.accept(key.get())).collect(Collectors.toList()); + final Collection> filteredRemovedKeys = removedKeys.stream() + .filter(key -> locationKeyFilter.accept(key.get())).collect(Collectors.toList()); + outputListener.handleTableLocationKeysUpdate(filteredAddedKeys, filteredRemovedKeys); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java index 79f50e1a9e1..977c4d8f0ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/SingleTableLocationProvider.java @@ -4,27 +4,28 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.engine.liveness.LiveSupplier; -import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; +import io.deephaven.engine.liveness.LivenessReferent; import io.deephaven.engine.table.impl.TableUpdateMode; import io.deephaven.engine.table.impl.locations.*; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.lang.ref.WeakReference; import java.util.function.Consumer; import java.util.function.Predicate; /** - * A {@link TableLocationProvider} that provides access to exactly one, previously-known {@link TableLocation}. + * A {@link TableLocationProvider} that provides access to exactly one, previously-known {@link TableLocation}. In + * contrast to {@link AbstractTableLocationProvider}, this class does not manage the liveness of the table location. + * Managment must be done externally (as in {@link io.deephaven.engine.table.impl.SourcePartitionedTable}). */ public final class SingleTableLocationProvider implements TableLocationProvider { private static final String IMPLEMENTATION_NAME = SingleTableLocationProvider.class.getSimpleName(); - private static class TrackedKeySupplier extends ReferenceCountedLivenessNode - implements LiveSupplier { + private static class TrackedKeySupplier implements LiveSupplier, LivenessReferent { final ImmutableTableLocationKey key; protected TrackedKeySupplier(final ImmutableTableLocationKey key) { - super(false); this.key = key; } @@ -32,6 +33,19 @@ protected TrackedKeySupplier(final ImmutableTableLocationKey key) { public ImmutableTableLocationKey get() { return key; } + + @Override + public boolean tryRetainReference() { + return true; + } + + @Override + public void dropReference() {} + + @Override + public WeakReference getWeakReference() { + return new WeakReference<>(this); + } } private final TrackedKeySupplier immutableKeySupplier; @@ -45,7 +59,6 @@ public SingleTableLocationProvider( @NotNull final TableLocation tableLocation, final TableUpdateMode locationUpdateMode) { this.tableLocation = tableLocation; - // TODO: it seems like we should manage this, but SingleTableLocationProvider isn't a LivenessManager. immutableKeySupplier = new TrackedKeySupplier(tableLocation.getKey()); this.locationUpdateMode = locationUpdateMode; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java index 981cc11d720..e7a5fab9a59 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java @@ -132,16 +132,6 @@ public synchronized void reset() { // TableLocationProvider.Listener implementation // ------------------------------------------------------------------------------------------------------------------ - @Override - public void beginTransaction(@NotNull final Object token) { - throw new UnsupportedOperationException("Transactions are not supported by this provider."); - } - - @Override - public void endTransaction(@NotNull final Object token) { - throw new UnsupportedOperationException("Transactions are not supported by this provider."); - } - @Override public void handleTableLocationKeyAdded(@NotNull final LiveSupplier tableLocationKey) { synchronized (updateLock) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java index 3f0a4a862fb..87fa75bccfa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java @@ -158,7 +158,7 @@ public void cancel() { @Override public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( @NotNull final AbstractTableLocationProvider tableLocationProvider) { - return new ScheduledTableLocationProviderRefresh(tableLocationProvider, + return scheduleTableLocationProviderRefresh(tableLocationProvider, tableLocationProviderDefaultRefreshIntervalMillis); } @@ -172,7 +172,7 @@ public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( @Override public CancellableSubscriptionToken scheduleTableLocationRefresh( @NotNull final AbstractTableLocation tableLocation) { - return new ScheduledTableLocationRefresh(tableLocation, tableLocationDefaultRefreshIntervalMillis); + return scheduleTableLocationRefresh(tableLocation, tableLocationDefaultRefreshIntervalMillis); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 8ae4445ad41..14cace8f619 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -42,7 +42,7 @@ public class RegionedColumnSourceManager implements ColumnSourceManager, Delegat private static final Logger log = LoggerFactory.getLogger(RegionedColumnSourceManager.class); /** - * Whether this column source manager is serving a refreshing dynamic table. + * The liveness node to which this column source manager will delegate. */ private final LivenessNode livenessNode; @@ -116,7 +116,6 @@ public class RegionedColumnSourceManager implements ColumnSourceManager, Delegat /** * A reference to a delayed error notifier for the {@link #includedLocationsTable}, if one is pending. */ - @SuppressWarnings("unused") @ReferentialIntegrity private Runnable delayedErrorReference; @@ -743,7 +742,7 @@ private void regionAllocated(final int regionIndex) { public Map getTableAttributes( @NotNull TableUpdateMode tableUpdateMode, @NotNull TableUpdateMode tableLocationUpdateMode) { - final Map attributes = new LinkedHashMap<>(); + final Map attributes = new HashMap<>(); // NOTE: Current RegionedColumnSourceManager implementation appends new locations and does not reuse // region indices. This is important for the following attributes to be correct. diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java index 0e507df3c1e..d3fc36d80b4 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java @@ -144,6 +144,8 @@ public void setUp() throws Exception { { allowing(locationProvider).getUpdateMode(); will(returnValue(TableUpdateMode.ADD_REMOVE)); + allowing(locationProvider).getLocationUpdateMode(); + will(returnValue(TableUpdateMode.ADD_REMOVE)); } }); @@ -188,6 +190,10 @@ public void setUp() throws Exception { allowing(columnSourceManager).getWeakReference(); will(returnValue(new WeakReference<>(columnSourceManager))); allowing(columnSourceManager).dropReference(); + allowing(columnSourceManager).getTableAttributes(with(any(TableUpdateMode.class)), + with(any(TableUpdateMode.class))); + will(returnValue(Collections.EMPTY_MAP)); + } }); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java index c23c56f3e9c..48c994ebe4a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java @@ -74,7 +74,10 @@ public void setUp() throws Exception { checking(new Expectations() { { allowing(columnSourceManager).allLocations(); - will(returnValue(Collections.EMPTY_SET)); + will(returnValue(Collections.EMPTY_MAP)); + allowing(columnSourceManager).getTableAttributes(with(any(TableUpdateMode.class)), + with(any(TableUpdateMode.class))); + will(returnValue(Collections.EMPTY_MAP)); } }); @@ -103,6 +106,7 @@ public void setUp() throws Exception { will(returnValue(true)); allowing(keySupplier).getWeakReference(); will(returnValue(new WeakReference<>(keySupplier))); + allowing(keySupplier).retainReference(); allowing(keySupplier).dropReference(); allowing(locationProvider).getTableLocationKeys(); @@ -119,14 +123,16 @@ public Object invoke(Invocation invocation) { }); allowing(locationProvider).getTableLocation(with(StandaloneTableLocationKey.getInstance())); will(returnValue(tableLocation)); + allowing(locationProvider).getUpdateMode(); + will(returnValue(TableUpdateMode.STATIC)); + allowing(locationProvider).getLocationUpdateMode(); + will(returnValue(TableUpdateMode.STATIC)); allowing(tableLocation).supportsSubscriptions(); will(returnValue(true)); allowing(tableLocation).getKey(); will(returnValue(StandaloneTableLocationKey.getInstance())); allowing(locationProvider).supportsSubscriptions(); will(returnValue(true)); - allowing(locationProvider).getUpdateMode(); - will(returnValue(TableUpdateMode.ADD_REMOVE)); } }); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index 2b7415fc144..83dc57625d7 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -125,4 +125,9 @@ protected final void deactivateUnderlyingDataSource() { subscriptionToken = null; } } + + @Override + protected boolean matchSubscriptionToken(final T token) { + return token == subscriptionToken; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index 16c0f0a5d73..db070a15b1b 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -85,9 +85,4 @@ protected void deactivateUnderlyingDataSource() { throw new IllegalStateException( "deactivateUnderlyingDataSource() called on a static Iceberg table location provider"); } - - @Override - protected boolean matchSubscriptionToken(final T token) { - return false; - } } From 06a3bd58779c37bce405a1f9e0734e433fb9cefb Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 4 Oct 2024 15:55:18 -0700 Subject: [PATCH 44/56] More test failure fixes. --- .../java/io/deephaven/iceberg/layout/IcebergBaseLayout.java | 3 +++ .../java/io/deephaven/iceberg/util/IcebergTableAdapter.java | 2 +- .../src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java | 3 +-- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index d5905587235..67fee36dc23 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -136,6 +136,9 @@ private URI dataFileUri(@NotNull DataFile df) { @Override public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) { + if (snapshot == null) { + return; + } final Table table = tableAdapter.icebergTable(); try { // Retrieve the manifest files from the snapshot diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java index 87de7b07cd7..819309f64d7 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -360,7 +360,7 @@ public IcebergTable table( // Do we want the latest or a specific snapshot? final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); - final Schema schema = table.schemas().get(snapshot.schemaId()); + final Schema schema = snapshot == null ? table.schema() : table.schemas().get(snapshot.schemaId()); // Load the partitioning schema. final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index d08e1735db0..3a9c88d099d 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -43,7 +43,6 @@ void createEmptyTable() { table = catalogAdapter.readTable(myTableId, null); assertThat(table.getDefinition()).isEqualTo(expectedDefinition); } - // Note: this is failing w/ NPE, assumes that Snapshot is non-null. - // assertThat(table.isEmpty()).isTrue(); + assertThat(table.isEmpty()).isTrue(); } } From 91ba92c49d2400a25f09cfd03a19305a2b8943d3 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 7 Oct 2024 09:47:26 -0700 Subject: [PATCH 45/56] Liveness management re-ordering in SourcePartitionedTable --- .../io/deephaven/engine/table/impl/SourcePartitionedTable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index b329d110bdf..3be41bb5109 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java @@ -128,8 +128,8 @@ private UnderlyingTableMaintainer( if (needToRefreshLocations || refreshSizes) { result.setRefreshing(true); refreshCombiner = new UpdateSourceCombiner(result.getUpdateGraph()); - manage(refreshCombiner); result.addParentReference(this); + manage(refreshCombiner); } else { refreshCombiner = null; } From 73e8824c38dfb0955d80d7cea32da058de2f4536 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 9 Oct 2024 09:50:36 -0700 Subject: [PATCH 46/56] Addressing open PR comments. --- .../engine/table/impl/TableUpdateMode.java | 49 +++++++++++++++++++ .../impl/AbstractTableLocationProvider.java | 2 +- .../impl/CompositeTableDataService.java | 32 ++---------- .../impl/FilteredTableDataService.java | 6 +++ 4 files changed, 61 insertions(+), 28 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java index eb3529c2ff3..37aae2453c5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java @@ -3,6 +3,10 @@ // package io.deephaven.engine.table.impl; +import org.apache.commons.lang3.mutable.MutableBoolean; + +import java.util.stream.Stream; + /** * Records the guarantees that the table offers regarding addition and removal of data elements. This can apply to * {@link SourceTable source table} location additions and removals or row additions and removals within a location. @@ -10,6 +14,9 @@ public enum TableUpdateMode { STATIC, APPEND_ONLY, ADD_ONLY, ADD_REMOVE; + /** + * Returns true if the addition is allowed. + */ public boolean addAllowed() { switch (this) { case APPEND_ONLY: @@ -22,6 +29,9 @@ public boolean addAllowed() { } } + /** + * Returns true if the removal is allowed. + */ public boolean removeAllowed() { switch (this) { case ADD_REMOVE: @@ -31,4 +41,43 @@ public boolean removeAllowed() { return false; } } + + /** + * Returns the most permissive mode from the given stream of modes. Permissiveness is ranked as follows (from most + * to least permissive): + *

      + *
    • {@link #ADD_REMOVE}
    • + *
    • {@link #ADD_ONLY}
    • + *
    • {@link #APPEND_ONLY}
    • + *
    • {@link #STATIC}
    • + *
    + * + * @param modes a stream of modes + * @return the most permissive mode encountered in the stream + */ + public static TableUpdateMode mostPermissiveMode(Stream modes) { + // Analyze the location update modes of the input providers to determine the location update mode + // of the composite provider. The resultant mode is the most permissive mode of the input provider + // locations. + final MutableBoolean anyRemoves = new MutableBoolean(false); + final MutableBoolean anyAdditions = new MutableBoolean(false); + final MutableBoolean anyAppends = new MutableBoolean(false); + + modes.forEach(mode -> { + if (mode.removeAllowed()) { + anyRemoves.setTrue(); + } else if (mode == TableUpdateMode.ADD_ONLY) { + anyAdditions.setTrue(); + } else if (mode == TableUpdateMode.APPEND_ONLY) { + anyAppends.setTrue(); + } + }); + // @formatter:off + return + anyRemoves.booleanValue() ? TableUpdateMode.ADD_REMOVE + : anyAdditions.booleanValue() ? TableUpdateMode.ADD_ONLY + : anyAppends.booleanValue() ? TableUpdateMode.APPEND_ONLY + : TableUpdateMode.STATIC; + // @formatter:on + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java index 32bddcb1c92..c42a0be1b82 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java @@ -109,7 +109,7 @@ private TableLocation getTableLocation() { // Make a new location, have the tracked key manage it, then store the location in the tracked // key. tableLocation = localTableLocation = makeTableLocation(key); - manage(tableLocation); + manage(localTableLocation); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java index d824961cadc..199e6a2ca87 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java @@ -134,37 +134,15 @@ private TableLocationProviderImpl(@NotNull final TableDataService[] inputService // Analyze the update modes of the input providers to determine the update mode of the composite provider. // The resultant mode is the most permissive mode of the input providers, with the exception that we will // never return APPEND_ONLY. - boolean anyRemoves = false; - boolean anyAdditions = false; - boolean anyAppends = false; - for (final TableLocationProvider provider : inputProviders) { - if (provider.getUpdateMode().removeAllowed()) { - anyRemoves = true; - } else if (provider.getLocationUpdateMode() == TableUpdateMode.ADD_ONLY) { - anyAdditions = true; - } else if (provider.getLocationUpdateMode() == TableUpdateMode.APPEND_ONLY) { - anyAppends = true; - } - } - updateMode = anyRemoves ? TableUpdateMode.ADD_REMOVE - : (anyAdditions || anyAppends) ? TableUpdateMode.ADD_ONLY : TableUpdateMode.STATIC; + final TableUpdateMode tmpUpdateMode = TableUpdateMode.mostPermissiveMode( + inputProviders.stream().map(TableLocationProvider::getUpdateMode)); + updateMode = tmpUpdateMode == TableUpdateMode.APPEND_ONLY ? TableUpdateMode.ADD_ONLY : tmpUpdateMode; // Analyze the location update modes of the input providers to determine the location update mode // of the composite provider. The resultant mode is the most permissive mode of the input provider // locations. - anyRemoves = anyAdditions = anyAppends = false; - for (final TableLocationProvider provider : inputProviders) { - if (provider.getLocationUpdateMode().removeAllowed()) { - anyRemoves = true; - } else if (provider.getLocationUpdateMode() == TableUpdateMode.ADD_ONLY) { - anyAdditions = true; - } else if (provider.getLocationUpdateMode() == TableUpdateMode.APPEND_ONLY) { - anyAppends = true; - } - } - locationUpdateMode = anyRemoves ? TableUpdateMode.ADD_REMOVE - : anyAdditions ? TableUpdateMode.ADD_ONLY - : anyAppends ? TableUpdateMode.APPEND_ONLY : TableUpdateMode.STATIC; + locationUpdateMode = TableUpdateMode.mostPermissiveMode( + inputProviders.stream().map(TableLocationProvider::getLocationUpdateMode)); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java index 62226c8c98a..b604c7ba700 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java @@ -208,6 +208,8 @@ public void handleTableLocationKeyRemoved( public void handleTableLocationKeysUpdate( @NotNull final Collection> addedKeys, @NotNull final Collection> removedKeys) { + // NOTE: We are filtering the added and removed keys for every listener. We should consider refactoring to + // filter once and then notify all listeners with the filtered lists (similar to SubscriptionAggregator). final TableLocationProvider.Listener outputListener = getWrapped(); if (outputListener != null) { // Produce filtered lists of added and removed keys. @@ -215,6 +217,10 @@ public void handleTableLocationKeysUpdate( .filter(key -> locationKeyFilter.accept(key.get())).collect(Collectors.toList()); final Collection> filteredRemovedKeys = removedKeys.stream() .filter(key -> locationKeyFilter.accept(key.get())).collect(Collectors.toList()); + + if (filteredAddedKeys.isEmpty() && filteredRemovedKeys.isEmpty()) { + return; + } outputListener.handleTableLocationKeysUpdate(filteredAddedKeys, filteredRemovedKeys); } } From be689efe1c29aa0a3b48ff1d2614967da16c89da Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 15 Oct 2024 14:44:02 -0700 Subject: [PATCH 47/56] Many PR comments addressed. --- .../engine/table/impl/TableUpdateMode.java | 7 +- .../AbstractTableLocationProviderTest.java | 18 +- ...ebergAutoRefreshTableLocationProvider.java | 35 +- .../iceberg/layout/IcebergBaseLayout.java | 59 ++++ ...ergManualRefreshTableLocationProvider.java | 66 +--- .../IcebergStaticTableLocationProvider.java | 22 +- .../IcebergTableLocationProviderBase.java | 39 ++- .../iceberg/util/IcebergCatalogAdapter.java | 301 ------------------ .../iceberg/util/IcebergInstructions.java | 12 +- .../deephaven/iceberg/util/IcebergTable.java | 24 +- .../iceberg/util/IcebergTableAdapter.java | 77 +++-- .../iceberg/util/IcebergTableImpl.java | 4 +- .../iceberg/util/IcebergUpdateMode.java | 47 ++- .../deephaven/iceberg/CatalogAdapterTest.java | 8 +- py/server/deephaven/experimental/iceberg.py | 2 +- 15 files changed, 240 insertions(+), 481 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java index 37aae2453c5..45de9453e69 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableUpdateMode.java @@ -15,7 +15,7 @@ public enum TableUpdateMode { STATIC, APPEND_ONLY, ADD_ONLY, ADD_REMOVE; /** - * Returns true if the addition is allowed. + * Returns true if addition is allowed. */ public boolean addAllowed() { switch (this) { @@ -30,7 +30,7 @@ public boolean addAllowed() { } /** - * Returns true if the removal is allowed. + * Returns true if removal is allowed. */ public boolean removeAllowed() { switch (this) { @@ -56,9 +56,6 @@ public boolean removeAllowed() { * @return the most permissive mode encountered in the stream */ public static TableUpdateMode mostPermissiveMode(Stream modes) { - // Analyze the location update modes of the input providers to determine the location update mode - // of the composite provider. The resultant mode is the most permissive mode of the input provider - // locations. final MutableBoolean anyRemoves = new MutableBoolean(false); final MutableBoolean anyAdditions = new MutableBoolean(false); final MutableBoolean anyAppends = new MutableBoolean(false); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java index 84956ee4ac7..36e0fe2353f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java @@ -5,7 +5,6 @@ import io.deephaven.api.SortColumn; import io.deephaven.base.verify.Assert; -import io.deephaven.configuration.Configuration; import io.deephaven.engine.liveness.LiveSupplier; import io.deephaven.engine.liveness.StandaloneLivenessManager; import io.deephaven.engine.table.BasicDataIndex; @@ -14,9 +13,7 @@ import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider; import io.deephaven.engine.table.impl.locations.local.URITableLocationKey; import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; -import io.deephaven.io.logger.StreamLoggerImpl; import io.deephaven.test.types.OutOfBandTest; -import io.deephaven.util.process.ProcessEnvironment; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.junit.Test; @@ -92,17 +89,17 @@ protected void destroy() { @Override protected @NotNull ColumnLocation makeColumnLocation(@NotNull String name) { - return null; + throw new UnsupportedOperationException(); } @Override public @Nullable BasicDataIndex loadDataIndex(@NotNull String... columns) { - return null; + throw new UnsupportedOperationException(); } @Override public void refresh() { - Assert.statementNeverExecuted(); + throw new UnsupportedOperationException(); } @Override @@ -123,19 +120,10 @@ public boolean hasDataIndex(@NotNull String... columns) { @Override public void setUp() throws Exception { - if (null == ProcessEnvironment.tryGet()) { - ProcessEnvironment.basicServerInitialization(Configuration.getInstance(), - "AbstractTableLocationProviderTest", new StreamLoggerImpl()); - } super.setUp(); setExpectError(false); } - @Override - public void tearDown() throws Exception { - super.tearDown(); - } - private List createKeys(final int count) { final List keys = new ArrayList<>(); for (int i = 0; i < count; i++) { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index 83dc57625d7..8fcc1172c48 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -14,9 +14,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; -import java.util.HashSet; -import java.util.Set; - /** *

    * Automatically refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location @@ -68,44 +65,24 @@ public String getImplementationName() { @Override public synchronized void refresh() { - adapter.refresh(); - final Snapshot latestSnapshot = adapter.currentSnapshot(); - if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) { - locationKeyFinder.snapshot = latestSnapshot; - refreshSnapshot(); + if (locationKeyFinder.maybeUpdateSnapshot()) { + refreshLocations(); } } @Override public void update() { - throw new IllegalStateException("An automatically refreshing Iceberg table cannot be manually updated"); + throw new UnsupportedOperationException("Automatically refreshing Iceberg tables cannot be manually updated"); } @Override public void update(long snapshotId) { - throw new IllegalStateException("An automatically refreshing Iceberg table cannot be manually updated"); + throw new UnsupportedOperationException("Automatically refreshing Iceberg tables cannot be manually updated"); } @Override public void update(Snapshot snapshot) { - throw new IllegalStateException("An automatically refreshing Iceberg table cannot be manually updated"); - } - - /** - * Refresh the table location provider with the latest snapshot from the catalog. This method will identify new - * locations and removed locations. - */ - private void refreshSnapshot() { - beginTransaction(this); - final Set missedKeys = new HashSet<>(); - getTableLocationKeys(ttlk -> missedKeys.add(ttlk.get())); - locationKeyFinder.findKeys(tableLocationKey -> { - missedKeys.remove(tableLocationKey); - handleTableLocationKeyAdded(tableLocationKey, this); - }); - missedKeys.forEach(tlk -> handleTableLocationKeyRemoved(tlk, this)); - endTransaction(this); - setInitialized(); + throw new UnsupportedOperationException("Automatically refreshing Iceberg tables cannot be manually updated"); } // ------------------------------------------------------------------------------------------------------------------ @@ -114,7 +91,7 @@ private void refreshSnapshot() { @Override protected final void activateUnderlyingDataSource() { - refreshSnapshot(); + refresh(); subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this, refreshIntervalMs); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index 67fee36dc23..1408c07d70e 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -166,4 +166,63 @@ public synchronized void findKeys(@NotNull final Consumer snapshot.sequenceNumber()) { + snapshot = latestSnapshot; + return true; + } + return false; + } + + /** + * Update the snapshot to the user specified snapshot. This will fail with an {@link IllegalArgumentException} if + * the input snapshot is not newer (higher in sequence number) than the current snapshot or if the snapshot cannot + * be found. + * + * @param snapshotId the id of the snapshot to load + */ + protected void updateSnapshot(long snapshotId) { + final List snapshots = tableAdapter.listSnapshots(); + + final Snapshot snapshot = snapshots.stream() + .filter(s -> s.snapshotId() == snapshotId).findFirst() + .orElse(null); + + if (snapshot == null) { + throw new IllegalArgumentException( + "Snapshot " + snapshotId + " was not found in the list of snapshots for table " + tableAdapter + + ". Snapshots: " + snapshots); + } + updateSnapshot(snapshot); + } + + /** + * Update the snapshot to the user specified snapshot. This will fail with an {@link IllegalArgumentException} if + * the input snapshot is not newer (higher in sequence number) than the current snapshot. + * + * @param snapshot the snapshot to load + */ + protected void updateSnapshot(Snapshot snapshot) { + // Validate that we are not trying to update to an older snapshot. + if (snapshot == null) { + throw new IllegalArgumentException("Input snapshot cannot be null"); + } + if (this.snapshot != null && snapshot.sequenceNumber() <= this.snapshot.sequenceNumber()) { + throw new IllegalArgumentException( + "Update snapshot sequence number (" + snapshot.sequenceNumber() + + ") must be higher than the current snapshot sequence number (" + + this.snapshot.sequenceNumber() + ") for table " + tableAdapter); + } + + this.snapshot = snapshot; + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index c05689d38da..8e62f4971b7 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -12,10 +12,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - /** *

    * Manually refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} @@ -28,8 +24,6 @@ public class IcebergManualRefreshTableLocationProvider snapshots = adapter.listSnapshots(); - - final Snapshot snapshot = snapshots.stream() - .filter(s -> s.snapshotId() == snapshotId).findFirst() - .orElse(null); - - if (snapshot == null) { - throw new IllegalArgumentException( - "Snapshot " + snapshotId + " was not found in the list of snapshots for table " + tableIdentifier - + ". Snapshots: " + snapshots); - } - update(snapshot); + // delegate to the locationKeyFinder to update the snapshot + locationKeyFinder.updateSnapshot(snapshotId); + refreshLocations(); } @Override public synchronized void update(final Snapshot snapshot) { - // Verify that the input snapshot is newer (higher in sequence number) than the current snapshot. - if (snapshot.sequenceNumber() <= locationKeyFinder.snapshot.sequenceNumber()) { - throw new IllegalArgumentException( - "Update snapshot sequence number (" + snapshot.sequenceNumber() - + ") must be higher than the current snapshot sequence number (" - + locationKeyFinder.snapshot.sequenceNumber() + ") for table " + tableIdentifier); - } // Update the snapshot. - locationKeyFinder.snapshot = snapshot; - refreshSnapshot(); - } - - /** - * Refresh the table location provider with the latest snapshot from the catalog. This method will identify new - * locations and removed locations. - */ - private void refreshSnapshot() { - beginTransaction(this); - final Set missedKeys = new HashSet<>(); - getTableLocationKeys(ttlk -> missedKeys.add(ttlk.get())); - locationKeyFinder.findKeys(tlk -> { - missedKeys.remove(tlk); - handleTableLocationKeyAdded(tlk, this); - }); - missedKeys.forEach(tlk -> handleTableLocationKeyRemoved(tlk, this)); - endTransaction(this); - setInitialized(); + locationKeyFinder.updateSnapshot(snapshot); + refreshLocations(); } // ------------------------------------------------------------------------------------------------------------------ @@ -122,11 +83,8 @@ private void refreshSnapshot() { @Override protected void activateUnderlyingDataSource() { - if (!initialized) { - refreshSnapshot(); - activationSuccessful(this); - initialized = true; - } + ensureInitialized(); + activationSuccessful(this); } @Override diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index db070a15b1b..f6366635a27 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -57,32 +57,16 @@ public void refresh() { @Override public void update() { - throw new IllegalStateException("A static table location provider cannot be updated"); + throw new UnsupportedOperationException("A static table location provider cannot be updated"); } @Override public void update(long snapshotId) { - throw new IllegalStateException("A static table location provider cannot be updated"); + throw new UnsupportedOperationException("A static table location provider cannot be updated"); } @Override public void update(Snapshot snapshot) { - throw new IllegalStateException("A static table location provider cannot be updated"); - } - - // ------------------------------------------------------------------------------------------------------------------ - // SubscriptionAggregator implementation - // ------------------------------------------------------------------------------------------------------------------ - - @Override - protected void activateUnderlyingDataSource() { - throw new IllegalStateException( - "activateUnderlyingDataSource() called on a static Iceberg table location provider"); - } - - @Override - protected void deactivateUnderlyingDataSource() { - throw new IllegalStateException( - "deactivateUnderlyingDataSource() called on a static Iceberg table location provider"); + throw new UnsupportedOperationException("A static table location provider cannot be updated"); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java index 6b646eec88e..f53b2a843be 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java @@ -13,6 +13,9 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.HashSet; +import java.util.Set; + public abstract class IcebergTableLocationProviderBase extends AbstractTableLocationProvider { @@ -38,23 +41,27 @@ public IcebergTableLocationProviderBase( } /** - * Update the table location provider with the latest snapshot from the catalog. + * Update a manually refreshing table location provider with the latest snapshot from the catalog. This will throw + * an {@link UnsupportedOperationException} if the table is not manually refreshing. */ public abstract void update(); /** - * Update the table location provider with a specific snapshot from the catalog. If the {@code snapshotId} is not - * found in the list of snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot - * must also be newer (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} - * is thrown. + * Update a manually refreshing table location provider with a specific snapshot from the catalog. If the + * {@code snapshotId} is not found in the list of snapshots for the table, an {@link IllegalArgumentException} is + * thrown. The input snapshot must also be newer (higher in sequence number) than the current snapshot or an + * {@link IllegalArgumentException} is thrown. This will throw an {@link UnsupportedOperationException} if the table + * is not manually refreshing. * * @param snapshotId The identifier of the snapshot to use when updating the table. */ public abstract void update(final long snapshotId); /** - * Update the table location provider with a specific snapshot from the catalog. The input snapshot must be newer - * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown. + * Update a manually refreshing table location provider with a specific snapshot from the catalog. The input + * snapshot must be newer (higher in sequence number) than the current snapshot or an + * {@link IllegalArgumentException} is thrown. This will throw an {@link UnsupportedOperationException} if the table + * is not manually refreshing. * * @param snapshot The snapshot to use when updating the table. */ @@ -66,4 +73,22 @@ protected TableLocation makeTableLocation(@NotNull final TableLocationKey locati // noinspection unchecked return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, null); } + + /** + * Refresh the table location provider with the latest snapshot from the catalog. This method will identify new + * locations and removed locations. + */ + protected void refreshLocations() { + final Object token = new Object(); + beginTransaction(token); + final Set missedKeys = new HashSet<>(); + getTableLocationKeys(ttlk -> missedKeys.add(ttlk.get())); + locationKeyFinder.findKeys(tlk -> { + missedKeys.remove(tlk); + handleTableLocationKeyAdded(tlk, token); + }); + missedKeys.forEach(tlk -> handleTableLocationKeyRemoved(tlk, token)); + endTransaction(token); + setInitialized(); + } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java index 134a16a8671..4515c1d5f4f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java @@ -8,13 +8,11 @@ import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; import io.deephaven.util.annotations.VisibleForTesting; -import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; import java.util.*; @@ -214,305 +212,6 @@ public IcebergTableAdapter loadTable(@NotNull final TableIdentifier tableIdentif return new IcebergTableAdapter(tableIdentifier, table, dataInstructionsProvider); } - /** - * List all {@link Snapshot snapshots} of a given Iceberg table. - * - * @param tableIdentifier The identifier of the table from which to gather snapshots. - * @return A list of all snapshots of the given table. - */ - @Deprecated(forRemoval = true) - public List listSnapshots(@NotNull final String tableIdentifier) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.listSnapshots(); - } - - /** - * List all {@link Snapshot snapshots} of a given Iceberg table. - * - * @param tableIdentifier The identifier of the table from which to gather snapshots. - * @return A list of all snapshots of the given table. - */ - @Deprecated(forRemoval = true) - public List listSnapshots(@NotNull final TableIdentifier tableIdentifier) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.listSnapshots(); - } - - /** - * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting - * table will be static and contain the following columns: - *

    - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
    Column NameDescription
    IdThe snapshot identifier (can be used for updating the table or loading a specific snapshot)
    TimestampMsThe timestamp of the snapshot
    OperationThe data operation that created this snapshot
    SummaryAdditional information about the snapshot from the Iceberg metadata
    SnapshotObjectA Java object containing the Iceberg API snapshot
    - * - * @param tableIdentifier The identifier of the table from which to gather snapshots. - * @return A list of all tables in the given namespace. - */ - @Deprecated(forRemoval = true) - public Table snapshots(@NotNull final TableIdentifier tableIdentifier) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.snapshots(); - } - - /** - * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting - * table will be static and contain the same information as {@link #listSnapshots(TableIdentifier)}. - * - * @param tableIdentifier The identifier of the table from which to gather snapshots. - * @return A list of all tables in the given namespace. - */ - @Deprecated(forRemoval = true) - public Table snapshots(@NotNull final String tableIdentifier) { - return snapshots(TableIdentifier.parse(tableIdentifier)); - } - - /** - * Return {@link TableDefinition table definition} for a given Iceberg table, with optional instructions for - * customizations while reading. - * - * @param tableIdentifier The identifier of the table to load - * @param instructions The instructions for customizations while reading - * @return The table definition - */ - @Deprecated(forRemoval = true) - public TableDefinition getTableDefinition( - @NotNull final String tableIdentifier, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.definition(instructions); - } - - /** - * Return {@link TableDefinition table definition} for a given Iceberg table, with optional instructions for - * customizations while reading. - * - * @param tableIdentifier The identifier of the table to load - * @param instructions The instructions for customizations while reading - * @return The table definition - */ - @Deprecated(forRemoval = true) - public TableDefinition getTableDefinition( - @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.definition(instructions); - } - - /** - * Return {@link TableDefinition table definition} for a given Iceberg table and snapshot id, with optional - * instructions for customizations while reading. - * - * @param tableIdentifier The identifier of the table to load - * @param snapshotId The identifier of the snapshot to load - * @param instructions The instructions for customizations while reading - * @return The table definition - */ - @Deprecated(forRemoval = true) - public TableDefinition getTableDefinition( - @NotNull final String tableIdentifier, - final long snapshotId, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.definition(snapshotId, instructions); - } - - /** - * Return {@link TableDefinition table definition} for a given Iceberg table and snapshot id, with optional - * instructions for customizations while reading. - * - * @param tableIdentifier The identifier of the table to load - * @param tableSnapshot The snapshot to load - * @param instructions The instructions for customizations while reading - * @return The table definition - */ - @Deprecated(forRemoval = true) - public TableDefinition getTableDefinition( - @NotNull final TableIdentifier tableIdentifier, - @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.definition(tableSnapshot, instructions); - } - - /** - * Return {@link Table table} containing the {@link TableDefinition definition} of a given Iceberg table, with - * optional instructions for customizations while reading. - * - * @param tableIdentifier The identifier of the table to load - * @param instructions The instructions for customizations while reading - * @return The table definition as a Deephaven table - */ - @Deprecated(forRemoval = true) - public Table getTableDefinitionTable( - @NotNull final String tableIdentifier, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.definitionTable(instructions); - } - - /** - * Return {@link Table table} containing the {@link TableDefinition definition} of a given Iceberg table, with - * optional instructions for customizations while reading. - * - * @param tableIdentifier The identifier of the table to load - * @param instructions The instructions for customizations while reading - * @return The table definition as a Deephaven table - */ - @Deprecated(forRemoval = true) - public Table getTableDefinitionTable( - @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.definitionTable(instructions); - } - - /** - * Return {@link Table table} containing the {@link TableDefinition definition} of a given Iceberg table and - * snapshot id, with optional instructions for customizations while reading. - * - * @param tableIdentifier The identifier of the table to load - * @param snapshotId The identifier of the snapshot to load - * @param instructions The instructions for customizations while reading - * @return The table definition as a Deephaven table - */ - @Deprecated(forRemoval = true) - public Table getTableDefinitionTable( - @NotNull final String tableIdentifier, - final long snapshotId, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.definitionTable(snapshotId, instructions); - } - - /** - * Return {@link Table table} containing the {@link TableDefinition definition} of a given Iceberg table and - * snapshot id, with optional instructions for customizations while reading. - * - * @param tableIdentifier The identifier of the table to load - * @param tableSnapshot The snapshot to load - * @param instructions The instructions for customizations while reading - * @return The table definition as a Deephaven table - */ - @Deprecated(forRemoval = true) - public Table getTableDefinitionTable( - @NotNull final TableIdentifier tableIdentifier, - @Nullable final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.definitionTable(tableSnapshot, instructions); - } - - /** - * Read the latest static snapshot of an Iceberg table from the Iceberg catalog. - * - * @param tableIdentifier The table identifier to load - * @param instructions The instructions for customizations while reading - * @return The loaded table - */ - @SuppressWarnings("unused") - @Deprecated(forRemoval = true) - public IcebergTable readTable( - @NotNull final TableIdentifier tableIdentifier, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.table(instructions); - } - - /** - * Read the latest static snapshot of an Iceberg table from the Iceberg catalog. - * - * @param tableIdentifier The table identifier to load - * @param instructions The instructions for customizations while reading - * @return The loaded table - */ - @SuppressWarnings("unused") - @Deprecated(forRemoval = true) - public IcebergTable readTable( - @NotNull final String tableIdentifier, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.table(instructions); - } - - /** - * Read a static snapshot of an Iceberg table from the Iceberg catalog. - * - * @param tableIdentifier The table identifier to load - * @param tableSnapshotId The snapshot id to load - * @param instructions The instructions for customizations while reading - * @return The loaded table - */ - @SuppressWarnings("unused") - @Deprecated(forRemoval = true) - public IcebergTable readTable( - @NotNull final TableIdentifier tableIdentifier, - final long tableSnapshotId, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.table(tableSnapshotId, instructions); - } - - /** - * Read a static snapshot of an Iceberg table from the Iceberg catalog. - * - * @param tableIdentifier The table identifier to load - * @param tableSnapshotId The snapshot id to load - * @param instructions The instructions for customizations while reading - * @return The loaded table - */ - @SuppressWarnings("unused") - @Deprecated(forRemoval = true) - public IcebergTable readTable( - @NotNull final String tableIdentifier, - final long tableSnapshotId, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.table(tableSnapshotId, instructions); - } - - /** - * Read a static snapshot of an Iceberg table from the Iceberg catalog. - * - * @param tableIdentifier The table identifier to load - * @param tableSnapshot The {@link Snapshot snapshot} to load - * @param instructions The instructions for customizations while reading - * @return The loaded table - */ - @SuppressWarnings("unused") - @Deprecated(forRemoval = true) - public IcebergTable readTable( - @NotNull final TableIdentifier tableIdentifier, - @NotNull final Snapshot tableSnapshot, - @Nullable final IcebergInstructions instructions) { - final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); - return tableAdapter.table(tableSnapshot, instructions); - } - /** * Returns the underlying Iceberg {@link Catalog catalog} used by this adapter. */ diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 9bc776c43b1..169cc1183ed 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -3,7 +3,6 @@ // package io.deephaven.iceberg.util; -import io.deephaven.annotations.BuildableStyle; import io.deephaven.engine.table.TableDefinition; import org.immutables.value.Value; import org.immutables.value.Value.Immutable; @@ -16,7 +15,11 @@ * this class may change in the future. As such, callers may wish to explicitly set the values. */ @Immutable -@BuildableStyle +@Value.Style(visibility = Value.Style.ImplementationVisibility.PACKAGE, + strictBuilder = true, + weakInterning = true, + jdkOnly = true, + includeHashCode = "getClass().hashCode()") public abstract class IcebergInstructions { /** * The default {@link IcebergInstructions} to use when reading Iceberg data files. Providing this will use system @@ -46,6 +49,11 @@ public static Builder builder() { */ public abstract Map columnRenames(); + /** + * Return a copy of this instructions object with the column renames updated to the provided entries. + */ + public abstract IcebergInstructions withColumnRenames(Map entries); + /** * The {@link IcebergUpdateMode} mode to use when reading the Iceberg data files. Default is * {@link IcebergUpdateMode#staticMode()}. diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java index c9d4b031354..89ef0994011 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java @@ -9,22 +9,34 @@ public interface IcebergTable extends Table { /** - * Update the table with the latest snapshot from the catalog. + * When the {@link IcebergInstructions#updateMode() update mode} for this table is + * {@link IcebergUpdateMode#manualRefreshingMode()}, this call will update the table with the latest snapshot from + * the catalog. + *

    + * If any other update mode is specified, this call will throw an {@link UnsupportedOperationException}. */ void update(); /** - * Update the table with a specific snapshot from the catalog. If the {@code snapshotId} is not found in the list of - * snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot must also be newer - * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown. + * When the {@link IcebergInstructions#updateMode() update mode} for this table is + * {@link IcebergUpdateMode#manualRefreshingMode()}, this call will update the table with a specific snapshot from + * the catalog. If the {@code snapshotId} is not found in the list of snapshots for the table, an + * {@link IllegalArgumentException} is thrown. The input snapshot must also be newer (higher in sequence number) + * than the current snapshot or an {@link IllegalArgumentException} is thrown. + *

    + * If any other update mode is specified, this call will throw an {@link UnsupportedOperationException}. * * @param snapshotId The identifier of the snapshot to use when updating the table. */ void update(final long snapshotId); /** - * Update the table with a specific snapshot from the catalog. The input snapshot must be newer (higher in sequence - * number) than the current snapshot or an {@link IllegalArgumentException} is thrown. + * When the {@link IcebergInstructions#updateMode() update mode} for this table is + * {@link IcebergUpdateMode#manualRefreshingMode()}, this call will update the table with a specific snapshot from + * the catalog. The input snapshot must be newer (higher in sequence number) than the current snapshot or an + * {@link IllegalArgumentException} is thrown. + *

    + * If any other update mode is specified, this call will throw an {@link UnsupportedOperationException}. * * @param snapshot The snapshot to use when updating the table. */ diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java index 819309f64d7..69024b6c97f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -66,14 +66,17 @@ public IcebergTableAdapter( } /** - * Get the current {@link Snapshot snapshot} of a given Iceberg table. + * Get the current {@link Snapshot snapshot} of a given Iceberg table or {@code null} if there are no snapshots. * - * @return The current snapshot of the table. + * @return The current snapshot of the table or {@code null} if there are no snapshots. */ public Snapshot currentSnapshot() { + // Refresh the table to update the snapshot list from the catalog. + table.refresh(); + final List snapshots = listSnapshots(); if (snapshots.isEmpty()) { - throw new IllegalStateException("No snapshots found for table " + tableIdentifier); + return null; } return snapshots.get(snapshots.size() - 1); } @@ -84,6 +87,9 @@ public Snapshot currentSnapshot() { * @return A list of all snapshots of the given table. */ public List listSnapshots() { + // Refresh the table to update the snapshot list from the catalog. + table.refresh(); + final List snapshots = new ArrayList<>(); table.snapshots().forEach(snapshots::add); return snapshots; @@ -102,7 +108,7 @@ public List listSnapshots() { * The snapshot identifier (can be used for updating the table or loading a specific snapshot) * * - * TimestampMs + * Timestamp * The timestamp of the snapshot * * @@ -185,7 +191,7 @@ private Optional snapshot(final long snapshotId) { */ public TableDefinition definition() { // Load the table from the catalog. - return definition(null, null); + return definition(null); } /** @@ -210,7 +216,6 @@ public TableDefinition definition(@Nullable final IcebergInstructions instructio public TableDefinition definition( final long snapshotId, @Nullable final IcebergInstructions instructions) { - // Find the snapshot with the given snapshot id final Snapshot tableSnapshot = snapshot(snapshotId).orElseThrow(() -> new IllegalArgumentException( @@ -231,7 +236,18 @@ public TableDefinition definition( public TableDefinition definition( @Nullable final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { - final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); + // Refresh the table to update the snapshot list from the catalog. + table.refresh(); + + final Snapshot snapshot; + if (tableSnapshot == null) { + // Refresh the table to update the snapshot list from the catalog. + table.refresh(); + snapshot = table.currentSnapshot(); + } else { + snapshot = tableSnapshot; + } + final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema(); final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; @@ -248,7 +264,7 @@ public TableDefinition definition( * @return The table definition as a Deephaven table */ public Table definitionTable() { - return definitionTable(null, null); + return TableTools.metaTable(definition()); } /** @@ -259,7 +275,7 @@ public Table definitionTable() { * @return The table definition as a Deephaven table */ public Table definitionTable(@Nullable final IcebergInstructions instructions) { - return definitionTable(null, instructions); + return TableTools.metaTable(definition(null, instructions)); } /** @@ -273,13 +289,7 @@ public Table definitionTable(@Nullable final IcebergInstructions instructions) { public Table definitionTable( final long snapshotId, @Nullable final IcebergInstructions instructions) { - - // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = - snapshot(snapshotId).orElseThrow(() -> new IllegalArgumentException( - "Snapshot with id " + snapshotId + " not found for table " + tableIdentifier)); - - return definitionTable(tableSnapshot, instructions); + return TableTools.metaTable(definition(snapshotId, instructions)); } /** @@ -293,8 +303,7 @@ public Table definitionTable( public Table definitionTable( @Nullable final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { - final TableDefinition definition = definition(tableSnapshot, instructions); - return TableTools.metaTable(definition); + return TableTools.metaTable(definition(tableSnapshot, instructions)); } /** @@ -303,7 +312,7 @@ public Table definitionTable( * @return The loaded table */ public IcebergTable table() { - return table(null, null); + return table(null); } /** @@ -323,12 +332,7 @@ public IcebergTable table(@Nullable final IcebergInstructions instructions) { * @return The loaded table */ public IcebergTable table(final long tableSnapshotId) { - // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = - snapshot(tableSnapshotId).orElseThrow(() -> new IllegalArgumentException( - "Snapshot with id " + tableSnapshotId + " not found for table " + tableIdentifier)); - - return table(tableSnapshot, null); + return table(tableSnapshotId, null); } /** @@ -350,7 +354,7 @@ public IcebergTable table(final long tableSnapshotId, @Nullable final IcebergIns /** * Read a snapshot of an Iceberg table from the Iceberg catalog. * - * @param tableSnapshot The snapshot id to load + * @param tableSnapshot The snapshot to load * @param instructions The instructions for customizations while reading * @return The loaded table */ @@ -359,7 +363,15 @@ public IcebergTable table( @Nullable final IcebergInstructions instructions) { // Do we want the latest or a specific snapshot? - final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot(); + final Snapshot snapshot; + if (tableSnapshot == null) { + // Refresh the table to update the snapshot list from the catalog. + table.refresh(); + snapshot = table.currentSnapshot(); + } else { + snapshot = tableSnapshot; + } + final Schema schema = snapshot == null ? table.schema() : table.schemas().get(snapshot.schemaId()); // Load the partitioning schema. @@ -378,15 +390,18 @@ public IcebergTable table( // applying column renames). final TableDefinition tableDef = fromSchema(schema, partitionSpec, userTableDef, legalizedColumnRenames); - final IcebergBaseLayout keyFinder; + // Create the final instructions with the legalized column renames. + final IcebergInstructions finalInstructions = userInstructions.withColumnRenames(legalizedColumnRenames); + final IcebergBaseLayout keyFinder; if (partitionSpec.isUnpartitioned()) { // Create the flat layout location key finder - keyFinder = new IcebergFlatLayout(this, snapshot, userInstructions, dataInstructionsProviderLoader); + keyFinder = new IcebergFlatLayout(this, snapshot, finalInstructions, + dataInstructionsProviderLoader); } else { // Create the partitioning column location key finder - keyFinder = new IcebergKeyValuePartitionedLayout(this, snapshot, partitionSpec, userInstructions, - dataInstructionsProviderLoader); + keyFinder = new IcebergKeyValuePartitionedLayout(this, snapshot, partitionSpec, + finalInstructions, dataInstructionsProviderLoader); } if (instructions == null diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java index a73a55d40be..a801cf5bebe 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java @@ -29,8 +29,8 @@ public class IcebergTableImpl extends PartitionAwareSourceTable implements Icebe * @param tableDefinition The {@link TableDefinition} describing the table schema * @param description A human-readable description for this table * @param componentFactory A component factory for creating column source managers - * @param locationProvider A {@link io.deephaven.engine.table.impl.locations.TableLocationProvider}, for use in - * discovering the locations that compose this table + * @param locationProvider A {@link IcebergTableLocationProviderBase}, for use in discovering the locations that + * compose this table * @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live */ IcebergTableImpl( diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java index 43da976fce9..1e27c54e259 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java @@ -13,7 +13,7 @@ public abstract class IcebergUpdateMode { private static final IcebergUpdateMode STATIC = builder().updateType(IcebergUpdateType.STATIC).build(); private static final IcebergUpdateMode AUTO_REFRESHING = - builder().updateType(IcebergUpdateType.AUTO_REFRESHING).build(); + builder().updateType(IcebergUpdateType.AUTO_REFRESHING).autoRefreshMs(REFRESH_INTERVAL_MS).build(); private static final IcebergUpdateMode MANUAL_REFRESHING = builder().updateType(IcebergUpdateType.MANUAL_REFRESHING).build(); @@ -21,22 +21,32 @@ public enum IcebergUpdateType { STATIC, AUTO_REFRESHING, MANUAL_REFRESHING } - public static Builder builder() { - return ImmutableIcebergUpdateMode.builder(); - } - + /** + * Set a static update mode for this table. + */ public static IcebergUpdateMode staticMode() { return STATIC; } + /** + * Set a manually-refreshing update mode for this table. + */ public static IcebergUpdateMode manualRefreshingMode() { return MANUAL_REFRESHING; } + /** + * Set a automatically-refreshing update mode for this table using the default refresh interval of 60 seconds. + */ public static IcebergUpdateMode autoRefreshingMode() { return AUTO_REFRESHING; } + /** + * Set a automatically-refreshing update mode for this table using the provided refresh interval + * + * @param refreshMs the refresh interval in milliseconds + */ public static IcebergUpdateMode autoRefreshingMode(final long refreshMs) { return ImmutableIcebergUpdateMode.builder() .updateType(IcebergUpdateType.AUTO_REFRESHING) @@ -44,17 +54,29 @@ public static IcebergUpdateMode autoRefreshingMode(final long refreshMs) { .build(); } + /** + * Set the update mode for the result table. The default is {@link IcebergUpdateType#STATIC}. + */ @Value.Default public IcebergUpdateType updateType() { return IcebergUpdateType.STATIC; } + /** + * When the update type is {@link IcebergUpdateType#AUTO_REFRESHING}, this value specifies the minimum interval in + * milliseconds between automatic refreshes of the table. + */ @Value.Default public long autoRefreshMs() { - return REFRESH_INTERVAL_MS; + return 0; } - public interface Builder { + + private static Builder builder() { + return ImmutableIcebergUpdateMode.builder(); + } + + protected interface Builder { @SuppressWarnings("unused") Builder updateType(IcebergUpdateType updateType); @@ -63,4 +85,15 @@ public interface Builder { IcebergUpdateMode build(); } + + @Value.Check + protected void checkAutoRefreshingInterval() { + if (updateType() != IcebergUpdateType.AUTO_REFRESHING && autoRefreshMs() != 0) { + throw new IllegalArgumentException( + "Auto-refresh interval must not be set when update type is not auto-refreshing"); + } + if (updateType() == IcebergUpdateType.AUTO_REFRESHING && autoRefreshMs() <= 0) { + throw new IllegalArgumentException("Auto-refresh interval must be positive"); + } + } } diff --git a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java index 3a9c88d099d..bfba6f34f01 100644 --- a/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java +++ b/extensions/iceberg/src/test/java/io/deephaven/iceberg/CatalogAdapterTest.java @@ -7,6 +7,7 @@ import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.iceberg.junit5.CatalogAdapterBase; +import io.deephaven.iceberg.util.IcebergTableAdapter; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -31,6 +32,8 @@ void createEmptyTable() { final TableIdentifier myTableId = TableIdentifier.of(myNamespace, "MyTable"); catalogAdapter.catalog().createTable(myTableId, schema); + final IcebergTableAdapter tableAdapter = catalogAdapter.loadTable(myTableId); + assertThat(catalogAdapter.listNamespaces()).containsExactly(myNamespace); assertThat(catalogAdapter.listTables(myNamespace)).containsExactly(myTableId); final Table table; @@ -39,8 +42,9 @@ void createEmptyTable() { ColumnDefinition.ofString("Foo"), ColumnDefinition.ofInt("Bar"), ColumnDefinition.ofDouble("Baz")); - assertThat(catalogAdapter.getTableDefinition(myTableId, null)).isEqualTo(expectedDefinition); - table = catalogAdapter.readTable(myTableId, null); + + assertThat(tableAdapter.definition()).isEqualTo(expectedDefinition); + table = tableAdapter.table(); assertThat(table.getDefinition()).isEqualTo(expectedDefinition); } assertThat(table.isEmpty()).isTrue(); diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index d1f33ac13c1..15d3c0c9356 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -501,7 +501,7 @@ def adapter( _JIcebergTools.createAdapter( name, j_hashmap(properties if properties is not None else {}), - j_hashmap(hadoop_config if hadoopConfig is not None else {}))) + j_hashmap(hadoop_config if hadoop_config is not None else {}))) except Exception as e: raise DHError(e, "Failed to build Iceberg Catalog Adapter") from e From b15c78aba0a3f46f53b841a37841fdf28a72251d Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 15 Oct 2024 17:20:33 -0700 Subject: [PATCH 48/56] IcebergTableAdapter synchronization changes and cleanup. --- .../iceberg/layout/IcebergBaseLayout.java | 5 +- .../iceberg/layout/IcebergFlatLayout.java | 3 +- .../IcebergKeyValuePartitionedLayout.java | 3 +- .../iceberg/util/IcebergTableAdapter.java | 93 ++++++++++++------- 4 files changed, 64 insertions(+), 40 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index 1408c07d70e..cf63682206c 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -109,7 +109,7 @@ protected IcebergTableLocationKey locationKey( */ public IcebergBaseLayout( @NotNull final IcebergTableAdapter tableAdapter, - @NotNull final Snapshot tableSnapshot, + @Nullable final Snapshot tableSnapshot, @NotNull final IcebergInstructions instructions, @NotNull final DataInstructionsProviderLoader dataInstructionsProvider) { this.tableAdapter = tableAdapter; @@ -171,8 +171,7 @@ public synchronized void findKeys(@NotNull final Consumer type, int index) { */ public IcebergKeyValuePartitionedLayout( @NotNull final IcebergTableAdapter tableAdapter, - @NotNull final org.apache.iceberg.Snapshot tableSnapshot, + @Nullable final Snapshot tableSnapshot, @NotNull final PartitionSpec partitionSpec, @NotNull final IcebergInstructions instructions, @NotNull final DataInstructionsProviderLoader dataInstructionsProvider) { diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java index 69024b6c97f..bbca836299b 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -70,15 +70,10 @@ public IcebergTableAdapter( * * @return The current snapshot of the table or {@code null} if there are no snapshots. */ - public Snapshot currentSnapshot() { - // Refresh the table to update the snapshot list from the catalog. - table.refresh(); - - final List snapshots = listSnapshots(); - if (snapshots.isEmpty()) { - return null; - } - return snapshots.get(snapshots.size() - 1); + public synchronized Snapshot currentSnapshot() { + // Refresh the table to update the current snapshot. + refresh(); + return table.currentSnapshot(); } /** @@ -86,10 +81,9 @@ public Snapshot currentSnapshot() { * * @return A list of all snapshots of the given table. */ - public List listSnapshots() { - // Refresh the table to update the snapshot list from the catalog. - table.refresh(); - + public synchronized List listSnapshots() { + // Refresh the table to update the snapshot list. + refresh(); final List snapshots = new ArrayList<>(); table.snapshots().forEach(snapshots::add); return snapshots; @@ -184,6 +178,27 @@ private Optional snapshot(final long snapshotId) { .findFirst(); } + /** + * Retrieve the current {@link Schema schema} of an Iceberg table. + */ + public synchronized Schema currentSchema() { + refresh(); + return table.schema(); + } + + /** + * Retrieve a specific {@link Schema schema} of an Iceberg table. + * + * @param schemaId The identifier of the schema to load. + */ + public synchronized Schema schema(final int schemaId) { + // TODO: discuss refresh() strategy for this and other functions: + // 1) ALWAYS refresh() before searching for a match (safe, might be slow) + // 2) NEVER refresh() before searching (user should call refresh() manually) + // 3) HYBRID, refresh() if search fails, then re-search + return table.schemas().get(schemaId); + } + /** * Return {@link TableDefinition table definition}. * @@ -236,24 +251,29 @@ public TableDefinition definition( public TableDefinition definition( @Nullable final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { - // Refresh the table to update the snapshot list from the catalog. - table.refresh(); final Snapshot snapshot; + final Schema schema; + final org.apache.iceberg.PartitionSpec partitionSpec; + if (tableSnapshot == null) { - // Refresh the table to update the snapshot list from the catalog. - table.refresh(); - snapshot = table.currentSnapshot(); + synchronized (this) { + // Refresh only once and record the current snapshot, using its schema and spec. + refresh(); + snapshot = table.currentSnapshot(); + schema = snapshot != null ? schema(snapshot.schemaId()) : table.schema(); + partitionSpec = table.spec(); + } } else { snapshot = tableSnapshot; + schema = schema(tableSnapshot.schemaId()); + partitionSpec = table.spec(); } - final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema(); - final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; return fromSchema(schema, - table.spec(), + partitionSpec, userInstructions.tableDefinition().orElse(null), getRenameColumnMap(table, schema, userInstructions)); } @@ -362,21 +382,24 @@ public IcebergTable table( @Nullable final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { - // Do we want the latest or a specific snapshot? final Snapshot snapshot; + final Schema schema; + final org.apache.iceberg.PartitionSpec partitionSpec; + if (tableSnapshot == null) { - // Refresh the table to update the snapshot list from the catalog. - table.refresh(); - snapshot = table.currentSnapshot(); + synchronized (this) { + // Refresh only once and record the current snapshot, using its schema and spec. + refresh(); + snapshot = table.currentSnapshot(); + schema = snapshot != null ? schema(snapshot.schemaId()) : table.schema(); + partitionSpec = table.spec(); + } } else { snapshot = tableSnapshot; + schema = schema(tableSnapshot.schemaId()); + partitionSpec = table.spec(); } - final Schema schema = snapshot == null ? table.schema() : table.schemas().get(snapshot.schemaId()); - - // Load the partitioning schema. - final org.apache.iceberg.PartitionSpec partitionSpec = table.spec(); - // Get default instructions if none are provided final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions; @@ -453,7 +476,7 @@ public IcebergTable table( /** * Refresh the table with the latest information from the Iceberg catalog, including new snapshots and schema. */ - public void refresh() { + public synchronized void refresh() { table.refresh(); } @@ -517,14 +540,14 @@ private Map getRenameColumnMap( * @param schema The schema of the table. * @param partitionSpec The partition specification of the table. * @param userTableDef The table definition. - * @param columnRename The map for renaming columns. + * @param columnRenameMap The map for renaming columns. * @return The generated TableDefinition. */ private static TableDefinition fromSchema( @NotNull final Schema schema, @NotNull final PartitionSpec partitionSpec, @Nullable final TableDefinition userTableDef, - @NotNull final Map columnRename) { + @NotNull final Map columnRenameMap) { final Set columnNames = userTableDef != null ? userTableDef.getColumnNameSet() @@ -533,13 +556,13 @@ private static TableDefinition fromSchema( final Set partitionNames = partitionSpec.fields().stream() .map(PartitionField::name) - .map(colName -> columnRename.getOrDefault(colName, colName)) + .map(colName -> columnRenameMap.getOrDefault(colName, colName)) .collect(Collectors.toSet()); final List> columns = new ArrayList<>(); for (final Types.NestedField field : schema.columns()) { - final String name = columnRename.getOrDefault(field.name(), field.name()); + final String name = columnRenameMap.getOrDefault(field.name(), field.name()); // Skip columns that are not in the provided table definition. if (columnNames != null && !columnNames.contains(name)) { continue; From de9f6ae5102870a3e720ec6cbbf242e4d6220cb9 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 18 Oct 2024 11:02:34 -0700 Subject: [PATCH 49/56] PR comments addressed. --- .../iceberg/layout/IcebergBaseLayout.java | 28 +++----- ...ergManualRefreshTableLocationProvider.java | 13 +++- .../iceberg/util/IcebergInstructions.java | 9 +-- .../iceberg/util/IcebergTableAdapter.java | 69 +++++++++++++------ 4 files changed, 71 insertions(+), 48 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index cf63682206c..d49b394438b 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java @@ -170,7 +170,7 @@ public synchronized void findKeys(@NotNull final Consumer snapshots = tableAdapter.listSnapshots(); @@ -205,23 +202,18 @@ protected void updateSnapshot(long snapshotId) { } /** - * Update the snapshot to the user specified snapshot. This will fail with an {@link IllegalArgumentException} if - * the input snapshot is not newer (higher in sequence number) than the current snapshot. - * - * @param snapshot the snapshot to load + * Update the snapshot to the user specified snapshot. See + * {@link io.deephaven.iceberg.util.IcebergTable#update(Snapshot)} for more details. */ - protected void updateSnapshot(Snapshot snapshot) { + protected void updateSnapshot(@NotNull final Snapshot updateSnapshot) { // Validate that we are not trying to update to an older snapshot. - if (snapshot == null) { - throw new IllegalArgumentException("Input snapshot cannot be null"); - } - if (this.snapshot != null && snapshot.sequenceNumber() <= this.snapshot.sequenceNumber()) { + if (snapshot != null && updateSnapshot.sequenceNumber() <= snapshot.sequenceNumber()) { throw new IllegalArgumentException( - "Update snapshot sequence number (" + snapshot.sequenceNumber() + "Update snapshot sequence number (" + updateSnapshot.sequenceNumber() + ") must be higher than the current snapshot sequence number (" - + this.snapshot.sequenceNumber() + ") for table " + tableAdapter); + + snapshot.sequenceNumber() + ") for table " + tableAdapter); } - this.snapshot = snapshot; + snapshot = updateSnapshot; } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index 8e62f4971b7..1a249d4e15f 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -51,9 +51,8 @@ public String getImplementationName() { } @Override - public synchronized void refresh() { - // There is no refresh service for this provider, but this is called as part of the initialization process. - refreshLocations(); + public void refresh() { + ensureInitialized(); } @Override @@ -72,11 +71,19 @@ public synchronized void update(final long snapshotId) { @Override public synchronized void update(final Snapshot snapshot) { + if (snapshot == null) { + throw new IllegalArgumentException("Input snapshot cannot be null"); + } // Update the snapshot. locationKeyFinder.updateSnapshot(snapshot); refreshLocations(); } + @Override + protected synchronized void doInitialization() { + refreshLocations(); + } + // ------------------------------------------------------------------------------------------------------------------ // SubscriptionAggregator implementation // ------------------------------------------------------------------------------------------------------------------ diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java index 169cc1183ed..a77a02b341e 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java @@ -3,6 +3,7 @@ // package io.deephaven.iceberg.util; +import io.deephaven.annotations.CopyableStyle; import io.deephaven.engine.table.TableDefinition; import org.immutables.value.Value; import org.immutables.value.Value.Immutable; @@ -15,11 +16,7 @@ * this class may change in the future. As such, callers may wish to explicitly set the values. */ @Immutable -@Value.Style(visibility = Value.Style.ImplementationVisibility.PACKAGE, - strictBuilder = true, - weakInterning = true, - jdkOnly = true, - includeHashCode = "getClass().hashCode()") +@CopyableStyle public abstract class IcebergInstructions { /** * The default {@link IcebergInstructions} to use when reading Iceberg data files. Providing this will use system @@ -50,7 +47,7 @@ public static Builder builder() { public abstract Map columnRenames(); /** - * Return a copy of this instructions object with the column renames updated to the provided entries. + * Return a copy of this instructions object with the column renames replaced by {@code entries}. */ public abstract IcebergInstructions withColumnRenames(Map entries); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java index bbca836299b..fa78259a9e5 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -77,13 +77,22 @@ public synchronized Snapshot currentSnapshot() { } /** - * List all {@link Snapshot snapshots} of the Iceberg table. + * Get the current list of all {@link Snapshot snapshots} of the Iceberg table. * * @return A list of all snapshots of the given table. */ public synchronized List listSnapshots() { // Refresh the table to update the snapshot list. refresh(); + return getSnapshots(); + } + + /** + * Get a list of all {@link Snapshot snapshots} of the Iceberg table (without refreshing). + * + * @return A list of all snapshots of the given table. + */ + private List getSnapshots() { final List snapshots = new ArrayList<>(); table.snapshots().forEach(snapshots::add); return snapshots; @@ -124,6 +133,7 @@ public synchronized List listSnapshots() { * @return A Table containing a list of all tables in the given namespace. */ public Table snapshots() { + // Retrieve the current list of snapshots final List snapshots = listSnapshots(); final long size = snapshots.size(); @@ -173,9 +183,17 @@ public Table snapshots() { * @return An Optional containing the requested snapshot if it exists. */ private Optional snapshot(final long snapshotId) { - return listSnapshots().stream() + Optional found = getSnapshots().stream() .filter(snapshot -> snapshot.snapshotId() == snapshotId) .findFirst(); + if (found.isEmpty()) { + // Refresh the table to update the snapshot list, then try again. + refresh(); + found = getSnapshots().stream() + .filter(snapshot -> snapshot.snapshotId() == snapshotId) + .findFirst(); + } + return found; } /** @@ -186,17 +204,27 @@ public synchronized Schema currentSchema() { return table.schema(); } + /** + * Retrieve the current {@link Schema schema} of an Iceberg table. + */ + public synchronized Map schemas() { + refresh(); + return table.schemas(); + } + /** * Retrieve a specific {@link Schema schema} of an Iceberg table. * * @param schemaId The identifier of the schema to load. */ - public synchronized Schema schema(final int schemaId) { - // TODO: discuss refresh() strategy for this and other functions: - // 1) ALWAYS refresh() before searching for a match (safe, might be slow) - // 2) NEVER refresh() before searching (user should call refresh() manually) - // 3) HYBRID, refresh() if search fails, then re-search - return table.schemas().get(schemaId); + public synchronized Optional schema(final int schemaId) { + Schema found = table.schemas().get(schemaId); + if (found == null) { + // Refresh the table to update the snapshot list, then try again. + refresh(); + found = table.schemas().get(schemaId); + } + return Optional.ofNullable(found); } /** @@ -252,21 +280,19 @@ public TableDefinition definition( @Nullable final Snapshot tableSnapshot, @Nullable final IcebergInstructions instructions) { - final Snapshot snapshot; final Schema schema; final org.apache.iceberg.PartitionSpec partitionSpec; if (tableSnapshot == null) { synchronized (this) { - // Refresh only once and record the current snapshot, using its schema and spec. + // Refresh only once and record the current schema and partition spec. refresh(); - snapshot = table.currentSnapshot(); - schema = snapshot != null ? schema(snapshot.schemaId()) : table.schema(); + schema = table.schema(); partitionSpec = table.spec(); } } else { - snapshot = tableSnapshot; - schema = schema(tableSnapshot.schemaId()); + // Use the schema from the snapshot + schema = schema(tableSnapshot.schemaId()).get(); partitionSpec = table.spec(); } @@ -388,15 +414,17 @@ public IcebergTable table( if (tableSnapshot == null) { synchronized (this) { - // Refresh only once and record the current snapshot, using its schema and spec. + // Refresh only once and record the current snapshot, schema (which may be newer than the + // snapshot schema), and partition spec. refresh(); snapshot = table.currentSnapshot(); - schema = snapshot != null ? schema(snapshot.schemaId()) : table.schema(); + schema = table.schema(); partitionSpec = table.spec(); } } else { snapshot = tableSnapshot; - schema = schema(tableSnapshot.schemaId()); + // Use the schema from the snapshot + schema = schema(tableSnapshot.schemaId()).get(); partitionSpec = table.spec(); } @@ -427,8 +455,7 @@ public IcebergTable table( finalInstructions, dataInstructionsProviderLoader); } - if (instructions == null - || instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) { + if (finalInstructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) { final IcebergTableLocationProviderBase locationProvider = new IcebergStaticTableLocationProvider<>( StandaloneTableKey.getInstance(), @@ -447,7 +474,7 @@ public IcebergTable table( final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph(); final IcebergTableLocationProviderBase locationProvider; - if (instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) { + if (finalInstructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) { locationProvider = new IcebergManualRefreshTableLocationProvider<>( StandaloneTableKey.getInstance(), keyFinder, @@ -460,7 +487,7 @@ public IcebergTable table( keyFinder, new IcebergTableLocationFactory(), TableDataRefreshService.getSharedRefreshService(), - instructions.updateMode().autoRefreshMs(), + finalInstructions.updateMode().autoRefreshMs(), this, tableIdentifier); } From d5c13e671ffefe488f77c6ad03d78f4e346a9875 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 18 Oct 2024 11:31:28 -0700 Subject: [PATCH 50/56] Fix for iceberg.py to use table adapter --- py/server/deephaven/experimental/iceberg.py | 38 ++------------------- 1 file changed, 3 insertions(+), 35 deletions(-) diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 15d3c0c9356..1b5489e853e 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -229,14 +229,9 @@ def table(self, instructions: Optional[IcebergInstructions] = None, snapshot_id: Table: the table read from the catalog. """ - if instructions is not None: - instructions_object = instructions.j_object - else: - instructions_object = _JIcebergInstructions.DEFAULT - - if snapshot_id is not None: - return IcebergTable(self.j_object.table(snapshot_id, instructions_object)) - return IcebergTable(self.j_object.readTable(instructions_object)) + if snapshot_id: + return IcebergTable(self.j_object.table(snapshot_id, instructions)) + return IcebergTable(self.j_object.table(instructions)) @property def j_object(self) -> jpy.JType: @@ -317,33 +312,6 @@ def snapshots(self, table_identifier: str) -> Table: return self.j_object.snapshots(table_identifier) - def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> IcebergTable: - """ - NOTE: deprecated, use `load_table(table_identifier).table()` instead. - - Reads the table from the catalog using the provided instructions. Optionally, a snapshot id can be provided to - read a specific snapshot of the table. - - Args: - table_identifier (str): the table to read. - instructions (Optional[IcebergInstructions]): the instructions for reading the table. These instructions - can include column renames, table definition, and specific data instructions for reading the data files - from the provider. If omitted, the table will be read with default instructions. - snapshot_id (Optional[int]): the snapshot id to read; if omitted the most recent snapshot will be selected. - - Returns: - Table: the table read from the catalog. - """ - - if instructions is not None: - instructions_object = instructions.j_object - else: - instructions_object = _JIcebergInstructions.DEFAULT - - if snapshot_id is not None: - return IcebergTable(self.j_object.readTable(table_identifier, snapshot_id, instructions_object)) - return IcebergTable(self.j_object.readTable(table_identifier, instructions_object)) - @property def j_object(self) -> jpy.JType: return self.j_catalog_adapter From d1557c548f2296c0e1dbfc9c8c0fba941d031a87 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 18 Oct 2024 15:24:54 -0700 Subject: [PATCH 51/56] Addressed PR comments. --- .../AbstractTableLocationProviderTest.java | 142 +++++++++--------- .../iceberg/util/IcebergTableAdapter.java | 2 +- py/server/deephaven/experimental/iceberg.py | 70 ++++----- 3 files changed, 96 insertions(+), 118 deletions(-) diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java index 36e0fe2353f..23664cb05d7 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/AbstractTableLocationProviderTest.java @@ -24,6 +24,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.stream.IntStream; @Category(OutOfBandTest.class) public class AbstractTableLocationProviderTest extends RefreshingTableTestCase { @@ -160,7 +161,11 @@ public void testTableLocationKeyManagement() { // Get a list of the LiveSupplier<> keys final List> initialTrackedKeys = new ArrayList<>(); - provider.getTableLocationKeys(initialTrackedKeys::add); + provider.getTableLocationKeys(lstlk -> { + // Externally manage all keys (simulate a TLSB) + manager.manage(lstlk); + initialTrackedKeys.add(lstlk); + }); // Resolve and create all the table locations final List tableLocations = new ArrayList<>(); @@ -169,63 +174,55 @@ public void testTableLocationKeyManagement() { tableLocations.add(tl); } - // Externally manage keys 2 & 3 (simulate a TLSB) - manager.manage(initialTrackedKeys.get(2)); - manager.manage(initialTrackedKeys.get(3)); - - // Also manage the table locations for key 3 (simulate a filtered RCSM) - manager.manage(tableLocations.get(3)); - // Drop the first 4 keys from the provider for (int i = 0; i < 4; i++) { final TableLocationKey removedKey = initialKeys.get(i); provider.removeKey(removedKey); } + // Simulate delivering the initial keys to the RCSM + final List includedLocations = new ArrayList<>(); + includedLocations.add(provider.getTableLocation(initialKeys.get(2))); + includedLocations.forEach(manager::manage); + initialTrackedKeys.forEach(manager::unmanage); + // Verify only the last key is present for new listeners keys = new HashSet<>(provider.getTableLocationKeys()); Assert.eqTrue(keys.contains(initialKeys.get(4)), "keys.contains(initialKeys.get(4))"); Assert.eq(keys.size(), "keys.size()", 1); // Verify that we CAN'T retrieve the unmanaged locations from the provider (they were dropped) - Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(0)), - "provider.hasTableLocationKey(initialKeys.get(0))"); - Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(1)), - "provider.hasTableLocationKey(initialKeys.get(1))"); + IntStream.range(0, 4).forEach( + i -> { + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(i)), + "provider.hasTableLocationKey(initialKeys.get(" + i + "))"); + }); - // Verify the tableLocations for the unmanaged keys are destroyed + // Verify the tableLocations for only the unmanaged keys are destroyed Assert.eqTrue(tableLocations.get(0).isDestroyed(), "tableLocations.get(0).isDestroyed()"); Assert.eqTrue(tableLocations.get(1).isDestroyed(), "tableLocations.get(1).isDestroyed()"); - - // Verify that we CAN retrieve the managed locations from the provider (they are still live) - Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(2)), - "provider.hasTableLocationKey(initialKeys.get(2))"); - Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(3)), - "provider.hasTableLocationKey(initialKeys.get(3))"); - - // Verify the tableLocations for the managed keys are NOT destroyed + Assert.eqTrue(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); Assert.eqFalse(tableLocations.get(2).isDestroyed(), "tableLocations.get(2).isDestroyed()"); - Assert.eqFalse(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); - // Un-manage the two keys - manager.unmanage(initialTrackedKeys.get(2)); - manager.unmanage(initialTrackedKeys.get(3)); - - // location #2 should be destroyed, location #3 should not (because an RCSM is managing it) + // Verify the tableLocations for the previously included keys are destroyed + includedLocations.forEach(manager::unmanage); Assert.eqTrue(tableLocations.get(2).isDestroyed(), "tableLocations.get(2).isDestroyed()"); - Assert.eqFalse(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); - // Verify that we CAN'T retrieve the (now) unmanaged locations from the provider (they were dropped) - Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(2)), - "provider.hasTableLocationKey(initialKeys.get(2))"); - Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(3)), - "provider.hasTableLocationKey(initialKeys.get(3))"); + // Verify that we CAN retrieve the last key from the provider and the location is not destroyed + Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(4)), + "provider.hasTableLocationKey(initialKeys.get(4))"); + Assert.eqFalse(tableLocations.get(4).isDestroyed(), "tableLocations.get(4).isDestroyed()"); - // Release the table location being held by the RCSM - manager.unmanage(tableLocations.get(3)); - Assert.eqTrue(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); + // Drop the final key from the provider + provider.removeKey(initialKeys.get(4)); + + // Verify that we CAN retrieve the last key from the provider and the location is not destroyed + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(4)), + "provider.hasTableLocationKey(initialKeys.get(4))"); + Assert.eqTrue(tableLocations.get(4).isDestroyed(), "tableLocations.get(4).isDestroyed()"); } + /** * Test the management and release of the liveness of table location keys is correct when using transactions. */ @@ -233,12 +230,12 @@ public void testTableLocationKeyManagement() { public void testTableLocationKeyTransactionManagement() { // Create a test table location provider - final TestTableLocationProvider provider = new TestTableLocationProvider(true); + final TestTableLocationProvider provider = new TestTableLocationProvider(false); // Create a set of table location keys final List initialKeys = createKeys(5); - // Add the keys to the table location provider + // Add the keys to the table location provider in a single tranasction provider.beginTransaction(provider); for (final TableLocationKey locationKey : initialKeys) { provider.addKey(locationKey, provider); @@ -256,7 +253,11 @@ public void testTableLocationKeyTransactionManagement() { // Get a list of the LiveSupplier<> keys final List> initialTrackedKeys = new ArrayList<>(); - provider.getTableLocationKeys(initialTrackedKeys::add); + provider.getTableLocationKeys(lstlk -> { + // Externally manage all keys (simulate a TLSB) + manager.manage(lstlk); + initialTrackedKeys.add(lstlk); + }); // Resolve and create all the table locations final List tableLocations = new ArrayList<>(); @@ -265,14 +266,7 @@ public void testTableLocationKeyTransactionManagement() { tableLocations.add(tl); } - // Externally manage keys 2 & 3 (simulate a TLSB) - manager.manage(initialTrackedKeys.get(2)); - manager.manage(initialTrackedKeys.get(3)); - - // Also manage the table locations for key 3 (simulate a filtered RCSM) - manager.manage(tableLocations.get(3)); - - // Drop the first 4 keys from the provider + // Drop the first 4 keys from the provider in a single tranasction provider.beginTransaction(provider); for (int i = 0; i < 4; i++) { final TableLocationKey removedKey = initialKeys.get(i); @@ -280,47 +274,45 @@ public void testTableLocationKeyTransactionManagement() { } provider.endTransaction(provider); + // Simulate delivering the initial keys to the RCSM + final List includedLocations = new ArrayList<>(); + includedLocations.add(provider.getTableLocation(initialKeys.get(2))); + includedLocations.forEach(manager::manage); + initialTrackedKeys.forEach(manager::unmanage); + // Verify only the last key is present for new listeners keys = new HashSet<>(provider.getTableLocationKeys()); Assert.eqTrue(keys.contains(initialKeys.get(4)), "keys.contains(initialKeys.get(4))"); Assert.eq(keys.size(), "keys.size()", 1); // Verify that we CAN'T retrieve the unmanaged locations from the provider (they were dropped) - Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(0)), - "provider.hasTableLocationKey(initialKeys.get(0))"); - Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(1)), - "provider.hasTableLocationKey(initialKeys.get(1))"); + IntStream.range(0, 4).forEach( + i -> { + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(i)), + "provider.hasTableLocationKey(initialKeys.get(" + i + "))"); + }); - // Verify the tableLocations for the unmanaged keys are destroyed + // Verify the tableLocations for only the unmanaged keys are destroyed Assert.eqTrue(tableLocations.get(0).isDestroyed(), "tableLocations.get(0).isDestroyed()"); Assert.eqTrue(tableLocations.get(1).isDestroyed(), "tableLocations.get(1).isDestroyed()"); - - // Verify that we CAN retrieve the managed locations from the provider (they are still live) - Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(2)), - "provider.hasTableLocationKey(initialKeys.get(2))"); - Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(3)), - "provider.hasTableLocationKey(initialKeys.get(3))"); - - // Verify the tableLocations for the managed keys are NOT destroyed + Assert.eqTrue(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); Assert.eqFalse(tableLocations.get(2).isDestroyed(), "tableLocations.get(2).isDestroyed()"); - Assert.eqFalse(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); - - // Un-manage the two keys - manager.unmanage(initialTrackedKeys.get(2)); - manager.unmanage(initialTrackedKeys.get(3)); - // location #2 should be destroyed, location #3 should not (because an RCSM is managing it) + // Verify the tableLocations for the previously included keys are destroyed + includedLocations.forEach(manager::unmanage); Assert.eqTrue(tableLocations.get(2).isDestroyed(), "tableLocations.get(2).isDestroyed()"); - Assert.eqFalse(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); - // Verify that we CAN'T retrieve the (now) unmanaged locations from the provider (they were dropped) - Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(2)), - "provider.hasTableLocationKey(initialKeys.get(2))"); - Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(3)), - "provider.hasTableLocationKey(initialKeys.get(3))"); + // Verify that we CAN retrieve the last key from the provider and the location is not destroyed + Assert.eqTrue(provider.hasTableLocationKey(initialKeys.get(4)), + "provider.hasTableLocationKey(initialKeys.get(4))"); + Assert.eqFalse(tableLocations.get(4).isDestroyed(), "tableLocations.get(4).isDestroyed()"); - // Release the table location being held by the RCSM and verify its destruction. - manager.unmanage(tableLocations.get(3)); - Assert.eqTrue(tableLocations.get(3).isDestroyed(), "tableLocations.get(3).isDestroyed()"); + // Drop the final key from the provider + provider.removeKey(initialKeys.get(4)); + + // Verify that we CAN retrieve the last key from the provider and the location is not destroyed + Assert.eqFalse(provider.hasTableLocationKey(initialKeys.get(4)), + "provider.hasTableLocationKey(initialKeys.get(4))"); + Assert.eqTrue(tableLocations.get(4).isDestroyed(), "tableLocations.get(4).isDestroyed()"); } } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java index fa78259a9e5..8ea62b26027 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -209,7 +209,7 @@ public synchronized Schema currentSchema() { */ public synchronized Map schemas() { refresh(); - return table.schemas(); + return Map.copyOf(table.schemas()); } /** diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 95e554ef94f..1699725b8ae 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -34,13 +34,13 @@ class IcebergUpdateMode(JObjectWrapper): """ - This class specifies the update mode for the Iceberg table to be loaded into Deephaven. The modes are: + This class specifies the update mode for an Iceberg table to be loaded into Deephaven. The modes are: - - `static()`: The table is loaded once and does not change - - `manual_refresh()`: The table can be manually refreshed by the user. - - `auto_refresh()`: The table will be automatically refreshed at a system-defined interval (also can call - `auto_refreshing(auto_refresh_ms: int)` to specify an interval rather than use the system default - of 60 seconds). + - :py:func:`static() `: The table is loaded once and does not change + - :py:func:`manual_refresh() `: The table can be manually refreshed by the user. + - :py:func:`auto_refresh() `: The table will be automatically refreshed at a + system-defined interval (also can call :py:func:`auto_refresh(auto_refresh_ms: int) ` + to specify an interval rather than use the system default of 60 seconds). """ j_object_type = _JIcebergUpdateMode @@ -64,9 +64,11 @@ def manual_refresh(cls) -> IcebergUpdateMode: @classmethod def auto_refresh(cls, auto_refresh_ms:Optional[int] = None) -> IcebergUpdateMode: """ - Creates an IcebergUpdateMode with auto-refreshing mode enabled using the provided refresh interval. + Creates an IcebergUpdateMode with auto-refreshing enabled. - :param auto_refresh_ms (int): the refresh interval in milliseconds. + Args: + auto_refresh_ms (int): the refresh interval in milliseconds; if omitted, the default of 60 seconds + is used. """ if auto_refresh_ms is None: return IcebergUpdateMode(_JIcebergUpdateMode.autoRefreshingMode()) @@ -102,7 +104,7 @@ def __init__(self, column_renames (Optional[Dict[str, str]]): A dictionary of old to new column names that will be renamed in the output table. update_mode (Optional[IcebergUpdateMode]): The update mode for the table. If omitted, the default update - mode of `IcebergUpdateMode.STATIC` is used. + mode of :py:func:`IcebergUpdateMode.static() ` is used. Raises: DHError: If unable to build the instructions object. @@ -145,10 +147,12 @@ def __init__(self, j_table: jpy.JType): def update(self, snapshot_id:Optional[int] = None): """ - Updates the table with a specific snapshot. If no snapshot is provided, the most recent snapshot is used. + Updates the table to match the contents of the specified snapshot. This may result in row removes and additions + that will be propagated asynchronously via this IcebergTable's UpdateGraph. If no snapshot is provided, the + most recent snapshot is used. - NOTE: this method is only valid when the table is in `MANUAL_REFRESHING` mode. `STATIC` and `AUTO_REFRESHING` - Iceberg tables cannot be updated manually and will throw an exception if this method is called. + NOTE: this method is only valid when the table is in `manual_refresh()` mode. Iceberg tables in `static()` or + `auto_refresh()` mode cannot be updated manually and will throw an exception if this method is called. Args: snapshot_id (Optional[int]): the snapshot id to update to; if omitted the most recent snapshot will be used. @@ -172,8 +176,8 @@ def j_object(self) -> jpy.JType: class IcebergTableAdapter(JObjectWrapper): """ - This class provides an interface for interacting with Iceberg tables. It allows listing snapshots, table - definitions, as well as reading Iceberg tables into Deephaven tables. + This class provides an interface for interacting with Iceberg tables. It allows the user to list snapshots, + retrieve table definitions and reading Iceberg tables into Deephaven tables. """ j_object_type = _JIcebergTableAdapter or type(None) @@ -200,13 +204,16 @@ def snapshots(self) -> Table: def definition(self, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> Table: """ - Returns information on the tables in the specified namespace as a Deephaven table. + Returns the Deephaven table definition as a Deephaven table. Args: - namespace (str): the namespace from which to list tables. + instructions (Optional[IcebergInstructions]): the instructions for reading the table. These instructions + can include column renames, table definition, and specific data instructions for reading the data files + from the provider. If omitted, the table will be read with default instructions. + snapshot_id (Optional[int]): the snapshot id to read; if omitted the most recent snapshot will be selected. Returns: - a table containing the tables in the provided namespace. + a table containing the table definition. """ if snapshot_id is not None: @@ -215,14 +222,14 @@ def definition(self, instructions: Optional[IcebergInstructions] = None, snapsho def table(self, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> IcebergTable: """ - Reads the table using the provided instructions. Optionally, a snapshot id can be provided to - read a specific snapshot of the table. + Reads the table using the provided instructions. Optionally, a snapshot id can be provided to read a specific + snapshot of the table. Args: - table_identifier (str): the table to read. instructions (Optional[IcebergInstructions]): the instructions for reading the table. These instructions can include column renames, table definition, and specific data instructions for reading the data files - from the provider. If omitted, the table will be read with default instructions. + from the provider. If omitted, the table will be read in `static()` mode without column renames or data + instructions. snapshot_id (Optional[int]): the snapshot id to read; if omitted the most recent snapshot will be selected. Returns: @@ -291,27 +298,6 @@ def load_table(self, table_identifier: str) -> IcebergTable: return IcebergTableAdapter(self.j_object.loadTable(table_identifier)) - def snapshots(self, table_identifier: str) -> Table: - """ - NOTE: deprecated, use `load_table(table_identifier).snapshots()` instead. - - Returns information on the snapshots of the specified table as a Deephaven table. The table contains the - following columns: - - `Id`: the snapshot identifier (can be used for updating the table or loading a specific snapshot). - - `TimestampMs`: the timestamp of the snapshot. - - `Operation`: the data operation that created this snapshot. - - `Summary`: additional information about this snapshot from the Iceberg metadata. - - `SnapshotObject`: a Java object containing the Iceberg API snapshot. - - Args: - table_identifier (str): the table from which to list snapshots. - - Returns: - a table containing the snapshot information. - """ - - return self.j_object.snapshots(table_identifier) - @property def j_object(self) -> jpy.JType: return self.j_catalog_adapter From 975a01348c30adf54fd5aa1c843cd668b1906836 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 18 Oct 2024 17:05:06 -0700 Subject: [PATCH 52/56] Close to merging. --- .../iceberg/util/IcebergToolsTest.java | 31 +++++++++++++++++++ ...ebergAutoRefreshTableLocationProvider.java | 4 ++- py/server/deephaven/experimental/iceberg.py | 3 ++ 3 files changed, 37 insertions(+), 1 deletion(-) diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java index 8c454000eee..54f9666c342 100644 --- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java +++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java @@ -15,6 +15,7 @@ import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog; import io.deephaven.test.types.OutOfBandTest; +import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -737,6 +738,14 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx io.deephaven.engine.table.Table table5 = tableAdapter.table(snapshots.get(5), instructions); Assert.eq(table5.size(), "table5.size()", 0, "expected rows in the table"); Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); + + try { + io.deephaven.engine.table.Table missing = tableAdapter.table(987654321L, instructions); + Assert.statementNeverExecuted("Expected an exception for invalid snapshot"); + } catch (final Exception e) { + Assert.assertion(e instanceof IllegalArgumentException, "e instanceof IllegalArgumentException"); + Assert.eqTrue(e.getMessage().contains("Snapshot with id 987654321 not found"), "Exception message"); + } } @Test @@ -775,6 +784,28 @@ public void testTableDefinition() { Assert.equals(tableDef, "tableDef", SALES_MULTI_DEFINITION); } + @Test + public void testTableSchema() { + final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); + final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi"); + + // Request a schema that does not exist. + final Optional missingSchema = tableAdapter.schema(1000); + Assert.eqFalse(missingSchema.isPresent(), "schema.isPresent()"); + + // Request a schema that does exist. + final Optional schema0 = tableAdapter.schema(0); + Assert.eqTrue(schema0.isPresent(), "schema.isPresent()"); + + // Request the current schema, assert it matches schema0 + final Schema currentSchema = tableAdapter.currentSchema(); + Assert.eq(currentSchema, "currentSchema", schema0.get(), "schema0.get()"); + + // Request the schema map. + final Map schemaMap = tableAdapter.schemas(); + Assert.eq(schemaMap.size(), "schemaMap.size()", 1, "expected number of schemas"); + } + @Test public void testTableDefinitionTable() { final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog); diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index 8fcc1172c48..18611e58bf3 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -91,8 +91,10 @@ public void update(Snapshot snapshot) { @Override protected final void activateUnderlyingDataSource() { - refresh(); + ensureInitialized(); + refreshLocations(); subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this, refreshIntervalMs); + activationSuccessful(this); } @Override diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 1699725b8ae..8b6a3aab2d8 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -236,6 +236,9 @@ def table(self, instructions: Optional[IcebergInstructions] = None, snapshot_id: Table: the table read from the catalog. """ + if instructions: + instructions = instructions.j_object + if snapshot_id: return IcebergTable(self.j_object.table(snapshot_id, instructions)) return IcebergTable(self.j_object.table(instructions)) From ad5a225ec7cdfea15b37c5cdbde30bf21e02146d Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Mon, 21 Oct 2024 14:43:13 -0700 Subject: [PATCH 53/56] Changes in the refresh and initialization for Static/Manual/Refreshing Iceberg tables. --- .../IcebergAutoRefreshTableLocationProvider.java | 14 ++++++++++++-- .../IcebergManualRefreshTableLocationProvider.java | 4 +++- .../layout/IcebergStaticTableLocationProvider.java | 9 ++++++++- 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java index 18611e58bf3..25c413a7183 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -63,6 +63,13 @@ public String getImplementationName() { return IMPLEMENTATION_NAME; } + @Override + protected synchronized void doInitialization() { + if (!isInitialized()) { + refreshLocations(); + } + } + @Override public synchronized void refresh() { if (locationKeyFinder.maybeUpdateSnapshot()) { @@ -91,8 +98,11 @@ public void update(Snapshot snapshot) { @Override protected final void activateUnderlyingDataSource() { - ensureInitialized(); - refreshLocations(); + synchronized (this) { + if (locationKeyFinder.maybeUpdateSnapshot() || !isInitialized()) { + refreshLocations(); + } + } subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this, refreshIntervalMs); activationSuccessful(this); } diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java index 1a249d4e15f..45d65f921ea 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -81,7 +81,9 @@ public synchronized void update(final Snapshot snapshot) { @Override protected synchronized void doInitialization() { - refreshLocations(); + if (!isInitialized()) { + refreshLocations(); + } } // ------------------------------------------------------------------------------------------------------------------ diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java index f6366635a27..a07806eb9fe 100644 --- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -48,7 +48,14 @@ public String getImplementationName() { } @Override - public void refresh() { + protected synchronized void doInitialization() { + if (!isInitialized()) { + refresh(); + } + } + + @Override + public synchronized void refresh() { beginTransaction(this); locationKeyFinder.findKeys(tlk -> handleTableLocationKeyAdded(tlk, this)); endTransaction(this); From e5c8f4964ac778833089d88cd61818e60a333143 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 22 Oct 2024 10:09:10 -0700 Subject: [PATCH 54/56] Python fix for definition(), cleanup --- py/server/deephaven/experimental/iceberg.py | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 8b6a3aab2d8..f8ae6e57c2b 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -113,17 +113,17 @@ def __init__(self, try: builder = self.j_object_type.builder() - if table_definition is not None: + if table_definition: builder.tableDefinition(TableDefinition(table_definition).j_table_definition) - if data_instructions is not None: + if data_instructions: builder.dataInstructions(data_instructions.j_object) - if column_renames is not None: + if column_renames: for old_name, new_name in column_renames.items(): builder.putColumnRenames(old_name, new_name) - if update_mode is not None: + if update_mode: builder.updateMode(update_mode.j_object) self._j_object = builder.build() @@ -216,6 +216,9 @@ def definition(self, instructions: Optional[IcebergInstructions] = None, snapsho a table containing the table definition. """ + if instructions: + instructions = instructions.j_object + if snapshot_id is not None: return Table(self.j_object.definitionTable(snapshot_id, instructions)) return Table(self.j_object.definitionTable(instructions)) @@ -290,7 +293,7 @@ def tables(self, namespace: str) -> IcebergTableAdapter: def load_table(self, table_identifier: str) -> IcebergTable: """ - Load the table from the catalog using the provided instructions. + Load the table from the catalog. Args: table_identifier (str): the table to read. From 1f44a441926296a4d152766ce3d437cc454b2597 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 22 Oct 2024 12:40:46 -0700 Subject: [PATCH 55/56] More cleanup. --- py/server/deephaven/experimental/iceberg.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index f8ae6e57c2b..81cabdf47c8 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -278,7 +278,7 @@ def namespaces(self, namespace: Optional[str] = None) -> Table: return Table(self.j_object.namespaces(namespace)) return Table(self.j_object.namespaces()) - def tables(self, namespace: str) -> IcebergTableAdapter: + def tables(self, namespace: str) -> Table: """ Returns information on the tables in the specified namespace as a Deephaven table. @@ -291,7 +291,7 @@ def tables(self, namespace: str) -> IcebergTableAdapter: return Table(self.j_object.tables(namespace)) - def load_table(self, table_identifier: str) -> IcebergTable: + def load_table(self, table_identifier: str) -> IcebergCatalogAdapter: """ Load the table from the catalog. From 300f836bfe617161a0c7540cf7b50e5aaa9aa5a7 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Tue, 22 Oct 2024 15:11:46 -0700 Subject: [PATCH 56/56] Final pre-merge change. --- py/server/deephaven/experimental/iceberg.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 81cabdf47c8..41fb0af901a 100644 --- a/py/server/deephaven/experimental/iceberg.py +++ b/py/server/deephaven/experimental/iceberg.py @@ -194,9 +194,6 @@ def snapshots(self) -> Table: - `Summary`: additional information about this snapshot from the Iceberg metadata. - `SnapshotObject`: a Java object containing the Iceberg API snapshot. - Args: - table_identifier (str): the table from which to list snapshots. - Returns: a table containing the snapshot information. """