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 ccb114cfbe1..cc76ffe0c44 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 notification to all subscribers. Clean up any GC'd subscriptions. + * + * @param procedure The notification procedure to invoke + * @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 firstNotification, + @Nullable final NOTIFICATION_TYPE secondNotification, + 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, firstNotification, secondNotification); + } + ++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/ColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java index 1d5ba841b7a..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 @@ -6,11 +6,7 @@ import io.deephaven.engine.liveness.LivenessReferent; 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 +49,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() @@ -116,8 +112,7 @@ public interface ColumnSourceManager extends LivenessReferent { /** * 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 ImmutableTableLocationKey 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 a43b6a88068..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 @@ -6,6 +6,7 @@ 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.select.analyzers.SelectAndViewAnalyzer; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; @@ -204,7 +205,7 @@ 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, @NotNull final Collection locationKeys) { @@ -221,30 +222,42 @@ 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(LiveSupplier::get) + .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(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, - ImmutableTableLocationKey.class, null)); + // Add the tracked keys to the table + // noinspection unchecked,rawtypes + partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource( + (Collection) (Collection) foundLocationKeys, + LiveSupplier.class, + null)); + final Table filteredColumnPartitionTable = TableTools .newTable(foundLocationKeys.size(), partitionTableColumnNames, partitionTableColumnSources) .where(Filter.and(partitioningColumnFilters)); if (filteredColumnPartitionTable.size() == foundLocationKeys.size()) { return foundLocationKeys; } - final Iterable iterable = - () -> filteredColumnPartitionTable.columnIterator(LOCATION_KEY_COLUMN_NAME); + + // 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/SourcePartitionedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourcePartitionedTable.java index e772241ca3a..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.*; @@ -163,9 +164,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.get())); + }, locationKeyMatcher); + try (final RowSet added = sortAndAddLocations(locations.stream())) { resultRows.insert(added); } } @@ -213,9 +217,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) { @@ -242,6 +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(LiveSupplier::get) .filter(locationKeyMatcher) .map(tableLocationProvider::getTableLocation) .map(PendingLocationState::new) @@ -263,6 +273,7 @@ private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpd final Set relevantRemovedLocations = locationUpdate.getPendingRemovedLocationKeys() .stream() + .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 e3508fc45c7..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 @@ -5,26 +5,23 @@ 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.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.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; -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; import org.apache.commons.lang3.mutable.MutableObject; import org.jetbrains.annotations.NotNull; +import java.util.ArrayList; import java.util.Collection; /** @@ -101,12 +98,12 @@ 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); - setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE); } /** @@ -144,39 +141,49 @@ private void initializeAvailableLocations() { if (isRefreshing()) { final TableLocationSubscriptionBuffer locationBuffer = new TableLocationSubscriptionBuffer(locationProvider); - final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = - locationBuffer.processPending(); - - maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); - maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); + manage(locationBuffer); + try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = + locationBuffer.processPending()) { + maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); + maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); + } updateSourceRegistrar.addSource(locationChangePoller = new LocationChangePoller(locationBuffer)); } else { locationProvider.refresh(); - maybeAddLocations(locationProvider.getTableLocationKeys()); + 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))); + .forEach(lk -> columnSourceManager.addLocation(locationProvider.getTableLocation(lk.get()))); } - private ImmutableTableLocationKey[] maybeRemoveLocations( - @NotNull final Collection removedKeys) { + private void maybeRemoveLocations( + @NotNull final Collection> removedKeys) { if (removedKeys.isEmpty()) { - return ImmutableTableLocationKey.ZERO_LENGTH_IMMUTABLE_TABLE_LOCATION_KEY_ARRAY; + return; } - return filterLocationKeys(removedKeys).stream() - .filter(columnSourceManager::removeLocationKey) - .toArray(ImmutableTableLocationKey[]::new); + filterLocationKeys(removedKeys).stream() + .map(LiveSupplier::get) + .forEach(columnSourceManager::removeLocationKey); } private void initializeLocationSizes() { @@ -216,14 +223,11 @@ private LocationChangePoller(@NotNull final TableLocationSubscriptionBuffer loca @Override 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); + try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = + locationBuffer.processPending()) { + maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys()); + maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); } - maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); // This class previously had functionality to notify "location listeners", but it was never used. // Resurrect from git history if needed. @@ -232,13 +236,16 @@ protected void instrumentedRefresh() { return; } - final RowSet added = columnSourceManager.refresh(); - if (added.isEmpty()) { + final TableUpdate update = columnSourceManager.refresh(); + if (update.empty()) { + update.release(); return; } - rowSet.insert(added); - notifyListeners(added, RowSetFactory.empty(), RowSetFactory.empty()); + Assert.eqTrue(update.shifted().empty(), "update.shifted().empty()"); + rowSet.remove(update.removed()); + rowSet.insert(update.added()); + notifyListeners(update); } @Override @@ -257,8 +264,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..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,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.LivenessReferent; 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, 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 84969d6249f..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,11 +3,14 @@ // 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; import java.util.Collection; +import java.util.function.Consumer; +import java.util.function.Predicate; /** * Discovery utility for {@link TableLocation}s for a given table. @@ -27,20 +30,55 @@ public interface TableLocationProvider extends NamedImplementation { interface Listener extends BasicTableDataListener { /** - * 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. + * Begin a transaction that collects location key additions and removals to be processed atomically. * - * @param tableLocationKey The new table location key + * @param token A token to identify the transaction. */ - void handleTableLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey); + void beginTransaction(@NotNull Object token); /** - * Notify the listener of a {@link TableLocationKey} that has been removed. + * End the transaction and process the location changes. * - * @param tableLocationKey The table location key that was removed + * @param token A token to identify the transaction. */ - void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey); + 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 + * 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 LiveSupplier tableLocationKey); + + /** + * 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 LiveSupplier tableLocationKey); + + /** + *

+ * Notify the listener of collections of {@link TableLocationKey TableLocationKeys} added or removed while + * 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. + */ + default void handleTableLocationKeysUpdate( + @NotNull Collection> addedKeys, + @NotNull Collection> removedKeys) { + removedKeys.forEach(this::handleTableLocationKeyRemoved); + addedKeys.forEach(this::handleTableLocationKeyAdded); + } } /** @@ -86,7 +124,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(); @@ -95,21 +133,34 @@ 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 + * @param consumer A consumer to receive the location keys */ - @NotNull - Collection getTableLocationKeys(); + default void getTableLocationKeys(Consumer> consumer) { + getTableLocationKeys(consumer, key -> true); + } + + /** + * 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); /** * 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); + boolean hasTableLocationKey(@NotNull 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 @@ -122,8 +173,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/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index 8391c0a774f..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 @@ -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; @@ -26,7 +27,7 @@ */ public abstract class AbstractTableLocation extends SubscriptionAggregator - implements TableLocation { + implements TableLocation, DelegatingLivenessReferent { private final ImmutableTableKey tableKey; private final ImmutableTableLocationKey tableLocationKey; @@ -35,6 +36,8 @@ public abstract class AbstractTableLocation private final KeyedObjectHashMap columnLocations = new KeyedObjectHashMap<>(StringUtils.charSequenceKey()); + private final ReferenceCountedLivenessReferent livenessReferent; + @SuppressWarnings("rawtypes") private static final AtomicReferenceFieldUpdater CACHED_DATA_INDEXES_UPDATER = AtomicReferenceFieldUpdater.newUpdater( @@ -58,6 +61,13 @@ protected AbstractTableLocation(@NotNull final TableKey tableKey, super(supportsSubscriptions); this.tableKey = Require.neqNull(tableKey, "tableKey").makeImmutable(); this.tableLocationKey = Require.neqNull(tableLocationKey, "tableLocationKey").makeImmutable(); + + livenessReferent = new ReferenceCountedLivenessNode(false) { + @Override + protected void destroy() { + AbstractTableLocation.this.destroy(); + } + }; } @Override @@ -65,6 +75,10 @@ public final String toString() { return toStringHelper(); } + @Override + public LivenessReferent asLivenessReferent() { + return livenessReferent; + } // ------------------------------------------------------------------------------------------------------------------ // TableLocationState implementation @@ -158,7 +172,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 +243,19 @@ public final BasicDataIndex getDataIndex(@NotNull final String... columns) { @InternalUseOnly @Nullable public abstract BasicDataIndex loadDataIndex(@NotNull String... columns); + + // ------------------------------------------------------------------------------------------------------------------ + // Reference counting implementation + // ------------------------------------------------------------------------------------------------------------------ + + /** + * The reference count has reached zero or we are being GC'd, we can clear this location and release any resources. + */ + 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 b474775955a..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 @@ -3,6 +3,10 @@ // 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.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; @@ -11,12 +15,15 @@ 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}. *

* 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, Object)}. *

* Note that subclasses are responsible for determining when it's appropriate to call {@link #setInitialized()} and/or * override {@link #doInitialization()}. @@ -25,23 +32,128 @@ public abstract class AbstractTableLocationProvider extends SubscriptionAggregator implements TableLocationProvider { + 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; + + synchronized void addLocationKey(ImmutableTableLocationKey locationKey) { + if (locationsAdded == EMPTY_TABLE_LOCATION_KEYS) { + locationsAdded = new HashSet<>(); + } + // 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."); + } + } + + synchronized void removeLocationKey(ImmutableTableLocationKey locationKey) { + if (locationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + locationsRemoved = new HashSet<>(); + } + // 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."); + } + // 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."); + } + } + } + + 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; + // Open transactions that are being accumulated + 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 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 KeyedObjectHashMap tableLocations = - new KeyedObjectHashMap<>(LocationKeyDefinition.INSTANCE); - @SuppressWarnings("unchecked") - private final Collection unmodifiableTableLocationKeys = - (Collection) (Collection) Collections - .unmodifiableCollection(tableLocations.keySet()); + private final StandaloneLivenessManager livenessManager; private volatile boolean initialized; @@ -58,6 +170,8 @@ protected AbstractTableLocationProvider(@NotNull final TableKey tableKey, final super(supportsSubscriptions); this.tableKey = tableKey.makeImmutable(); this.partitionKeys = null; + + livenessManager = new StandaloneLivenessManager(false); } /** @@ -84,7 +198,92 @@ public final ImmutableTableKey getKey() { @Override protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) { - unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKey); + // 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()); + } + } + + /** + * 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) { + // 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(); + }); + } + + /** + * 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) { + // 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."); + } + + // Return early if there are no changes to process + if (transaction.locationsAdded.isEmpty() && transaction.locationsRemoved.isEmpty()) { + 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())) { + synchronized (subscriptions) { + // Push the notifications to the subscribers + if (subscriptions.deliverNotification( + Listener::handleTableLocationKeysUpdate, + addedKeys, + removedKeys, + 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); + } + } } /** @@ -93,44 +292,120 @@ protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider * @param locationKey The new key * @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) { + 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( + @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."); + } + // Store an immutable key + transaction.addLocationKey(locationKey.makeImmutable()); + return; + } + if (!supportsSubscriptions()) { - tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable); - visitLocationKey(toKeyImmutable(locationKey)); + tableLocationKeyMap.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 // certainly just deliver the notification in observeInsert. That said, I'm happier with this approach, // 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); - if (subscriptions.deliverNotification(Listener::handleTableLocationKey, toKeyImmutable(result), true)) { + verifyPartitionKeys(result.get()); + if (subscriptions.deliverNotification( + Listener::handleTableLocationKeyAdded, + (LiveSupplier) result, + true)) { + onEmpty(); + } + } + } + } + + /** + * 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(locationKey.makeImmutable()); + return; + } + + if (!supportsSubscriptions()) { + final TrackedKeySupplier trackedKey = tableLocationKeyMap.get(locationKey); + trackedKey.deactivate(); + livenessManager.unmanage(trackedKey); + return; + } + + // If we're not in a transaction, we should push this key immediately. + synchronized (subscriptions) { + final TrackedKeySupplier trackedKey = tableLocationKeyMap.removeKey(locationKey); + if (trackedKey != null) { + trackedKey.deactivate(); + if (subscriptions.deliverNotification( + Listener::handleTableLocationKeyRemoved, + (LiveSupplier) trackedKey, + true)) { onEmpty(); } + livenessManager.unmanage(trackedKey); } } } /** - * 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 - * 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. */ 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; - return locationKey.makeImmutable(); + + final TrackedKeySupplier trackedKey = toTrackedKey(locationKey); + livenessManager.manage(trackedKey); + + return trackedKey; } /** @@ -178,45 +453,47 @@ protected void doInitialization() { } @Override - @NotNull - public final Collection getTableLocationKeys() { - return unmodifiableTableLocationKeys; + public void getTableLocationKeys( + final Consumer> consumer, + final Predicate filter) { + // 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); + } } @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 TableLocationKey immutableKey = (TableLocationKey) current; - // noinspection SynchronizationOnLocalVariableOrMethodParameter - synchronized (immutableKey) { - current = tableLocations.get(immutableKey); - if (immutableKey == current) { - // Note, this may contend for the lock on tableLocations - tableLocations.add(current = makeTableLocation(immutableKey)); + 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 (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(ImmutableTableLocationKey) 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. * @@ -237,31 +514,7 @@ 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(); - } - } - } - - /** - * Notify subscribers that {@code locationKey} was removed. - * - * @param locationKey the TableLocation that was removed - */ - protected void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey locationKey) { - if (supportsSubscriptions()) { - synchronized (subscriptions) { - if (subscriptions.deliverNotification(Listener::handleTableLocationKeyRemoved, locationKey, true)) { - onEmpty(); - } - } - } + handleTableLocationKeyRemoved(locationKey, null); } private void verifyPartitionKeys(@NotNull final TableLocationKey locationKey) { @@ -275,33 +528,24 @@ 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(); } } - private static TableLocationKey toKey(@NotNull final Object keyOrLocation) { - if (keyOrLocation instanceof TableLocation) { - return ((TableLocation) 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 TrackedKeySupplier toTrackedKey(@NotNull final TableLocationKey locationKey) { + return new TrackedKeySupplier(locationKey.makeImmutable(), this::releaseLocationKey); } private static boolean equals(Collection c1, Collection c2) { @@ -316,4 +560,14 @@ private static boolean equals(Collection c1, Collection c2) { } return !i2.hasNext(); } + + /** + * Called when every reference to the {@link LiveSupplier key} has been released. + * + * @param locationKey the key to release + */ + private void releaseLocationKey(@NotNull final TrackedKeySupplier locationKey) { + // We can now remove the key from the tableLocations map + tableLocationKeyMap.removeKey(locationKey.get()); + } } 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 d5ad0e1faf0..5924aeb6769 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; @@ -13,6 +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; /** @@ -145,7 +148,7 @@ public void subscribe(@NotNull final Listener listener) { p.subscribe(listener); } else { p.refresh(); - p.getTableLocationKeys().forEach(listener::handleTableLocationKey); + p.getTableLocationKeys(listener::handleTableLocationKeyAdded); } }); } @@ -171,29 +174,31 @@ public TableLocationProvider ensureInitialized() { } @Override - @NotNull - public Collection getTableLocationKeys() { - final Set locationKeys = new KeyedObjectHashSet<>(KeyKeyDefinition.INSTANCE); + 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(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 (!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())); + + } + }, filter)); } } @@ -255,16 +260,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 ImmutableTableLocationKey tableLocationKey) { - return tableLocationKey; + 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 21c03635091..7c68e71c3f6 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; @@ -12,7 +13,8 @@ import java.util.Collection; import java.util.Map; import java.util.WeakHashMap; -import java.util.stream.Collectors; +import java.util.function.Consumer; +import java.util.function.Predicate; /** * {@link TableDataService} implementation with support to filter the provided {@link TableLocation}s. @@ -135,9 +137,10 @@ public TableLocationProvider ensureInitialized() { } @Override - public @NotNull Collection getTableLocationKeys() { - return inputProvider.getTableLocationKeys().stream().filter(locationKeyFilter::accept) - .collect(Collectors.toList()); + public void getTableLocationKeys( + final Consumer> consumer, + final Predicate filter) { + inputProvider.getTableLocationKeys(consumer, filter); } @Override @@ -168,23 +171,53 @@ private FilteringListener(@NotNull final TableLocationProvider.Listener outputLi } @Override - public void handleTableLocationKey(@NotNull final ImmutableTableLocationKey tableLocationKey) { + 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) { 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); + if (outputListener != null && locationKeyFilter.accept(tableLocationKey.get())) { + outputListener.handleTableLocationKeyAdded(tableLocationKey); } } @Override - public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyRemoved( + @NotNull final LiveSupplier tableLocationKey) { final TableLocationProvider.Listener outputListener = getWrapped(); - if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) { + if (outputListener != null && locationKeyFilter.accept(tableLocationKey.get())) { outputListener.handleTableLocationKeyRemoved(tableLocationKey); } } + @Override + public void handleTableLocationKeysUpdate( + @NotNull Collection> addedKeys, + @NotNull 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/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java index 361a18c8c44..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 @@ -3,10 +3,7 @@ // 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; @@ -48,7 +45,7 @@ public String getImplementationName() { @Override public void refresh() { - locationKeyFinder.findKeys(this::handleTableLocationKey); + locationKeyFinder.findKeys(this::handleTableLocationKeyAdded); setInitialized(); } 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..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,20 +3,37 @@ // 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; -import java.util.Collection; -import java.util.Collections; +import java.util.function.Consumer; +import java.util.function.Predicate; /** * 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 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; /** @@ -24,6 +41,8 @@ public final class SingleTableLocationProvider implements TableLocationProvider */ public SingleTableLocationProvider(@NotNull final TableLocation tableLocation) { this.tableLocation = tableLocation; + // TODO: it seems like we should manage this, but SingleTableLocationProvider isn't a LivenessManager. + immutableKeySupplier = new TrackedKeySupplier(tableLocation.getKey()); } @Override @@ -59,10 +78,13 @@ public TableLocationProvider ensureInitialized() { return this; } - @NotNull @Override - public Collection getTableLocationKeys() { - return Collections.singleton(tableLocation.getKey()); + public void getTableLocationKeys( + final Consumer> consumer, + final Predicate filter) { + if (filter.test(immutableKeySupplier.get())) { + consumer.accept(immutableKeySupplier); + } } @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 fae32e8a0a1..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,54 +4,67 @@ 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.ImmutableTableLocationKey; 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; -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. */ -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; private boolean subscribed = false; private final Object updateLock = new Object(); - private Set pendingLocationKeys = EMPTY_TABLE_LOCATION_KEYS; - private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; + // These sets represent adds and removes from completed transactions. + 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); + } } /** @@ -60,7 +73,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? @@ -71,16 +84,18 @@ 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); + 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 = pendingLocationKeys; - pendingLocationKeys = EMPTY_TABLE_LOCATION_KEYS; + resultLocationKeys = pendingLocationsAdded; + pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; resultLocationsRemoved = pendingLocationsRemoved; pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; resultException = pendingException; @@ -105,7 +120,9 @@ public synchronized void reset() { subscribed = false; } synchronized (updateLock) { - pendingLocationKeys = EMPTY_TABLE_LOCATION_KEYS; + pendingLocationsAdded.forEach(this::unmanage); + pendingLocationsRemoved.forEach(this::unmanage); + pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS; pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS; pendingException = null; } @@ -116,30 +133,91 @@ public synchronized void reset() { // ------------------------------------------------------------------------------------------------------------------ @Override - public void handleTableLocationKey(@NotNull final ImmutableTableLocationKey tableLocationKey) { + 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) { - if (pendingLocationKeys == EMPTY_TABLE_LOCATION_KEYS) { - pendingLocationKeys = new HashSet<>(); + // 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<>(); } - pendingLocationKeys.add(tableLocationKey); + manage(tableLocationKey); + pendingLocationsAdded.add(tableLocationKey); } } @Override - public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) { + public void handleTableLocationKeyRemoved(@NotNull final LiveSupplier tableLocationKey) { synchronized (updateLock) { - // If we remove something that was pending to be added, just discard both. - if (pendingLocationKeys.remove(tableLocationKey)) { + // 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 and has not been replaced."); + } if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { pendingLocationsRemoved = new HashSet<>(); } + manage(tableLocationKey); pendingLocationsRemoved.add(tableLocationKey); } } + @Override + public void handleTableLocationKeysUpdate( + @Nullable Collection> addedKeys, + @Nullable Collection> removedKeys) { + synchronized (updateLock) { + if (removedKeys != null) { + for (final LiveSupplier 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 and has not been replaced."); + } + if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) { + pendingLocationsRemoved = new HashSet<>(); + } + manage(removedTableLocationKey); + pendingLocationsRemoved.add(removedTableLocationKey); + } + } + if (addedKeys != null) { + for (final LiveSupplier 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<>(); + } + manage(addedTableLocationKey); + pendingLocationsAdded.add(addedTableLocationKey); + } + } + } + } + @Override public void handleException(@NotNull final TableDataException exception) { 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 8c4e016607c..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 @@ -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}. @@ -29,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; @@ -44,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"); @@ -76,7 +76,8 @@ private abstract class ScheduledSubscriptionTask { - private ScheduledTableLocationProviderRefresh(@NotNull AbstractTableLocationProvider tableLocationProvider) { - super(tableLocationProvider, tableLocationProviderRefreshIntervalMillis); + private ScheduledTableLocationProviderRefresh( + @NotNull final AbstractTableLocationProvider tableLocationProvider, + final long refreshIntervalMillis) { + super(tableLocationProvider, refreshIntervalMillis); providerSubscriptions.increment(1); } @@ -131,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); } @@ -153,12 +158,27 @@ public void cancel() { @Override public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( @NotNull final AbstractTableLocationProvider tableLocationProvider) { - return new ScheduledTableLocationProviderRefresh(tableLocationProvider); + return new ScheduledTableLocationProviderRefresh(tableLocationProvider, + tableLocationProviderDefaultRefreshIntervalMillis); + } + + @Override + public CancellableSubscriptionToken scheduleTableLocationProviderRefresh( + @NotNull final AbstractTableLocationProvider tableLocationProvider, + final long refreshIntervalMillis) { + return new ScheduledTableLocationProviderRefresh(tableLocationProvider, refreshIntervalMillis); } @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 c6b823c3139..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 @@ -41,13 +41,35 @@ CancellableSubscriptionToken scheduleTableLocationProviderRefresh( @NotNull AbstractTableLocationProvider tableLocationProvider); /** - * Schedule run for an AbstractTableLocation. + * 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 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. * @@ -126,10 +148,24 @@ 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) { 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/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 1889f5dbb25..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 @@ -4,20 +4,19 @@ package io.deephaven.engine.table.impl.sources.regioned; import io.deephaven.base.verify.Assert; -import io.deephaven.engine.liveness.LivenessArtifact; -import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.*; 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; 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; @@ -32,13 +31,21 @@ 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. */ -public class RegionedColumnSourceManager extends LivenessArtifact 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. */ @@ -101,6 +108,11 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col @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. */ @@ -108,6 +120,29 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col @ReferentialIntegrity private Runnable delayedErrorReference; + /** + * 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; + /** * Construct a column manager with the specified component factory and definitions. * @@ -120,7 +155,6 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col @NotNull final RegionedTableComponentFactory componentFactory, @NotNull final ColumnToCodecMappings codecMappings, @NotNull final List> columnDefinitions) { - super(false); this.isRefreshing = isRefreshing; this.columnDefinitions = columnDefinitions; @@ -146,6 +180,15 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col ? 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, @@ -160,12 +203,23 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col } }; if (isRefreshing) { + livenessNode.manage(includedLocationsTable); rowSetModifiedColumnSet = includedLocationsTable.newModifiedColumnSet(ROWS_SET_COLUMN_NAME); - manage(includedLocationsTable); } else { rowSetModifiedColumnSet = null; } } + + invalidateCommitter = new UpdateCommitter<>(this, + ExecutionContext.getContext().getUpdateGraph(), + (instance) -> { + synchronized (instance) { + invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate); + invalidatedLocations.clear(); + releasedLocations.forEach(instance::unmanage); + releasedLocations.clear(); + } + }); } @Override @@ -177,6 +231,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. + livenessNode.manage(tableLocation); emptyTableLocations.add(new EmptyTableLocationEntry(tableLocation)); } else { // Duplicate location - not allowed @@ -198,7 +254,7 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation) } @Override - public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locationKey) { + public synchronized void removeLocationKey(final @NotNull ImmutableTableLocationKey locationKey) { final IncludedTableLocationEntry includedLocation = includedTableLocations.remove(locationKey); final EmptyTableLocationEntry emptyLocation = emptyTableLocations.remove(locationKey); @@ -206,12 +262,18 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati 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) { - includedLocation.invalidate(); - return true; + orderedIncludedTableLocations.remove(includedLocation); + removedTableLocations.add(includedLocation); + if (includedLocation.location instanceof AbstractTableLocation) { + releasedLocations.add((AbstractTableLocation) includedLocation.location); + } + invalidateCommitter.maybeActivate(); } - - return false; } @Override @@ -219,8 +281,11 @@ 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 + final TableUpdateImpl update = update(true); // noinspection resource - final TrackingWritableRowSet initialRowSet = update(true).toTracking(); + 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 -> { @@ -229,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); } @@ -262,7 +327,7 @@ public synchronized TrackingWritableRowSet initialize() { } @Override - public synchronized WritableRowSet refresh() { + public synchronized TableUpdate refresh() { if (!isRefreshing) { throw new UnsupportedOperationException("Cannot refresh a static table"); } @@ -281,9 +346,28 @@ public void deliverError(@NotNull final Throwable error, @Nullable final TableLi } } - private WritableRowSet update(final boolean initializing) { + private TableUpdateImpl update(final boolean initializing) { final RowSetBuilderSequential addedRowSetBuilder = 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)); + for (final IncludedTableLocationEntry removedLocation : removedTableLocations) { + final long regionFirstKey = RegionedColumnSource.getFirstRowKey(removedLocation.regionIndex); + removedRowSetBuilder.appendRowSequenceWithOffset(removedLocation.rowSetAtLastUpdate, regionFirstKey); + removedRegionBuilder.appendKey(removedLocation.regionIndex); + } + + // 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(); // Ordering matters, since we're using a sequential builder. @@ -291,10 +375,10 @@ private WritableRowSet 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); } @@ -323,13 +407,16 @@ private WritableRowSet update(final boolean initializing) { emptyTableLocations.removeAll(entriesToInclude); - final int previousNumRegions = includedTableLocations.size(); - final int newNumRegions = previousNumRegions + entriesToInclude.size(); + final RowSetBuilderSequential addedRegionBuilder = + entriesToInclude.isEmpty() ? null : RowSetFactory.builderSequential(); + + final int prevMaxIndex = nextRegionIndex; + final int maxIndex = nextRegionIndex + (entriesToInclude.isEmpty() ? 0 : entriesToInclude.size()); if (!entriesToInclude.isEmpty()) { 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); entriesToInclude.stream().sorted().forEachOrdered((final EmptyTableLocationEntry entryToInclude) -> { final IncludedTableLocationEntry entry = new IncludedTableLocationEntry(entryToInclude); @@ -345,16 +432,18 @@ private WritableRowSet 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); }); } + final RowSet addedRegions = addedRegionBuilder == null ? RowSetFactory.empty() : addedRegionBuilder.build(); - if (previousNumRegions != newNumRegions) { - includedLocationsTable.getRowSet().writableCast().insertRange(previousNumRegions, newNumRegions - 1); + if (addedRegions.isNonempty()) { + includedLocationsTable.getRowSet().writableCast().insert(addedRegions); } if (initializing) { - Assert.eqZero(previousNumRegions, "previousNumRegions"); + Assert.eqZero(prevMaxIndex, "prevMaxIndex"); if (isRefreshing) { rowSetSource.startTrackingPrevValues(); includedLocationsTable.getRowSet().writableCast().initializePreviousValue(); @@ -364,21 +453,27 @@ private WritableRowSet update(final boolean initializing) { } } else { final RowSet modifiedRegions = modifiedRegionBuilder.build(); - if (previousNumRegions == newNumRegions && modifiedRegions.isEmpty()) { - modifiedRegions.close(); + final RowSet removedRegions = + removedRegionBuilder == null ? RowSetFactory.empty() : removedRegionBuilder.build(); + if (addedRegions.isEmpty() && modifiedRegions.isEmpty() && removedRegions.isEmpty()) { + SafeCloseable.closeAll(addedRegions, modifiedRegions, removedRegions); } 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); includedLocationsTable.notifyListeners(update); } } - return addedRowSetBuilder.build(); + return new TableUpdateImpl( + addedRowSetBuilder.build(), + removedRowSetBuilder == null ? RowSetFactory.empty() : removedRowSetBuilder.build(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.EMPTY); } @Override @@ -421,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. @@ -478,7 +578,7 @@ private class IncludedTableLocationEntry implements Comparable> columnLocationStates = new ArrayList<>(); /** @@ -495,14 +595,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)); @@ -555,11 +655,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()) { @@ -568,7 +668,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)); } 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 6e555a9d123..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 @@ -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,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; } }); @@ -320,7 +317,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 +358,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 +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/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()); 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/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/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/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/build.gradle b/extensions/iceberg/s3/build.gradle index dfb53c52388..ba927dcce29 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 c545ab68540..de336677f32 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.aws.AwsClientProperties; import org.apache.iceberg.aws.glue.GlueCatalog; @@ -67,6 +68,7 @@ public static IcebergCatalogAdapter createS3Rest( } final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; + catalog.setConf(new Configuration()); catalog.initialize(catalogName, properties); return new IcebergCatalogAdapter(catalog, properties); @@ -98,6 +100,7 @@ public static IcebergCatalogAdapter createGlue( properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI; + catalog.setConf(new Configuration()); catalog.initialize(catalogName, properties); return new IcebergCatalogAdapter(catalog, properties); 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 eb1640f07c2..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 @@ -5,13 +5,19 @@ 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.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; +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; @@ -43,8 +49,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; @Category(OutOfBandTest.class) public abstract class IcebergToolsTest { @@ -196,12 +202,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.listNamespacesAsTable(); - Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog"); + final Table table = adapter.namespaces(); + Assert.eq(table.size(), "table.size()", 2, "namespace in the catalog"); Assert.equals(table.getDefinition(), "table.getDefinition()", NAMESPACE_DEFINITION); } @@ -212,20 +218,20 @@ 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)"); 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); - Assert.eq(table.size(), "table.size()", 4, "4 tables in the namespace"); + Table table = adapter.tables(ns); + 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.listTablesAsTable("sales"); - Assert.eq(table.size(), "table.size()", 4, "4 tables in the namespace"); + table = adapter.tables("sales"); + Assert.eq(table.size(), "table.size()", 4, "tables in the namespace"); Assert.equals(table.getDefinition(), "table.getDefinition()", TABLES_DEFINITION); } @@ -238,20 +244,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, "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.listSnapshotsAsTable(tableIdentifier); - Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); + 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.listSnapshotsAsTable("sales.sales_multi"); - Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi"); + table = adapter.snapshots("sales.sales_multi"); + Assert.eq(table.size(), "table.size()", 6, "snapshots for sales/sales_multi"); Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION); } @@ -266,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, "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, "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); } @@ -287,14 +295,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, "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()", 100_000, "100_000 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); } @@ -309,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, "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, "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); } @@ -331,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, "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); } @@ -351,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, "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); } @@ -421,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, "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); } @@ -450,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, "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); } @@ -479,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, "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); } @@ -499,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, "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); } @@ -589,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, "100_000 rows in the table"); + Assert.eq(table.size(), "table.size()", 100_000, "expected rows in the table"); } @Test @@ -607,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, "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); } @@ -636,7 +645,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, "expected rows in the table"); Assert.equals(table.getDefinition(), "table.getDefinition()", expected); } @@ -716,23 +725,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, "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()", 54373, "54373 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()", 72603, "72603 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, "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, "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, "expected rows in the table"); + Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @Test @@ -747,39 +766,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, "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()", 54373, "54373 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()", 72603, "72603 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, "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, "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()", 18266, "18266 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(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, "expected 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, "expected rows in the table"); Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION); - table3 = adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions); - Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table"); + 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); + Assert.eq(table5.size(), "table5.size()", 0, "expected rows in the table"); + Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION); } @Test @@ -793,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, "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); } @@ -833,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, "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, "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, "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, "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); } @@ -895,4 +930,112 @@ public void testTableDefinitionWithInstructions() { tableDef = adapter.getTableDefinition("sales.sales_multi", localInstructions); Assert.equals(tableDef, "tableDef", userTableDef); } + + @Test + public void testManualRefreshingTable() 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 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, "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, "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); + } + + @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/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 d871fbb9fd0..2de91cb1f34 100644 Binary files a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/226480b9-ac20-4445-b67e-7cd3f7ef27d0-m0.avro differ 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 21c52f9fe1c..d91a8b5dc74 100644 Binary files a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/4b9446ba-c04a-440a-afad-f090aa27c81a-m0.avro differ 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 00000000000..463995b0f52 Binary files /dev/null and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/bb0d7049-0780-4636-972a-7f46bb6c395a-m0.avro differ 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 9501a6ce37e..3e3c02833a0 100644 Binary files a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/ddb2a84e-df73-4cb9-a92f-18d737a54489-m0.avro differ 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 462894acbde..2cda88a5e78 100644 Binary files a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/e1bd1067-e40a-4222-bab9-959f5af4dd4d-m0.avro differ diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1277776933184906785-1-bb0d7049-0780-4636-972a-7f46bb6c395a.avro similarity index 90% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1277776933184906785-1-bb0d7049-0780-4636-972a-7f46bb6c395a.avro index b5659a5c9b9..efb0d789589 100644 Binary files a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1277776933184906785-1-bb0d7049-0780-4636-972a-7f46bb6c395a.avro differ diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro similarity index 88% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro index c6e3e85459b..dc24dd3d8e4 100644 Binary files a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro differ 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 28d439215a0..ca67d791ceb 100644 Binary files a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3825168261540020388-1-bdc6d711-4699-4d21-a7f8-907c7740c70c.avro differ diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro similarity index 89% rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro index 8774b62b1e7..b77fb08206c 100644 Binary files a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro differ 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 00000000000..790df47dae8 Binary files /dev/null and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro differ diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro new file mode 100644 index 00000000000..2a43520a033 Binary files /dev/null and b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro differ 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..5d2b302b611 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java @@ -0,0 +1,124 @@ +// +// 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.IcebergTableAdapter; +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 TableDataRefreshService refreshService; + 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 IcebergTableAdapter adapter, + @NotNull final TableIdentifier tableIdentifier) { + super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier); + + Assert.neqNull(refreshService, "refreshService"); + this.refreshService = refreshService; + this.refreshIntervalMs = refreshIntervalMs; + } + + // ------------------------------------------------------------------------------------------------------------------ + // AbstractTableLocationProvider implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + @Override + public synchronized void refresh() { + adapter.refresh(); + final Snapshot latestSnapshot = adapter.currentSnapshot(); + if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) { + locationKeyFinder.snapshot = latestSnapshot; + refreshSnapshot(); + } + } + + @Override + public void update() { + 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 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(); + } + + // ------------------------------------------------------------------------------------------------------------------ + // 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/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java index 7bf0f5222a2..678c2c1c805 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,10 +10,10 @@ 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 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; @@ -25,24 +25,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; - - /** - * The {@link Snapshot} to discover locations for. - */ - final Snapshot snapshot; + 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. @@ -54,6 +45,11 @@ public abstract class IcebergBaseLayout implements TableLocationKeyFinder cache; + /** + * The {@link Snapshot} from which to discover data files. + */ + Snapshot snapshot; + /** * The data instructions provider for creating instructions from URI and user-supplied properties. */ @@ -101,30 +97,26 @@ 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, @NotNull final DataInstructionsProviderLoader dataInstructionsProvider) { - this.tableDef = tableDef; - this.table = table; + this.tableAdapter = tableAdapter; this.snapshot = tableSnapshot; - this.fileIO = fileIO; this.instructions = instructions; this.dataInstructionsProvider = dataInstructionsProvider; + this.tableDef = tableAdapter.definition(tableSnapshot, instructions); + this.cache = new HashMap<>(); } @@ -132,9 +124,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) { @@ -142,7 +135,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 = @@ -155,7 +148,7 @@ public synchronized void findKeys(@NotNull final Consumer 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, @NotNull final DataInstructionsProviderLoader dataInstructionsProvider) { - super(tableDef, table, tableSnapshot, fileIO, instructions, dataInstructionsProvider); + super(tableAdapter, tableSnapshot, instructions, dataInstructionsProvider); // 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. @@ -83,7 +78,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 new file mode 100644 index 00000000000..68d558b4d7f --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java @@ -0,0 +1,132 @@ +// +// 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.IcebergTableAdapter; +import org.apache.iceberg.Snapshot; +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} + * discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a + * {@link TableLocationFactory}. + *

+ */ +public class IcebergManualRefreshTableLocationProvider + extends IcebergTableLocationProviderBase { + + private static final String IMPLEMENTATION_NAME = IcebergManualRefreshTableLocationProvider.class.getSimpleName(); + + private boolean initialized = false; + + public IcebergManualRefreshTableLocationProvider( + @NotNull final TK tableKey, + @NotNull final IcebergBaseLayout locationKeyFinder, + @NotNull final TableLocationFactory locationFactory, + @NotNull final IcebergTableAdapter adapter, + @NotNull final TableIdentifier tableIdentifier) { + super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier); + } + + // ------------------------------------------------------------------------------------------------------------------ + // AbstractTableLocationProvider implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + @Override + public synchronized void refresh() { + // There should be no refresh service for this provider. + throw new UnsupportedOperationException(); + } + + @Override + public synchronized void update() { + adapter.refresh(); + update(adapter.currentSnapshot()); + } + + @Override + public synchronized void update(final long snapshotId) { + adapter.refresh(); + final List 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); + } + + @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(); + } + + // ------------------------------------------------------------------------------------------------------------------ + // SubscriptionAggregator implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + protected void activateUnderlyingDataSource() { + if (!initialized) { + refreshSnapshot(); + activationSuccessful(this); + initialized = true; + } + } + + @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 new file mode 100644 index 00000000000..53a5e9fc172 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java @@ -0,0 +1,84 @@ +// +// 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 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 TableIdentifier tableIdentifier) { + super(tableKey, locationKeyFinder, locationFactory, false, null, tableIdentifier); + } + + // ------------------------------------------------------------------------------------------------------------------ + // AbstractTableLocationProvider implementation + // ------------------------------------------------------------------------------------------------------------------ + + @Override + public String getImplementationName() { + return IMPLEMENTATION_NAME; + } + + @Override + public void refresh() { + beginTransaction(this); + locationKeyFinder.findKeys(tlk -> handleTableLocationKeyAdded(tlk, this)); + endTransaction(this); + 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"); + } + + // ------------------------------------------------------------------------------------------------------------------ + // 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 new file mode 100644 index 00000000000..9ec4eeb18b0 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java @@ -0,0 +1,66 @@ +// +// 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.iceberg.util.IcebergTableAdapter; +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 IcebergTableAdapter adapter; + final TableIdentifier tableIdentifier; + + public IcebergTableLocationProviderBase( + @NotNull final TK tableKey, + @NotNull final IcebergBaseLayout locationKeyFinder, + @NotNull final TableLocationFactory locationFactory, + final boolean isRefreshing, + @Nullable final IcebergTableAdapter adapter, + @NotNull final TableIdentifier tableIdentifier) { + super(tableKey, isRefreshing); + this.locationKeyFinder = locationKeyFinder; + this.locationFactory = locationFactory; + this.adapter = adapter; + this.tableIdentifier = tableIdentifier; + } + + /** + * 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, null); + } +} 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 b76a750602d..9c1e659c504 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,45 +3,21 @@ // package io.deephaven.iceberg.util; -import io.deephaven.api.util.NameValidator; 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.impl.AbstractTableLocationProvider; -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.engine.util.TableTools; -import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; -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.types.Type; -import org.apache.iceberg.types.Types; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import io.deephaven.iceberg.internal.DataInstructionsProviderLoader; -import java.time.Instant; -import java.time.LocalDateTime; import java.util.*; -import java.util.stream.Collectors; public class IcebergCatalogAdapter { @@ -56,14 +32,6 @@ 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 DataInstructionsProviderLoader dataInstructionsProvider; @@ -86,121 +54,6 @@ public class IcebergCatalogAdapter { dataInstructionsProvider = DataInstructionsProviderLoader.create(Map.copyOf(properties)); } - /** - * 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 @@ -235,8 +88,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()); } /** @@ -245,7 +98,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(); @@ -278,8 +131,9 @@ public Table listNamespacesAsTable(@NotNull final Namespace namespace) { * * @return A {@link Table table} of all namespaces. */ - public Table listNamespacesAsTable(@NotNull final String... namespace) { - return listNamespacesAsTable(Namespace.of(namespace)); + @SuppressWarnings("unused") + public Table namespaces(@NotNull final String... namespace) { + return namespaces(Namespace.of(namespace)); } /** @@ -299,7 +153,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(); @@ -331,8 +185,23 @@ 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, dataInstructionsProvider); } /** @@ -341,125 +210,75 @@ 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. */ - public List listSnapshots(@NotNull final TableIdentifier tableIdentifier) { - final List snapshots = new ArrayList<>(); - catalog.loadTable(tableIdentifier).snapshots().forEach(snapshots::add); - return snapshots; + @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 as a Deephaven {@link Table table}. The resulting - * table will be static and contain the same information as {@link #listSnapshots(TableIdentifier)}. + * 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 tables in the given namespace. + * @return A list of all snapshots of the given table. */ - 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 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 same information as {@link #listSnapshots(TableIdentifier)}. + * 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. */ - public Table listSnapshotsAsTable(@NotNull final String tableIdentifier) { - return listSnapshotsAsTable(TableIdentifier.parse(tableIdentifier)); + @Deprecated(forRemoval = true) + public Table snapshots(@NotNull final TableIdentifier tableIdentifier) { + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.snapshots(); } /** - * Get a specific {@link Snapshot snapshot} of a given Iceberg table (or null if it does not exist). + * 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 - * @param snapshotId The id of the snapshot to retrieve - * @return The snapshot with the given id, or null if it does not exist - */ - private Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long snapshotId) { - return listSnapshots(tableIdentifier).stream() - .filter(snapshot -> snapshot.snapshotId() == snapshotId) - .findFirst() - .orElse(null); - } - - /** - * Get a legalized column rename map from a table schema and user instructions. + * @param tableIdentifier The identifier of the table from which to gather snapshots. + * @return A list of all tables in the given namespace. */ - 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)); } /** @@ -470,12 +289,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); } /** @@ -486,11 +305,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); } /** @@ -502,20 +322,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); - if (tableSnapshot == null) { - throw new IllegalArgumentException("Snapshot with id " + snapshotId + " not found"); - } - - // Load the table from the catalog. - return getTableDefinition(tableId, tableSnapshot, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definition(snapshotId, instructions); } /** @@ -527,12 +340,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); } /** @@ -543,11 +357,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); } /** @@ -558,11 +373,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); } /** @@ -574,19 +390,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); - if (tableSnapshot == null) { - throw new IllegalArgumentException("Snapshot with id " + snapshotId + " not found"); - } - - return getTableDefinitionTable(tableId, tableSnapshot, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.definitionTable(snapshotId, instructions); } /** @@ -598,35 +408,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); } /** @@ -637,10 +425,12 @@ private TableDefinition getTableDefinitionInternal( * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + @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); } /** @@ -651,58 +441,12 @@ public Table readTable( * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + @Deprecated(forRemoval = true) + public IcebergTable readTable( @NotNull final String tableIdentifier, @Nullable final IcebergInstructions instructions) { - return readTable(TableIdentifier.parse(tableIdentifier), instructions); - } - - /** - * Retrieve a snapshot of an Iceberg table from the Iceberg catalog. - * - * @param tableIdentifier The table identifier to load - * @param tableSnapshotId The snapshot id to load - * @return The loaded table - * @throws IllegalArgumentException if the snapshot with the given id is not found - */ - private Snapshot getTableSnapshot(@NotNull TableIdentifier tableIdentifier, long tableSnapshotId) { - return listSnapshots(tableIdentifier).stream() - .filter(snapshot -> snapshot.snapshotId() == tableSnapshotId) - .findFirst() - .orElseThrow(() -> new IllegalArgumentException( - "Snapshot with id " + tableSnapshotId + " for table " + tableIdentifier + " not found")); - } - - /** - * 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 - * @return The loaded table - */ - @SuppressWarnings("unused") - public Table readTable(@NotNull final TableIdentifier tableIdentifier, final long tableSnapshotId) { - // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = getTableSnapshot(tableIdentifier, tableSnapshotId); - - return readTableInternal(tableIdentifier, tableSnapshot, null); - } - - - /** - * 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 - * @return The loaded table - */ - @SuppressWarnings("unused") - public Table readTable(@NotNull final String tableIdentifier, final long tableSnapshotId) { - final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier); - // Find the snapshot with the given snapshot id - final Snapshot tableSnapshot = getTableSnapshot(tableId, tableSnapshotId); - - return readTableInternal(tableId, tableSnapshot, null); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.table(instructions); } /** @@ -714,16 +458,13 @@ public Table readTable(@NotNull final String tableIdentifier, final long tableSn * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + @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); - if (tableSnapshot == null) { - throw new IllegalArgumentException("Snapshot with id " + tableSnapshotId + " not found"); - } - return readTableInternal(tableIdentifier, tableSnapshot, instructions); + final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier); + return tableAdapter.table(tableSnapshotId, instructions); } /** @@ -735,11 +476,13 @@ public Table readTable( * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + @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); } /** @@ -751,76 +494,23 @@ public Table readTable( * @return The loaded table */ @SuppressWarnings("unused") - public Table readTable( + @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 Table 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 String description; - final TableLocationKeyFinder keyFinder; - final TableDataRefreshService refreshService; - final UpdateSourceRegistrar updateSourceRegistrar; - - if (partitionSpec.isUnpartitioned()) { - // Create the flat layout location key finder - keyFinder = new IcebergFlatLayout(tableDef, table, snapshot, table.io(), userInstructions, - dataInstructionsProvider); - } else { - // Create the partitioning column location key finder - keyFinder = new IcebergKeyValuePartitionedLayout(tableDef, table, snapshot, table.io(), partitionSpec, - userInstructions, dataInstructionsProvider); - } - - 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); - - return result; + /** + * 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)); } /** 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 b595b4cfd14..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 @@ -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; @@ -45,6 +46,15 @@ public static Builder builder() { */ public abstract Map columnRenames(); + /** + * The {@link IcebergUpdateMode} mode to use when reading the Iceberg data files. Default is + * {@link IcebergUpdateMode#staticMode()}. + */ + @Value.Default + public IcebergUpdateMode updateMode() { + return IcebergUpdateMode.staticMode(); + } + public interface Builder { @SuppressWarnings("unused") Builder tableDefinition(TableDefinition tableDefinition); @@ -58,6 +68,9 @@ public interface Builder { @SuppressWarnings("unused") Builder putAllColumnRenames(Map entries); + @SuppressWarnings("unused") + Builder updateMode(IcebergUpdateMode updateMode); + 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..c9d4b031354 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java @@ -0,0 +1,32 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.iceberg.util; + +import io.deephaven.engine.table.Table; +import org.apache.iceberg.Snapshot; +import org.jetbrains.annotations.NotNull; + +public interface IcebergTable extends Table { + /** + * Update the table with the latest snapshot from the catalog. + */ + 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 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. + * + * @param snapshot The snapshot to use when updating the table. + */ + 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..87de7b07cd7 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java @@ -0,0 +1,613 @@ +// +// 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.internal.DataInstructionsProviderLoader; +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; + private final DataInstructionsProviderLoader dataInstructionsProviderLoader; + + public IcebergTableAdapter( + final TableIdentifier tableIdentifier, + final org.apache.iceberg.Table table, + final DataInstructionsProviderLoader dataInstructionsProviderLoader) { + this.table = table; + this.tableIdentifier = tableIdentifier; + this.dataInstructionsProviderLoader = dataInstructionsProviderLoader; + } + + /** + * 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}. + * + * @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 + */ + 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. + * + * @param tableSnapshotId The snapshot id to load + * @param instructions The instructions for customizations while reading (or null for default instructions) + * @return The loaded table + */ + 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, dataInstructionsProviderLoader); + } else { + // Create the partitioning column location key finder + keyFinder = new IcebergKeyValuePartitionedLayout(this, snapshot, partitionSpec, userInstructions, + dataInstructionsProviderLoader); + } + + 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 new file mode 100644 index 00000000000..a73a55d40be --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java @@ -0,0 +1,60 @@ +// +// 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 implementation for static and refreshing Iceberg tables. + */ +public class IcebergTableImpl extends PartitionAwareSourceTable implements IcebergTable { + /** + * Location discovery. + */ + final IcebergTableLocationProviderBase locationProvider; + + /** + * Create an instance of the class with the provided parameters. + * + * @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 updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live + */ + IcebergTableImpl( + @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/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java index 5dd20f699a9..361159c9ffe 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 @@ -51,7 +51,7 @@ public static IcebergCatalogAdapter createAdapter( * Additional properties for the specific catalog should also be included, such as as S3-specific properties for * authentication or endpoint overriding. *

- * + * * @param name the name of the catalog; if omitted, the catalog URI will be used to generate a name * @param properties a map containing the Iceberg catalog properties to use * @return the Iceberg catalog adapter 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..43da976fce9 --- /dev/null +++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java @@ -0,0 +1,66 @@ +// +// 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 { + private static final long REFRESH_INTERVAL_MS = 60_000L; + + private static final IcebergUpdateMode STATIC = builder().updateType(IcebergUpdateType.STATIC).build(); + private static final IcebergUpdateMode AUTO_REFRESHING = + builder().updateType(IcebergUpdateType.AUTO_REFRESHING).build(); + private 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(); + } + + 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) + .build(); + } + + @Value.Default + public IcebergUpdateType updateType() { + return IcebergUpdateType.STATIC; + } + + @Value.Default + public long autoRefreshMs() { + return REFRESH_INTERVAL_MS; + } + + public interface Builder { + @SuppressWarnings("unused") + Builder updateType(IcebergUpdateType updateType); + + @SuppressWarnings("unused") + Builder autoRefreshMs(long autoRefreshMs); + + IcebergUpdateMode build(); + } +} 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"); + } +} 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; + } } diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py index 16fcd08f37d..6d10c85b765 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 @@ -14,7 +15,10 @@ from deephaven.jcompat import j_hashmap _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") _JIcebergTools = jpy.get_type("io.deephaven.iceberg.util.IcebergTools") # IcebergToolsS3 is an optional library @@ -28,6 +32,51 @@ _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_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). + """ + j_object_type = _JIcebergUpdateMode + + def __init__(self, mode: _JIcebergUpdateMode): + self._j_object = mode + + @classmethod + 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. + """ + 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 + + class IcebergInstructions(JObjectWrapper): """ This class specifies the instructions for reading an Iceberg table into Deephaven. These include column rename @@ -39,7 +88,8 @@ class IcebergInstructions(JObjectWrapper): def __init__(self, table_definition: Optional[TableDefinitionLike] = 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. @@ -51,6 +101,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. @@ -69,6 +121,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 @@ -78,6 +133,116 @@ def j_object(self) -> jpy.JType: return self._j_object +class IcebergTable(Table): + """ + 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 + + 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. + + Raises: + DHError: If unable to update the Iceberg table. + + """ + 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: + 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 @@ -102,10 +267,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. @@ -116,13 +281,32 @@ 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: """ - Returns information on the snapshots of the specified table as a Deephaven 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. @@ -131,10 +315,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) -> Table: + 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. @@ -155,8 +341,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: