From 470b09c178dadf15d4743430d398adcf3341e516 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Tue, 2 Jul 2024 08:28:07 -0700
Subject: [PATCH 01/56] Initial commit of refreshing Iceberg.
---
.../impl/locations/TableLocationProvider.java | 6 +-
.../impl/AbstractTableLocationProvider.java | 26 ++++-
.../impl/CompositeTableDataService.java | 2 +-
.../impl/FilteredTableDataService.java | 22 +++-
.../impl/PollingTableLocationProvider.java | 22 +++-
.../impl/TableLocationSubscriptionBuffer.java | 69 ++++++++---
.../impl/TestPartitionAwareSourceTable.java | 4 +-
.../iceberg/layout/IcebergBaseLayout.java | 10 +-
...cebergRefreshingTableLocationProvider.java | 107 ++++++++++++++++++
.../location/IcebergTableLocationKey.java | 3 +-
.../iceberg/util/IcebergCatalogAdapter.java | 83 ++++++++++----
.../iceberg/util/IcebergInstructions.java | 9 ++
.../deephaven/iceberg/util/IcebergTable.java | 74 ++++++++++++
13 files changed, 380 insertions(+), 57 deletions(-)
create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
index 84969d6249f..dd8b65826d4 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
@@ -26,6 +26,10 @@ public interface TableLocationProvider extends NamedImplementation {
*/
interface Listener extends BasicTableDataListener {
+ void beginTransaction();
+
+ void endTransaction();
+
/**
* Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location
* subscription. This should occur at most once per location, but the order of delivery is not
@@ -33,7 +37,7 @@ interface Listener extends BasicTableDataListener {
*
* @param tableLocationKey The new table location key
*/
- void handleTableLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey);
+ void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey);
/**
* Notify the listener of a {@link TableLocationKey} that has been removed.
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index b474775955a..3bd12e5253a 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -84,7 +84,9 @@ public final ImmutableTableKey getKey() {
@Override
protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) {
- unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKey);
+ listener.beginTransaction();
+ unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded);
+ listener.endTransaction();
}
/**
@@ -110,13 +112,26 @@ protected final void handleTableLocationKey(@NotNull final TableLocationKey loca
visitLocationKey(locationKey);
if (locationCreatedRecorder) {
verifyPartitionKeys(locationKey);
- if (subscriptions.deliverNotification(Listener::handleTableLocationKey, toKeyImmutable(result), true)) {
+ if (subscriptions.deliverNotification(Listener::handleTableLocationKeyAdded, toKeyImmutable(result),
+ true)) {
onEmpty();
}
}
}
}
+ protected final void beginTransaction() {
+ if (subscriptions != null) {
+ subscriptions.deliverNotification(Listener::beginTransaction, true);
+ }
+ }
+
+ protected final void endTransaction() {
+ if (subscriptions != null) {
+ subscriptions.deliverNotification(Listener::endTransaction, true);
+ }
+ }
+
/**
* Called after a table location has been visited by {@link #handleTableLocationKey(TableLocationKey)}, but
* before notifications have been delivered to any subscriptions, if applicable. The default implementation does
@@ -180,6 +195,13 @@ protected void doInitialization() {
@Override
@NotNull
public final Collection getTableLocationKeys() {
+ // We need to prevent reading the map (and maybe mutating it?) during a transaction.
+ // We could transition between two maps, a stable copy and a shadow copy that is being mutated.
+ // Or we could hold a bigger lock while mutating the map, and hold the same lock here. Sounds like a job for a
+ // read-write lock (e.g. ReentrantReadWriteLock), maybe. If you want `FunctionalLock`, the pattern (but mostly
+ // not the code) from io.deephaven.engine.updategraph.UpdateGraphLock could help.
+ // I think we need the read-write lock for correctness, and I think we need to make it explicit. That is, the
+ // user needs to be able to get a read lock and hold it while it's operating on the returned collection.
return unmodifiableTableLocationKeys;
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
index e9dbbc747ce..a9157da4b0f 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
@@ -115,7 +115,7 @@ public void subscribe(@NotNull final Listener listener) {
p.subscribe(listener);
} else {
p.refresh();
- p.getTableLocationKeys().forEach(listener::handleTableLocationKey);
+ p.getTableLocationKeys().forEach(listener::handleTableLocationKeyAdded);
}
});
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
index 8bfbeed184a..544852ecb21 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
@@ -157,12 +157,30 @@ private FilteringListener(@NotNull final TableLocationProvider.Listener outputLi
}
@Override
- public void handleTableLocationKey(@NotNull final ImmutableTableLocationKey tableLocationKey) {
+ public void beginTransaction() {
+ // Delegate to the wrapped listener.
+ final TableLocationProvider.Listener outputListener = getWrapped();
+ if (outputListener != null) {
+ outputListener.beginTransaction();
+ }
+ }
+
+ @Override
+ public void endTransaction() {
+ // Delegate to the wrapped listener.
+ final TableLocationProvider.Listener outputListener = getWrapped();
+ if (outputListener != null) {
+ outputListener.endTransaction();
+ }
+ }
+
+ @Override
+ public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) {
final TableLocationProvider.Listener outputListener = getWrapped();
// We can't try to clean up null listeners here, the underlying implementation may not allow concurrent
// unsubscribe operations.
if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) {
- outputListener.handleTableLocationKey(tableLocationKey);
+ outputListener.handleTableLocationKeyAdded(tableLocationKey);
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
index 361a18c8c44..0f9302b7a77 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
@@ -3,14 +3,14 @@
//
package io.deephaven.engine.table.impl.locations.impl;
-import io.deephaven.engine.table.impl.locations.TableKey;
-import io.deephaven.engine.table.impl.locations.TableLocation;
-import io.deephaven.engine.table.impl.locations.TableLocationKey;
-import io.deephaven.engine.table.impl.locations.TableLocationProvider;
+import io.deephaven.engine.table.impl.locations.*;
import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
+import java.util.HashSet;
+import java.util.Set;
+
/**
* Polling-driven {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key}
* discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a
@@ -46,9 +46,21 @@ public String getImplementationName() {
return IMPLEMENTATION_NAME;
}
+ // The simplest way to support "push" of new data availability is to provide a callback to the user that just calls
+ // `refresh`, which would need to become synchronized. Alternatively, we could make an Iceberg-specific aTLP
+ // implementation that exposes a more specific callback, e.g. with a snapshot ID, as well as the option to disable
+ // polling. We do need a mechanism to avoid going backwards, probably.
@Override
public void refresh() {
- locationKeyFinder.findKeys(this::handleTableLocationKey);
+ beginTransaction();
+ final Set missedKeys = new HashSet<>(getTableLocationKeys());
+ locationKeyFinder.findKeys(tableLocationKey -> {
+ // noinspection SuspiciousMethodCalls
+ missedKeys.remove(tableLocationKey);
+ handleTableLocationKey(tableLocationKey);
+ });
+ missedKeys.forEach(this::handleTableLocationKeyRemoved);
+ endTransaction();
setInitialized();
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
index fae32e8a0a1..05a32b5fb6d 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
@@ -26,9 +26,14 @@ public class TableLocationSubscriptionBuffer implements TableLocationProvider.Li
private boolean subscribed = false;
private final Object updateLock = new Object();
- private Set pendingLocationKeys = EMPTY_TABLE_LOCATION_KEYS;
+ // These sets represent a completed transaction of adds and removes.
+ private Set pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS;
private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS;
+
+ private Set accumulatedLocationsAdded = EMPTY_TABLE_LOCATION_KEYS;
+ private Set accumulatedLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS;
+
private TableDataException pendingException = null;
public TableLocationSubscriptionBuffer(@NotNull final TableLocationProvider tableLocationProvider) {
@@ -71,7 +76,7 @@ public synchronized LocationUpdate processPending() {
// NB: Providers that don't support subscriptions don't tick - this single call to run is
// sufficient.
tableLocationProvider.refresh();
- tableLocationProvider.getTableLocationKeys().forEach(this::handleTableLocationKey);
+ tableLocationProvider.getTableLocationKeys().forEach(this::handleTableLocationKeyAdded);
}
subscribed = true;
}
@@ -79,8 +84,8 @@ public synchronized LocationUpdate processPending() {
final Collection resultLocationsRemoved;
final TableDataException resultException;
synchronized (updateLock) {
- resultLocationKeys = pendingLocationKeys;
- pendingLocationKeys = EMPTY_TABLE_LOCATION_KEYS;
+ resultLocationKeys = pendingLocationsAdded;
+ pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS;
resultLocationsRemoved = pendingLocationsRemoved;
pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS;
resultException = pendingException;
@@ -105,7 +110,7 @@ public synchronized void reset() {
subscribed = false;
}
synchronized (updateLock) {
- pendingLocationKeys = EMPTY_TABLE_LOCATION_KEYS;
+ pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS;
pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS;
pendingException = null;
}
@@ -116,27 +121,65 @@ public synchronized void reset() {
// ------------------------------------------------------------------------------------------------------------------
@Override
- public void handleTableLocationKey(@NotNull final ImmutableTableLocationKey tableLocationKey) {
+ public void beginTransaction() {
+ synchronized (updateLock) {
+ // Assert that we can start a new transaction.
+ Require.eq(EMPTY_TABLE_LOCATION_KEYS, "accumulatedLocationsAdded", accumulatedLocationsAdded);
+ Require.eq(EMPTY_TABLE_LOCATION_KEYS, "accumulatedLocationsRemoved", accumulatedLocationsRemoved);
+ }
+ }
+
+ @Override
+ public void endTransaction() {
+ synchronized (updateLock) {
+ // If we already have a completed transaction, merge the accumulated sets into it.
+ if (pendingLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) {
+ pendingLocationsAdded.addAll(accumulatedLocationsAdded);
+ } else {
+ pendingLocationsAdded = accumulatedLocationsAdded;
+ }
+
+ if (pendingLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) {
+ // Remove any locations that were added and then removed by this transaction.
+ for (final ImmutableTableLocationKey tableLocationKey : accumulatedLocationsRemoved) {
+ if (pendingLocationsAdded.remove(tableLocationKey)) {
+ continue;
+ }
+ pendingLocationsRemoved.add(tableLocationKey);
+ }
+ } else {
+ pendingLocationsRemoved = accumulatedLocationsRemoved;
+ }
+
+ // Reset the accumulated sets to empty.
+ accumulatedLocationsAdded = EMPTY_TABLE_LOCATION_KEYS;
+ accumulatedLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS;
+ }
+ }
+
+ @Override
+ public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) {
synchronized (updateLock) {
- if (pendingLocationKeys == EMPTY_TABLE_LOCATION_KEYS) {
- pendingLocationKeys = new HashSet<>();
+ if (accumulatedLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) {
+ accumulatedLocationsAdded = new HashSet<>();
}
- pendingLocationKeys.add(tableLocationKey);
+ accumulatedLocationsAdded.add(tableLocationKey);
}
}
@Override
public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) {
synchronized (updateLock) {
+ // TODO: is it possible that we add and remove a location within a single transaction? Paranoid?
// If we remove something that was pending to be added, just discard both.
- if (pendingLocationKeys.remove(tableLocationKey)) {
+ if (accumulatedLocationsRemoved.remove(tableLocationKey)) {
return;
}
- if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
- pendingLocationsRemoved = new HashSet<>();
+ if (accumulatedLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
+ accumulatedLocationsRemoved = new HashSet<>();
}
- pendingLocationsRemoved.add(tableLocationKey);
+ accumulatedLocationsRemoved.add(tableLocationKey);
}
}
diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java
index f6b838c0646..774f25f7d16 100644
--- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java
+++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java
@@ -250,7 +250,7 @@ private void doInitializeCheck(final ImmutableTableLocationKey[] tableLocationKe
@Override
public Object invoke(Invocation invocation) {
subscriptionBuffer = (TableLocationSubscriptionBuffer) invocation.getParameter(0);
- Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKey);
+ Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded);
return null;
}
});
@@ -407,7 +407,7 @@ public Object invoke(Invocation invocation) {
private void doAddLocationsRefreshCheck(final ImmutableTableLocationKey[] tableLocationKeys,
final Set expectPassFilters) {
- Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKey);
+ Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded);
expectPassFilters.forEach(tl -> checking(new Expectations() {
{
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java
index f5334cf866c..70a65230f29 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java
@@ -33,11 +33,6 @@ public abstract class IcebergBaseLayout implements TableLocationKeyFinder cache;
+ /**
+ * The {@link Snapshot} from which to discover data files.
+ */
+ Snapshot snapshot;
+
/**
* The {@link ParquetInstructions} object that will be used to read any Parquet data files in this table. Only
* accessed while synchronized on {@code this}.
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
new file mode 100644
index 00000000000..5b510904c3d
--- /dev/null
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
@@ -0,0 +1,107 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.iceberg.layout;
+
+import io.deephaven.engine.table.impl.locations.*;
+import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider;
+import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
+import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
+import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
+import org.apache.iceberg.Snapshot;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Polling-driven {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key}
+ * discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a
+ * {@link TableLocationFactory}.
+ */
+public class IcebergRefreshingTableLocationProvider
+ extends AbstractTableLocationProvider {
+
+ private static final String IMPLEMENTATION_NAME = IcebergRefreshingTableLocationProvider.class.getSimpleName();
+
+ private final IcebergBaseLayout locationKeyFinder;
+ private final TableLocationFactory locationFactory;
+ private final TableDataRefreshService refreshService;
+
+ private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken;
+
+ public IcebergRefreshingTableLocationProvider(@NotNull final TK tableKey,
+ @NotNull final IcebergBaseLayout locationKeyFinder,
+ @NotNull final TableLocationFactory locationFactory,
+ @Nullable final TableDataRefreshService refreshService) {
+ super(tableKey, refreshService != null);
+ this.locationKeyFinder = locationKeyFinder;
+ this.locationFactory = locationFactory;
+ this.refreshService = refreshService;
+ }
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // AbstractTableLocationProvider implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public String getImplementationName() {
+ return IMPLEMENTATION_NAME;
+ }
+
+ // The simplest way to support "push" of new data availability is to provide a callback to the user that just calls
+ // `refresh`, which would need to become synchronized. Alternatively, we could make an Iceberg-specific aTLP
+ // implementation that exposes a more specific callback, e.g. with a snapshot ID, as well as the option to disable
+ // polling. We do need a mechanism to avoid going backwards, probably.
+ @Override
+ public void refresh() {
+ beginTransaction();
+ final Set missedKeys = new HashSet<>(getTableLocationKeys());
+ locationKeyFinder.findKeys(tableLocationKey -> {
+ // noinspection SuspiciousMethodCalls
+ missedKeys.remove(tableLocationKey);
+ handleTableLocationKey(tableLocationKey);
+ });
+ missedKeys.forEach(this::handleTableLocationKeyRemoved);
+ endTransaction();
+ setInitialized();
+ }
+
+
+ public void update(final Snapshot snapshot) {
+ // Update the snapshot to the new one
+ locationKeyFinder.snapshot = snapshot;
+ // Call the refresh
+ refresh();
+ }
+
+ @Override
+ @NotNull
+ protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) {
+ // noinspection unchecked
+ return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService);
+ }
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // SubscriptionAggregator implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ @Override
+ protected final void activateUnderlyingDataSource() {
+ subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this);
+ }
+
+ @Override
+ protected final void deactivateUnderlyingDataSource() {
+ if (subscriptionToken != null) {
+ subscriptionToken.cancel();
+ subscriptionToken = null;
+ }
+ }
+
+ @Override
+ protected final boolean matchSubscriptionToken(final T token) {
+ return token == subscriptionToken;
+ }
+}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java
index d6d5d4fb514..0fb37d9132d 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/location/IcebergTableLocationKey.java
@@ -3,12 +3,13 @@
//
package io.deephaven.iceberg.location;
+import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableLocationKey;
/**
* {@link TableLocationKey} implementation for use with data stored in Iceberg tables.
*/
-public interface IcebergTableLocationKey extends TableLocationKey {
+public interface IcebergTableLocationKey extends ImmutableTableLocationKey {
/**
* Get the read instructions for the location.
*
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index c379c715c6d..f4daec870d2 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -3,21 +3,25 @@
//
package io.deephaven.iceberg.util;
+import io.deephaven.engine.context.ExecutionContext;
import io.deephaven.engine.rowset.RowSetFactory;
import io.deephaven.engine.table.*;
import io.deephaven.engine.table.impl.PartitionAwareSourceTable;
import io.deephaven.engine.table.impl.QueryTable;
import io.deephaven.engine.table.impl.locations.TableDataException;
+import io.deephaven.engine.table.impl.locations.TableKey;
import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider;
+import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder;
import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider;
import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey;
-import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
import io.deephaven.engine.table.impl.sources.InMemoryColumnSource;
import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl;
import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
+import io.deephaven.iceberg.layout.IcebergBaseLayout;
import io.deephaven.iceberg.layout.IcebergFlatLayout;
import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout;
+import io.deephaven.iceberg.layout.IcebergRefreshingTableLocationProvider;
import io.deephaven.iceberg.location.IcebergTableLocationFactory;
import io.deephaven.iceberg.location.IcebergTableLocationKey;
import org.apache.iceberg.PartitionField;
@@ -55,7 +59,7 @@ public class IcebergCatalogAdapter {
/**
* Create a single {@link TableDefinition} from a given Schema, PartitionSpec, and TableDefinition. Takes into
- * account {@link Map column rename instructions}
+ * account {@link Map<> column rename instructions}
*
* @param schema The schema of the table.
* @param partitionSpec The partition specification of the table.
@@ -324,6 +328,13 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier
return new QueryTable(RowSetFactory.flat(size).toTracking(), columnSourceMap);
}
+ Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long snapshotId) {
+ return listSnapshots(tableIdentifier).stream()
+ .filter(snapshot -> snapshot.snapshotId() == snapshotId)
+ .findFirst()
+ .orElse(null);
+ }
+
/**
* Read the latest static snapshot of an Iceberg table from the Iceberg catalog.
*
@@ -353,10 +364,10 @@ public Table readTable(
@NotNull final IcebergInstructions instructions) {
// Find the snapshot with the given snapshot id
- final Snapshot tableSnapshot = listSnapshots(tableIdentifier).stream()
- .filter(snapshot -> snapshot.snapshotId() == tableSnapshotId)
- .findFirst()
- .orElseThrow(() -> new IllegalArgumentException("Snapshot with id " + tableSnapshotId + " not found"));
+ final Snapshot tableSnapshot = getSnapshot(tableIdentifier, tableSnapshotId);
+ if (tableSnapshot == null) {
+ throw new IllegalArgumentException("Snapshot with id " + tableSnapshotId + " not found");
+ }
return readTableInternal(tableIdentifier, tableSnapshot, instructions);
}
@@ -381,7 +392,6 @@ private Table readTableInternal(
@NotNull final TableIdentifier tableIdentifier,
@Nullable final Snapshot tableSnapshot,
@NotNull final IcebergInstructions instructions) {
-
// Load the table from the catalog
final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier);
@@ -427,7 +437,7 @@ private Table readTableInternal(
}
final String description;
- final TableLocationKeyFinder keyFinder;
+ final IcebergBaseLayout keyFinder;
final TableDataRefreshService refreshService;
final UpdateSourceRegistrar updateSourceRegistrar;
@@ -440,23 +450,46 @@ private Table readTableInternal(
instructions);
}
- refreshService = null;
- updateSourceRegistrar = null;
- description = "Read static iceberg table with " + keyFinder;
-
- final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>(
- StandaloneTableKey.getInstance(),
- keyFinder,
- new IcebergTableLocationFactory(),
- refreshService);
-
- final PartitionAwareSourceTable result = new PartitionAwareSourceTable(
- tableDef,
- description,
- RegionedTableComponentFactoryImpl.INSTANCE,
- locationProvider,
- updateSourceRegistrar);
+ if (instructions.isRefreshing()) {
+ description = "Read refreshing iceberg table with " + keyFinder;
+ updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
+
+ final IcebergRefreshingTableLocationProvider locationProvider =
+ new IcebergRefreshingTableLocationProvider<>(
+ StandaloneTableKey.getInstance(),
+ keyFinder,
+ new IcebergTableLocationFactory(),
+ TableDataRefreshService.getSharedRefreshService());
+
+ return new IcebergTable(
+ tableIdentifier,
+ this,
+ tableDef,
+ description,
+ RegionedTableComponentFactoryImpl.INSTANCE,
+ locationProvider,
+ updateSourceRegistrar);
+ } else {
+ description = "Read static iceberg table with " + keyFinder;
+ updateSourceRegistrar = null;
+
+ final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>(
+ StandaloneTableKey.getInstance(),
+ keyFinder,
+ new IcebergTableLocationFactory(),
+ null);
+
+ return new PartitionAwareSourceTable(
+ tableDef,
+ description,
+ RegionedTableComponentFactoryImpl.INSTANCE,
+ locationProvider,
+ updateSourceRegistrar);
+ }
+ }
- return result;
+ private static KnownLocationKeyFinder toKnownKeys(
+ final IcebergBaseLayout keyFinder) {
+ return KnownLocationKeyFinder.copyFrom(keyFinder, Comparator.naturalOrder());
}
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
index 4788e0e8714..57963a9e2d6 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
@@ -5,6 +5,7 @@
import io.deephaven.annotations.BuildableStyle;
import io.deephaven.engine.table.TableDefinition;
+import org.immutables.value.Value;
import org.immutables.value.Value.Immutable;
import java.util.Map;
@@ -38,6 +39,11 @@ public static Builder builder() {
*/
public abstract Map columnRenames();
+ @Value.Default
+ public Boolean isRefreshing() {
+ return false;
+ }
+
public interface Builder {
@SuppressWarnings("unused")
Builder tableDefinition(TableDefinition tableDefinition);
@@ -51,6 +57,9 @@ public interface Builder {
@SuppressWarnings("unused")
Builder putAllColumnRenames(Map entries);
+ @SuppressWarnings("unused")
+ Builder isRefreshing(Boolean isRefreshing);
+
IcebergInstructions build();
}
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
new file mode 100644
index 00000000000..cbc27b50461
--- /dev/null
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
@@ -0,0 +1,74 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.iceberg.util;
+
+import io.deephaven.engine.table.TableDefinition;
+import io.deephaven.engine.table.impl.PartitionAwareSourceTable;
+import io.deephaven.engine.table.impl.SourceTableComponentFactory;
+import io.deephaven.engine.table.impl.locations.TableKey;
+import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
+import io.deephaven.iceberg.layout.IcebergRefreshingTableLocationProvider;
+import io.deephaven.iceberg.location.IcebergTableLocationKey;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.List;
+
+/**
+ * Table wrapper for refreshing Iceberg tables.
+ */
+public class IcebergTable extends PartitionAwareSourceTable {
+ private final TableIdentifier tableIdentifier;
+ private final IcebergCatalogAdapter adapter;
+ /**
+ * Location discovery.
+ */
+ final IcebergRefreshingTableLocationProvider locationProvider;
+
+ /**
+ *
+ *
+ * @param tableDefinition A TableDefinition
+ * @param description A human-readable description for this table
+ * @param componentFactory A component factory for creating column source managers
+ * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table
+ * @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live
+ */
+ IcebergTable(
+ @NotNull TableIdentifier tableIdentifier,
+ @NotNull IcebergCatalogAdapter adapter,
+ @NotNull TableDefinition tableDefinition,
+ @NotNull String description,
+ @NotNull SourceTableComponentFactory componentFactory,
+ @NotNull IcebergRefreshingTableLocationProvider locationProvider,
+ @Nullable UpdateSourceRegistrar updateSourceRegistrar) {
+ super(tableDefinition, description, componentFactory, locationProvider, updateSourceRegistrar);
+ this.tableIdentifier = tableIdentifier;
+ this.adapter = adapter;
+ this.locationProvider = locationProvider;
+ }
+
+ public void update() {
+ // Find the latest snapshot.
+ final List snapshots = adapter.listSnapshots(tableIdentifier);
+ update(snapshots.get(snapshots.size() - 1));
+ }
+
+ public void update(final long snapshotId) {
+ // Find the snapshot with the given snapshot id
+ final Snapshot tableSnapshot = adapter.getSnapshot(tableIdentifier, snapshotId);
+ if (tableSnapshot == null) {
+ throw new IllegalArgumentException("Snapshot with id " + snapshotId + " not found");
+ }
+
+ update(tableSnapshot);
+ }
+
+ public void update(final @NotNull Snapshot snapshot) {
+ // Call the update function, this
+ locationProvider.update(snapshot);
+ }
+}
From 264fdb1eb5ff8ce4e1b4b667a1310efed7a1fffa Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Tue, 2 Jul 2024 09:02:59 -0700
Subject: [PATCH 02/56] Change IcebergInstructions refreshing indicator to enum
instead of boolean
---
.../iceberg/util/IcebergCatalogAdapter.java | 35 ++++++++++---------
.../iceberg/util/IcebergInstructions.java | 12 +++++--
2 files changed, 28 insertions(+), 19 deletions(-)
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index 97c467b04a4..69b5332caba 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -508,7 +508,23 @@ private Table readTableInternal(
userInstructions);
}
- if (instructions.isRefreshing()) {
+ if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.STATIC) {
+ description = "Read static iceberg table with " + keyFinder;
+ updateSourceRegistrar = null;
+
+ final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>(
+ StandaloneTableKey.getInstance(),
+ keyFinder,
+ new IcebergTableLocationFactory(),
+ null);
+
+ return new PartitionAwareSourceTable(
+ tableDef,
+ description,
+ RegionedTableComponentFactoryImpl.INSTANCE,
+ locationProvider,
+ updateSourceRegistrar);
+ } else if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) {
description = "Read refreshing iceberg table with " + keyFinder;
updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
@@ -528,22 +544,9 @@ private Table readTableInternal(
locationProvider,
updateSourceRegistrar);
} else {
- description = "Read static iceberg table with " + keyFinder;
- updateSourceRegistrar = null;
-
- final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>(
- StandaloneTableKey.getInstance(),
- keyFinder,
- new IcebergTableLocationFactory(),
- null);
-
- return new PartitionAwareSourceTable(
- tableDef,
- description,
- RegionedTableComponentFactoryImpl.INSTANCE,
- locationProvider,
- updateSourceRegistrar);
+ throw new UnsupportedOperationException("Unsupported refreshing mode: " + instructions.refreshing());
}
+
}
private static KnownLocationKeyFinder toKnownKeys(
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
index ebc0dff128e..e1063d06d52 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
@@ -25,6 +25,12 @@ public abstract class IcebergInstructions {
@SuppressWarnings("unused")
public static final IcebergInstructions DEFAULT = builder().build();
+ public enum IcebergRefreshing {
+ STATIC,
+ AUTO_REFRESHING,
+ MANUAL_REFRESHING
+ }
+
public static Builder builder() {
return ImmutableIcebergInstructions.builder();
}
@@ -47,8 +53,8 @@ public static Builder builder() {
public abstract Map columnRenames();
@Value.Default
- public Boolean isRefreshing() {
- return false;
+ public IcebergRefreshing refreshing() {
+ return IcebergRefreshing.STATIC;
}
public interface Builder {
@@ -65,7 +71,7 @@ public interface Builder {
Builder putAllColumnRenames(Map entries);
@SuppressWarnings("unused")
- Builder isRefreshing(Boolean isRefreshing);
+ Builder refreshing(IcebergRefreshing refreshing);
IcebergInstructions build();
}
From 58d0a730bdea0487cc10041e25a4ecfb5410c978 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Wed, 3 Jul 2024 07:59:33 -0700
Subject: [PATCH 03/56] WIP, for review
---
.../table/impl/ColumnSourceManager.java | 7 ++---
.../engine/table/impl/SourceTable.java | 16 +++++-------
.../regioned/RegionedColumnSourceManager.java | 26 +++++++++++++++----
.../iceberg/util/IcebergInstructions.java | 4 +--
4 files changed, 30 insertions(+), 23 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java
index 1d5ba841b7a..bc56a42d246 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/ColumnSourceManager.java
@@ -7,10 +7,7 @@
import io.deephaven.engine.rowset.RowSet;
import io.deephaven.engine.rowset.TrackingWritableRowSet;
import io.deephaven.engine.rowset.WritableRowSet;
-import io.deephaven.engine.table.ColumnSource;
-import io.deephaven.engine.table.DataIndex;
-import io.deephaven.engine.table.Table;
-import io.deephaven.engine.table.TableListener;
+import io.deephaven.engine.table.*;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableLocation;
import org.jetbrains.annotations.NotNull;
@@ -53,7 +50,7 @@ public interface ColumnSourceManager extends LivenessReferent {
*
* @return The set of added row keys, to be owned by the caller
*/
- WritableRowSet refresh();
+ TableUpdate refresh();
/**
* Advise this ColumnSourceManager that an error has occurred, and that it will no longer be {@link #refresh()
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java
index e3508fc45c7..5398bb80a86 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java
@@ -9,6 +9,7 @@
import io.deephaven.engine.rowset.TrackingWritableRowSet;
import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.TableDefinition;
+import io.deephaven.engine.table.TableUpdate;
import io.deephaven.engine.table.TableUpdateListener;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableDataException;
@@ -106,7 +107,6 @@ public abstract class SourceTable> exte
}
setRefreshing(isRefreshing);
- setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE);
}
/**
@@ -219,10 +219,6 @@ protected void instrumentedRefresh() {
final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending();
final ImmutableTableLocationKey[] removedKeys =
maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys());
- if (removedKeys.length > 0) {
- throw new TableLocationRemovedException("Source table does not support removed locations",
- removedKeys);
- }
maybeAddLocations(locationUpdate.getPendingAddedLocationKeys());
// This class previously had functionality to notify "location listeners", but it was never used.
@@ -232,13 +228,13 @@ protected void instrumentedRefresh() {
return;
}
- final RowSet added = columnSourceManager.refresh();
- if (added.isEmpty()) {
+ final TableUpdate update = columnSourceManager.refresh();
+ if (update.empty()) {
return;
}
-
- rowSet.insert(added);
- notifyListeners(added, RowSetFactory.empty(), RowSetFactory.empty());
+ rowSet.insert(update.added());
+ rowSet.remove(update.removed());
+ notifyListeners(update);
}
@Override
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
index d510f21a002..0ae2eaf1ba0 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
@@ -71,6 +71,8 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col
private final KeyedObjectHashMap includedTableLocations =
new KeyedObjectHashMap<>(INCLUDED_TABLE_LOCATION_ENTRY_KEY);
+ private final List removedTableLocations = new ArrayList<>();
+
/**
* Table locations that provide the regions backing our column sources, in insertion order.
*/
@@ -206,7 +208,8 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati
log.debug().append("EMPTY_LOCATION_REMOVED:").append(locationKey.toString()).endl();
}
} else if (includedLocation != null) {
- includedLocation.invalidate();
+ removedTableLocations.add(includedLocation);
+ // includedLocation.invalidate();
return true;
}
@@ -219,7 +222,8 @@ public synchronized TrackingWritableRowSet initialize() {
// Do our first pass over the locations to include as many as possible and build the initial row set
// noinspection resource
- final TrackingWritableRowSet initialRowSet = update(true).toTracking();
+ final TableUpdate update = update(true);
+ final TrackingWritableRowSet initialRowSet = update.added().writableCast().toTracking();
// Add single-column data indexes for all partitioning columns, whether refreshing or not
columnDefinitions.stream().filter(ColumnDefinition::isPartitioning).forEach(cd -> {
@@ -261,7 +265,7 @@ public synchronized TrackingWritableRowSet initialize() {
}
@Override
- public synchronized WritableRowSet refresh() {
+ public synchronized TableUpdate refresh() {
if (!isRefreshing) {
throw new UnsupportedOperationException("Cannot refresh a static table");
}
@@ -280,8 +284,19 @@ public void deliverError(@NotNull final Throwable error, @Nullable final TableLi
}
}
- private WritableRowSet update(final boolean initializing) {
+ private TableUpdate update(final boolean initializing) {
final RowSetBuilderSequential addedRowSetBuilder = RowSetFactory.builderSequential();
+ final RowSetBuilderSequential removedRowSetBuilder = RowSetFactory.builderSequential();
+
+ // Sort the removed locations by region index, so that we can process them in order.
+ removedTableLocations.sort(Comparator.comparingInt(e -> e.regionIndex));
+ for (final IncludedTableLocationEntry removedLocation : removedTableLocations) {
+ final long regionFirstKey = RegionedColumnSource.getFirstRowKey(removedLocation.regionIndex);
+ removedLocation.location.getRowSet()
+ .forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> removedRowSetBuilder
+ .appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey));
+ }
+ removedTableLocations.clear();
final RowSetBuilderSequential modifiedRegionBuilder = initializing ? null : RowSetFactory.builderSequential();
@@ -373,7 +388,8 @@ private WritableRowSet update(final boolean initializing) {
includedLocationsTable.notifyListeners(update);
}
}
- return addedRowSetBuilder.build();
+ return new TableUpdateImpl(addedRowSetBuilder.build(), removedRowSetBuilder.build(), RowSetFactory.empty(),
+ RowSetShiftData.EMPTY, ModifiedColumnSet.EMPTY);
}
@Override
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
index e1063d06d52..d6596b254de 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
@@ -26,9 +26,7 @@ public abstract class IcebergInstructions {
public static final IcebergInstructions DEFAULT = builder().build();
public enum IcebergRefreshing {
- STATIC,
- AUTO_REFRESHING,
- MANUAL_REFRESHING
+ STATIC, AUTO_REFRESHING, MANUAL_REFRESHING
}
public static Builder builder() {
From e0904742be80360735e5e3d1e66f9c4bbb34c07f Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Mon, 22 Jul 2024 19:47:36 -0700
Subject: [PATCH 04/56] Manual and auto-refreshing working, better
documentation.
---
.../regioned/RegionedColumnSourceManager.java | 36 +++++--
...cebergRefreshingTableLocationProvider.java | 99 ++++++++++++++++---
.../iceberg/util/IcebergCatalogAdapter.java | 79 ++++++++++-----
.../iceberg/util/IcebergInstructions.java | 17 ++++
.../deephaven/iceberg/util/IcebergTable.java | 24 +----
5 files changed, 190 insertions(+), 65 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
index 0ae2eaf1ba0..6d70dbf912f 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
@@ -24,6 +24,7 @@
import io.deephaven.io.logger.Logger;
import io.deephaven.util.SafeCloseable;
import io.deephaven.util.annotations.ReferentialIntegrity;
+import io.deephaven.util.mutable.MutableInt;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
@@ -71,8 +72,17 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col
private final KeyedObjectHashMap includedTableLocations =
new KeyedObjectHashMap<>(INCLUDED_TABLE_LOCATION_ENTRY_KEY);
+ /**
+ * List of locations that were removed this cycle. Will be cleared after each update.
+ */
private final List removedTableLocations = new ArrayList<>();
+ /**
+ * The next region index to assign to a location. We increment for each new location and will not reuse indices from
+ * regions that were removed.
+ */
+ private final MutableInt nextRegionIndex = new MutableInt(0);
+
/**
* Table locations that provide the regions backing our column sources, in insertion order.
*/
@@ -208,6 +218,7 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati
log.debug().append("EMPTY_LOCATION_REMOVED:").append(locationKey.toString()).endl();
}
} else if (includedLocation != null) {
+ orderedIncludedTableLocations.remove(includedLocation);
removedTableLocations.add(includedLocation);
// includedLocation.invalidate();
return true;
@@ -288,6 +299,8 @@ private TableUpdate update(final boolean initializing) {
final RowSetBuilderSequential addedRowSetBuilder = RowSetFactory.builderSequential();
final RowSetBuilderSequential removedRowSetBuilder = RowSetFactory.builderSequential();
+ final RowSetBuilderSequential removedRegionBuilder = RowSetFactory.builderSequential();
+
// Sort the removed locations by region index, so that we can process them in order.
removedTableLocations.sort(Comparator.comparingInt(e -> e.regionIndex));
for (final IncludedTableLocationEntry removedLocation : removedTableLocations) {
@@ -295,6 +308,7 @@ private TableUpdate update(final boolean initializing) {
removedLocation.location.getRowSet()
.forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> removedRowSetBuilder
.appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey));
+ removedRegionBuilder.appendKey(removedLocation.regionIndex);
}
removedTableLocations.clear();
@@ -333,6 +347,8 @@ private TableUpdate update(final boolean initializing) {
}
}
+ final RowSetBuilderSequential addedRegionBuilder = RowSetFactory.builderSequential();
+
final int previousNumRegions = includedTableLocations.size();
final int newNumRegions = previousNumRegions + (entriesToInclude == null ? 0 : entriesToInclude.size());
if (entriesToInclude != null) {
@@ -356,11 +372,13 @@ private TableUpdate update(final boolean initializing) {
// @formatter:on
locationSource.set(entry.regionIndex, entry.location);
rowSetSource.set(entry.regionIndex, entry.location.getRowSet());
+ addedRegionBuilder.appendKey(entry.regionIndex);
}
}
+ final RowSet addedRegions = addedRegionBuilder.build();
- if (previousNumRegions != newNumRegions) {
- includedLocationsTable.getRowSet().writableCast().insertRange(previousNumRegions, newNumRegions - 1);
+ if (addedRegions.isNonempty()) {
+ includedLocationsTable.getRowSet().writableCast().insert(addedRegions);
}
if (initializing) {
@@ -374,14 +392,16 @@ private TableUpdate update(final boolean initializing) {
}
} else {
final RowSet modifiedRegions = modifiedRegionBuilder.build();
- if (previousNumRegions == newNumRegions && modifiedRegions.isEmpty()) {
+ final RowSet removedRegions = removedRegionBuilder.build();
+ if (addedRegions.isEmpty() && modifiedRegions.isEmpty() && removedRegions.isEmpty()) {
+ addedRegions.close();
modifiedRegions.close();
+ removedRegions.close();
} else {
+ includedLocationsTable.getRowSet().writableCast().remove(removedRegions);
final TableUpdate update = new TableUpdateImpl(
- previousNumRegions == newNumRegions
- ? RowSetFactory.empty()
- : RowSetFactory.fromRange(previousNumRegions, newNumRegions - 1),
- RowSetFactory.empty(),
+ addedRegions,
+ removedRegions,
modifiedRegions,
RowSetShiftData.EMPTY,
modifiedRegions.isNonempty() ? rowSetModifiedColumnSet : ModifiedColumnSet.EMPTY);
@@ -488,7 +508,7 @@ private class IncludedTableLocationEntry implements Comparable> columnLocationStates = new ArrayList<>();
/**
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
index 5b510904c3d..43aa3d78939 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
@@ -8,17 +8,25 @@
import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
+import io.deephaven.iceberg.util.IcebergCatalogAdapter;
import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.catalog.TableIdentifier;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
import java.util.HashSet;
+import java.util.List;
import java.util.Set;
/**
- * Polling-driven {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key}
+ *
+ * Refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key}
* discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a
* {@link TableLocationFactory}.
+ *
+ *
+ * Supports both automatic and manual refreshing cases, distinguished by the {@code autoRefresh} parameter.
+ *
*/
public class IcebergRefreshingTableLocationProvider
extends AbstractTableLocationProvider {
@@ -28,17 +36,27 @@ public class IcebergRefreshingTableLocationProvider locationFactory;
private final TableDataRefreshService refreshService;
+ private final IcebergCatalogAdapter adapter;
+ private final TableIdentifier tableIdentifier;
+ private final boolean autoRefresh;
private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken;
- public IcebergRefreshingTableLocationProvider(@NotNull final TK tableKey,
+ public IcebergRefreshingTableLocationProvider(
+ @NotNull final TK tableKey,
@NotNull final IcebergBaseLayout locationKeyFinder,
@NotNull final TableLocationFactory locationFactory,
- @Nullable final TableDataRefreshService refreshService) {
+ @Nullable final TableDataRefreshService refreshService,
+ @NotNull final IcebergCatalogAdapter adapter,
+ @NotNull final TableIdentifier tableIdentifier,
+ final boolean autoRefresh) {
super(tableKey, refreshService != null);
this.locationKeyFinder = locationKeyFinder;
this.locationFactory = locationFactory;
this.refreshService = refreshService;
+ this.adapter = adapter;
+ this.tableIdentifier = tableIdentifier;
+ this.autoRefresh = autoRefresh;
}
// ------------------------------------------------------------------------------------------------------------------
@@ -50,12 +68,69 @@ public String getImplementationName() {
return IMPLEMENTATION_NAME;
}
- // The simplest way to support "push" of new data availability is to provide a callback to the user that just calls
- // `refresh`, which would need to become synchronized. Alternatively, we could make an Iceberg-specific aTLP
- // implementation that exposes a more specific callback, e.g. with a snapshot ID, as well as the option to disable
- // polling. We do need a mechanism to avoid going backwards, probably.
@Override
- public void refresh() {
+ public synchronized void refresh() {
+ if (autoRefresh) {
+ final Snapshot latestSnapshot = adapter.getCurrentSnapshot(tableIdentifier);
+ if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) {
+ locationKeyFinder.snapshot = latestSnapshot;
+ }
+ }
+ refreshSnapshot();
+ }
+
+ /**
+ * Update the table location provider with the latest snapshot from the catalog.
+ */
+ public synchronized void update() {
+ update(adapter.getCurrentSnapshot(tableIdentifier));
+ }
+
+ /**
+ * Update the table location provider with a specific snapshot from the catalog. If the {@code snapshotId} is not
+ * found in the list of snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot
+ * must also be newer (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException}
+ * is thrown.
+ */
+ public synchronized void update(final long snapshotId) {
+ final List snapshots = adapter.listSnapshots(tableIdentifier);
+
+ final Snapshot snapshot = snapshots.stream()
+ .filter(s -> s.snapshotId() == snapshotId).findFirst()
+ .orElse(null);
+
+ if (snapshot == null) {
+ throw new IllegalArgumentException(
+ "Snapshot " + snapshotId + " was not found in the list of snapshots for table " + tableIdentifier
+ + ". Snapshots: " + snapshots);
+ }
+ update(snapshot);
+ }
+
+ /**
+ * Update the table location provider with a specific snapshot from the catalog. The input snapshot must be newer
+ * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown.
+ *
+ * @param snapshot
+ */
+ public synchronized void update(final Snapshot snapshot) {
+ // Verify that the input snapshot is newer (higher in sequence number) than the current snapshot.
+ if (snapshot.sequenceNumber() <= locationKeyFinder.snapshot.sequenceNumber()) {
+ throw new IllegalArgumentException(
+ "Snapshot sequence number " + snapshot.sequenceNumber()
+ + " is older than the current snapshot sequence number "
+ + locationKeyFinder.snapshot.sequenceNumber() + " for table " + tableIdentifier);
+ }
+ // Update the snapshot.
+ locationKeyFinder.snapshot = snapshot;
+ refreshSnapshot();
+ }
+
+ /**
+ * Refresh the table location provider with the latest snapshot from the catalog. This method will identify new
+ * locations and removed locations.
+ */
+ private void refreshSnapshot() {
beginTransaction();
final Set missedKeys = new HashSet<>(getTableLocationKeys());
locationKeyFinder.findKeys(tableLocationKey -> {
@@ -68,14 +143,6 @@ public void refresh() {
setInitialized();
}
-
- public void update(final Snapshot snapshot) {
- // Update the snapshot to the new one
- locationKeyFinder.snapshot = snapshot;
- // Call the refresh
- refresh();
- }
-
@Override
@NotNull
protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) {
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index 69b5332caba..76d099e9499 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -14,6 +14,7 @@
import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder;
import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider;
import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey;
+import io.deephaven.engine.table.impl.locations.util.ExecutorTableDataRefreshService;
import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
import io.deephaven.engine.table.impl.sources.InMemoryColumnSource;
import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl;
@@ -360,6 +361,17 @@ Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long
.orElse(null);
}
+ /**
+ * Get the current {@link Snapshot snapshot} of a given Iceberg table.
+ *
+ * @param tableIdentifier The identifier of the table.
+ * @return The current snapshot of the table.
+ */
+ public Snapshot getCurrentSnapshot(@NotNull final TableIdentifier tableIdentifier) {
+ final List snapshots = listSnapshots(tableIdentifier);
+ return snapshots.get(snapshots.size() - 1);
+ }
+
/**
* Read the latest static snapshot of an Iceberg table from the Iceberg catalog.
*
@@ -496,8 +508,6 @@ private Table readTableInternal(
final String description;
final IcebergBaseLayout keyFinder;
- final TableDataRefreshService refreshService;
- final UpdateSourceRegistrar updateSourceRegistrar;
if (partitionSpec.isUnpartitioned()) {
// Create the flat layout location key finder
@@ -510,11 +520,10 @@ private Table readTableInternal(
if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.STATIC) {
description = "Read static iceberg table with " + keyFinder;
- updateSourceRegistrar = null;
final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>(
StandaloneTableKey.getInstance(),
- keyFinder,
+ toKnownKeys(keyFinder),
new IcebergTableLocationFactory(),
null);
@@ -523,30 +532,47 @@ private Table readTableInternal(
description,
RegionedTableComponentFactoryImpl.INSTANCE,
locationProvider,
- updateSourceRegistrar);
- } else if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) {
- description = "Read refreshing iceberg table with " + keyFinder;
- updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
+ null);
+ }
- final IcebergRefreshingTableLocationProvider locationProvider =
- new IcebergRefreshingTableLocationProvider<>(
- StandaloneTableKey.getInstance(),
- keyFinder,
- new IcebergTableLocationFactory(),
- TableDataRefreshService.getSharedRefreshService());
+ final UpdateSourceRegistrar updateSourceRegistrar;
+ final IcebergRefreshingTableLocationProvider locationProvider;
- return new IcebergTable(
- tableIdentifier,
+ if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) {
+ description = "Read manual refreshing iceberg table with " + keyFinder;
+ updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
+
+ locationProvider = new IcebergRefreshingTableLocationProvider<>(
+ StandaloneTableKey.getInstance(),
+ keyFinder,
+ new IcebergTableLocationFactory(),
+ TableDataRefreshService.getSharedRefreshService(),
this,
- tableDef,
- description,
- RegionedTableComponentFactoryImpl.INSTANCE,
- locationProvider,
- updateSourceRegistrar);
+ tableIdentifier,
+ false);
} else {
- throw new UnsupportedOperationException("Unsupported refreshing mode: " + instructions.refreshing());
+ description = "Read automatic refreshing iceberg table with " + keyFinder;
+ updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
+
+ final TableDataRefreshService refreshService =
+ new ExecutorTableDataRefreshService("Local", instructions.autoRefreshMs(), 30_000L, 10);
+
+ locationProvider = new IcebergRefreshingTableLocationProvider<>(
+ StandaloneTableKey.getInstance(),
+ keyFinder,
+ new IcebergTableLocationFactory(),
+ refreshService,
+ this,
+ tableIdentifier,
+ true);
}
+ return new IcebergTable(
+ tableDef,
+ description,
+ RegionedTableComponentFactoryImpl.INSTANCE,
+ locationProvider,
+ updateSourceRegistrar);
}
private static KnownLocationKeyFinder toKnownKeys(
@@ -561,4 +587,13 @@ private static KnownLocationKeyFinder toKnownKeys(
public Catalog catalog() {
return catalog;
}
+
+ /**
+ * Returns the underlying Iceberg {@link FileIO fileIO} used by this adapter.
+ */
+ @SuppressWarnings("unused")
+ public FileIO fileIO() {
+ return fileIO;
+ }
+
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
index d6596b254de..dc7515d7498 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
@@ -25,6 +25,7 @@ public abstract class IcebergInstructions {
@SuppressWarnings("unused")
public static final IcebergInstructions DEFAULT = builder().build();
+ @SuppressWarnings("unused")
public enum IcebergRefreshing {
STATIC, AUTO_REFRESHING, MANUAL_REFRESHING
}
@@ -50,11 +51,24 @@ public static Builder builder() {
*/
public abstract Map columnRenames();
+ /**
+ * The {@link IcebergRefreshing} mode to use when reading the Iceberg data files. Default is
+ * {@link IcebergRefreshing#STATIC}.
+ */
@Value.Default
public IcebergRefreshing refreshing() {
return IcebergRefreshing.STATIC;
}
+ /**
+ * When {@link #refreshing()} is set to {@code IcebergRefreshing.AUTO_REFRESHING}, specifies the number of
+ * milliseconds to wait before refreshing the Iceberg data files. Default is 60_000 milliseconds.
+ */
+ @Value.Default
+ public long autoRefreshMs() {
+ return 60_000L; // 60 second default
+ }
+
public interface Builder {
@SuppressWarnings("unused")
Builder tableDefinition(TableDefinition tableDefinition);
@@ -71,6 +85,9 @@ public interface Builder {
@SuppressWarnings("unused")
Builder refreshing(IcebergRefreshing refreshing);
+ @SuppressWarnings("unused")
+ Builder autoRefreshMs(long autoRefreshMs);
+
IcebergInstructions build();
}
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
index cbc27b50461..9c422b96968 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
@@ -15,14 +15,10 @@
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
-import java.util.List;
-
/**
* Table wrapper for refreshing Iceberg tables.
*/
public class IcebergTable extends PartitionAwareSourceTable {
- private final TableIdentifier tableIdentifier;
- private final IcebergCatalogAdapter adapter;
/**
* Location discovery.
*/
@@ -38,37 +34,27 @@ public class IcebergTable extends PartitionAwareSourceTable {
* @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live
*/
IcebergTable(
- @NotNull TableIdentifier tableIdentifier,
- @NotNull IcebergCatalogAdapter adapter,
@NotNull TableDefinition tableDefinition,
@NotNull String description,
@NotNull SourceTableComponentFactory componentFactory,
@NotNull IcebergRefreshingTableLocationProvider locationProvider,
@Nullable UpdateSourceRegistrar updateSourceRegistrar) {
super(tableDefinition, description, componentFactory, locationProvider, updateSourceRegistrar);
- this.tableIdentifier = tableIdentifier;
- this.adapter = adapter;
this.locationProvider = locationProvider;
}
+ @SuppressWarnings("unused")
public void update() {
- // Find the latest snapshot.
- final List snapshots = adapter.listSnapshots(tableIdentifier);
- update(snapshots.get(snapshots.size() - 1));
+ locationProvider.update();
}
+ @SuppressWarnings("unused")
public void update(final long snapshotId) {
- // Find the snapshot with the given snapshot id
- final Snapshot tableSnapshot = adapter.getSnapshot(tableIdentifier, snapshotId);
- if (tableSnapshot == null) {
- throw new IllegalArgumentException("Snapshot with id " + snapshotId + " not found");
- }
-
- update(tableSnapshot);
+ locationProvider.update(snapshotId);
}
+ @SuppressWarnings("unused")
public void update(final @NotNull Snapshot snapshot) {
- // Call the update function, this
locationProvider.update(snapshot);
}
}
From 57021ad244f4e9b1a84b239289c9953ef93f0eb6 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Tue, 23 Jul 2024 16:57:03 -0700
Subject: [PATCH 05/56] Addressed more PR comments, some remaining.
---
.../engine/table/impl/SourceTable.java | 9 +-
.../impl/locations/TableLocationProvider.java | 42 ++++---
.../impl/AbstractTableLocationProvider.java | 23 ++--
.../impl/CompositeTableDataService.java | 2 +-
.../impl/FilteredTableDataService.java | 20 ++--
.../impl/TableLocationSubscriptionBuffer.java | 109 +++++++++++-------
.../regioned/RegionedColumnSourceManager.java | 19 ++-
...cebergRefreshingTableLocationProvider.java | 63 +---------
.../IcebergStaticTableLocationProvider.java | 65 +++++++++++
.../IcebergTableLocationProviderBase.java | 97 ++++++++++++++++
.../iceberg/util/IcebergCatalogAdapter.java | 70 ++++++-----
.../deephaven/iceberg/util/IcebergTable.java | 59 +++-------
.../iceberg/util/IcebergTableRefreshing.java | 59 ++++++++++
.../iceberg/util/IcebergTableStatic.java | 49 ++++++++
14 files changed, 467 insertions(+), 219 deletions(-)
create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java
create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java
index 5398bb80a86..a9a6dce38cf 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java
@@ -7,19 +7,15 @@
import io.deephaven.base.verify.Require;
import io.deephaven.engine.liveness.LivenessScopeStack;
import io.deephaven.engine.rowset.TrackingWritableRowSet;
-import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.TableDefinition;
import io.deephaven.engine.table.TableUpdate;
import io.deephaven.engine.table.TableUpdateListener;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableDataException;
import io.deephaven.engine.table.impl.locations.TableLocationProvider;
-import io.deephaven.engine.table.impl.locations.TableLocationRemovedException;
import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder;
import io.deephaven.engine.table.impl.locations.impl.TableLocationSubscriptionBuffer;
-import io.deephaven.engine.rowset.RowSet;
-import io.deephaven.engine.rowset.RowSetFactory;
import io.deephaven.util.SafeCloseable;
import io.deephaven.util.annotations.TestUseOnly;
import org.apache.commons.lang3.mutable.Mutable;
@@ -230,10 +226,13 @@ protected void instrumentedRefresh() {
final TableUpdate update = columnSourceManager.refresh();
if (update.empty()) {
+ update.release();
return;
}
- rowSet.insert(update.added());
+
+ Assert.eqTrue(update.shifted().empty(), "update.shifted().empty()");
rowSet.remove(update.removed());
+ rowSet.insert(update.added());
notifyListeners(update);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
index dd8b65826d4..1390bb3b1f4 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
@@ -26,25 +26,41 @@ public interface TableLocationProvider extends NamedImplementation {
*/
interface Listener extends BasicTableDataListener {
- void beginTransaction();
+ /**
+ * Begin a transaction that collects location key additions and removals to be processed atomically.
+ *
+ * @param token A token to identify the transaction.
+ */
+ void beginTransaction(final Object token);
- void endTransaction();
+ /**
+ * End the transaction and process the location changes.
+ *
+ * @param token A token to identify the transaction.
+ */
+ void endTransaction(final Object token);
/**
* Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location
* subscription. This should occur at most once per location, but the order of delivery is not
* guaranteed.
*
- * @param tableLocationKey The new table location key
+ * @param tableLocationKey The new table location key.
+ * @param transactionToken The token identifying the transaction.
*/
- void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey);
+ void handleTableLocationKeyAdded(
+ @NotNull final ImmutableTableLocationKey tableLocationKey,
+ final Object transactionToken);
/**
* Notify the listener of a {@link TableLocationKey} that has been removed.
*
- * @param tableLocationKey The table location key that was removed
+ * @param tableLocationKey The table location key that was removed.
+ * @param transactionToken The token identifying the transaction.
*/
- void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey);
+ void handleTableLocationKeyRemoved(
+ @NotNull final ImmutableTableLocationKey tableLocationKey,
+ final Object transactionToken);
}
/**
@@ -90,7 +106,7 @@ interface Listener extends BasicTableDataListener {
* that {@link #refresh()} or {@link #subscribe(Listener)} has been called prior to calls to the various table
* location fetch methods.
*
- * @return this, to allow method chaining
+ * @return this, to allow method chaining.
*/
TableLocationProvider ensureInitialized();
@@ -99,7 +115,7 @@ interface Listener extends BasicTableDataListener {
* size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is
* guaranteed to succeed for all results.
*
- * @return A collection of keys for locations available from this provider
+ * @return A collection of keys for locations available from this provider.
*/
@NotNull
Collection getTableLocationKeys();
@@ -107,13 +123,13 @@ interface Listener extends BasicTableDataListener {
/**
* Check if this provider knows the supplied location key.
*
- * @param tableLocationKey The key to test for
- * @return Whether the key is known to this provider
+ * @param tableLocationKey The key to test.
+ * @return Whether the key is known to this provider.
*/
boolean hasTableLocationKey(@NotNull final TableLocationKey tableLocationKey);
/**
- * @param tableLocationKey A {@link TableLocationKey} specifying the location to get
+ * @param tableLocationKey A {@link TableLocationKey} specifying the location to get.
* @return The {@link TableLocation} matching the given key
*/
@NotNull
@@ -126,8 +142,8 @@ default TableLocation getTableLocation(@NotNull TableLocationKey tableLocationKe
}
/**
- * @param tableLocationKey A {@link TableLocationKey} specifying the location to get
- * @return The {@link TableLocation} matching the given key if present, else null
+ * @param tableLocationKey A {@link TableLocationKey} specifying the location to get.
+ * @return The {@link TableLocation} matching the given key if present, else null.
*/
@Nullable
TableLocation getTableLocationIfPresent(@NotNull TableLocationKey tableLocationKey);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index 3bd12e5253a..dd3eab3ca54 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -84,9 +84,9 @@ public final ImmutableTableKey getKey() {
@Override
protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) {
- listener.beginTransaction();
- unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded);
- listener.endTransaction();
+ listener.beginTransaction(this);
+ unmodifiableTableLocationKeys.forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, this));
+ listener.endTransaction(this);
}
/**
@@ -112,8 +112,10 @@ protected final void handleTableLocationKey(@NotNull final TableLocationKey loca
visitLocationKey(locationKey);
if (locationCreatedRecorder) {
verifyPartitionKeys(locationKey);
- if (subscriptions.deliverNotification(Listener::handleTableLocationKeyAdded, toKeyImmutable(result),
- true)) {
+ if (subscriptions.deliverNotification(
+ (listener, tlk) -> listener.handleTableLocationKeyAdded(tlk,
+ AbstractTableLocationProvider.this),
+ toKeyImmutable(result), true)) {
onEmpty();
}
}
@@ -122,13 +124,13 @@ protected final void handleTableLocationKey(@NotNull final TableLocationKey loca
protected final void beginTransaction() {
if (subscriptions != null) {
- subscriptions.deliverNotification(Listener::beginTransaction, true);
+ subscriptions.deliverNotification(listener -> listener.beginTransaction(this), true);
}
}
protected final void endTransaction() {
if (subscriptions != null) {
- subscriptions.deliverNotification(Listener::endTransaction, true);
+ subscriptions.deliverNotification(listener -> listener.endTransaction(this), true);
}
}
@@ -237,8 +239,8 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t
/**
* Remove a {@link TableLocationKey} and its corresponding {@link TableLocation} (if it was created). All
* subscribers to this TableLocationProvider will be
- * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(ImmutableTableLocationKey) notified}. If the
- * TableLocation was created, all of its subscribers will additionally be
+ * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(ImmutableTableLocationKey, Object)}
+ * notified}. If the TableLocation was created, all of its subscribers will additionally be
* {@link TableLocation.Listener#handleUpdate() notified} that it no longer exists. This TableLocationProvider will
* continue to update other locations and will no longer provide or request information about the removed location.
*
@@ -279,7 +281,8 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey)
protected void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey locationKey) {
if (supportsSubscriptions()) {
synchronized (subscriptions) {
- if (subscriptions.deliverNotification(Listener::handleTableLocationKeyRemoved, locationKey, true)) {
+ if (subscriptions.deliverNotification(
+ (listener, tlk) -> listener.handleTableLocationKeyRemoved(tlk, this), locationKey, true)) {
onEmpty();
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
index a9157da4b0f..a9e782496ca 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
@@ -115,7 +115,7 @@ public void subscribe(@NotNull final Listener listener) {
p.subscribe(listener);
} else {
p.refresh();
- p.getTableLocationKeys().forEach(listener::handleTableLocationKeyAdded);
+ p.getTableLocationKeys().forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, this));
}
});
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
index 544852ecb21..7c5c090f780 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
@@ -157,38 +157,42 @@ private FilteringListener(@NotNull final TableLocationProvider.Listener outputLi
}
@Override
- public void beginTransaction() {
+ public void beginTransaction(final Object token) {
// Delegate to the wrapped listener.
final TableLocationProvider.Listener outputListener = getWrapped();
if (outputListener != null) {
- outputListener.beginTransaction();
+ outputListener.beginTransaction(token);
}
}
@Override
- public void endTransaction() {
+ public void endTransaction(final Object token) {
// Delegate to the wrapped listener.
final TableLocationProvider.Listener outputListener = getWrapped();
if (outputListener != null) {
- outputListener.endTransaction();
+ outputListener.endTransaction(token);
}
}
@Override
- public void handleTableLocationKeyAdded(@NotNull final ImmutableTableLocationKey tableLocationKey) {
+ public void handleTableLocationKeyAdded(
+ @NotNull final ImmutableTableLocationKey tableLocationKey,
+ final Object transactionToken) {
final TableLocationProvider.Listener outputListener = getWrapped();
// We can't try to clean up null listeners here, the underlying implementation may not allow concurrent
// unsubscribe operations.
if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) {
- outputListener.handleTableLocationKeyAdded(tableLocationKey);
+ outputListener.handleTableLocationKeyAdded(tableLocationKey, transactionToken);
}
}
@Override
- public void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey tableLocationKey) {
+ public void handleTableLocationKeyRemoved(
+ @NotNull final ImmutableTableLocationKey tableLocationKey,
+ final Object transactionToken) {
final TableLocationProvider.Listener outputListener = getWrapped();
if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) {
- outputListener.handleTableLocationKeyRemoved(tableLocationKey);
+ outputListener.handleTableLocationKeyRemoved(tableLocationKey, transactionToken);
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
index 05a32b5fb6d..c9998af0ee6 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
@@ -9,10 +9,7 @@
import io.deephaven.engine.table.impl.locations.TableLocationProvider;
import org.jetbrains.annotations.NotNull;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
+import java.util.*;
/**
* Intermediates between push-based subscription to a TableLocationProvider and polling on update source refresh.
@@ -27,12 +24,14 @@ public class TableLocationSubscriptionBuffer implements TableLocationProvider.Li
private final Object updateLock = new Object();
- // These sets represent a completed transaction of adds and removes.
+ // These sets represent adds and removes from completed transactions.
private Set pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS;
private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS;
- private Set accumulatedLocationsAdded = EMPTY_TABLE_LOCATION_KEYS;
- private Set accumulatedLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS;
+ // These sets represent open transactions that are being accumulated.
+ private final Set
*/
public class IcebergRefreshingTableLocationProvider
- extends AbstractTableLocationProvider {
+ extends IcebergTableLocationProviderBase {
private static final String IMPLEMENTATION_NAME = IcebergRefreshingTableLocationProvider.class.getSimpleName();
- private final IcebergBaseLayout locationKeyFinder;
- private final TableLocationFactory locationFactory;
- private final TableDataRefreshService refreshService;
- private final IcebergCatalogAdapter adapter;
- private final TableIdentifier tableIdentifier;
private final boolean autoRefresh;
private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken;
@@ -50,12 +44,7 @@ public IcebergRefreshingTableLocationProvider(
@NotNull final IcebergCatalogAdapter adapter,
@NotNull final TableIdentifier tableIdentifier,
final boolean autoRefresh) {
- super(tableKey, refreshService != null);
- this.locationKeyFinder = locationKeyFinder;
- this.locationFactory = locationFactory;
- this.refreshService = refreshService;
- this.adapter = adapter;
- this.tableIdentifier = tableIdentifier;
+ super(tableKey, locationKeyFinder, locationFactory, refreshService, adapter, tableIdentifier);
this.autoRefresh = autoRefresh;
}
@@ -79,19 +68,12 @@ public synchronized void refresh() {
refreshSnapshot();
}
- /**
- * Update the table location provider with the latest snapshot from the catalog.
- */
+ @Override
public synchronized void update() {
update(adapter.getCurrentSnapshot(tableIdentifier));
}
- /**
- * Update the table location provider with a specific snapshot from the catalog. If the {@code snapshotId} is not
- * found in the list of snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot
- * must also be newer (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException}
- * is thrown.
- */
+ @Override
public synchronized void update(final long snapshotId) {
final List snapshots = adapter.listSnapshots(tableIdentifier);
@@ -107,12 +89,7 @@ public synchronized void update(final long snapshotId) {
update(snapshot);
}
- /**
- * Update the table location provider with a specific snapshot from the catalog. The input snapshot must be newer
- * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown.
- *
- * @param snapshot
- */
+ @Override
public synchronized void update(final Snapshot snapshot) {
// Verify that the input snapshot is newer (higher in sequence number) than the current snapshot.
if (snapshot.sequenceNumber() <= locationKeyFinder.snapshot.sequenceNumber()) {
@@ -134,7 +111,6 @@ private void refreshSnapshot() {
beginTransaction();
final Set missedKeys = new HashSet<>(getTableLocationKeys());
locationKeyFinder.findKeys(tableLocationKey -> {
- // noinspection SuspiciousMethodCalls
missedKeys.remove(tableLocationKey);
handleTableLocationKey(tableLocationKey);
});
@@ -142,33 +118,4 @@ private void refreshSnapshot() {
endTransaction();
setInitialized();
}
-
- @Override
- @NotNull
- protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) {
- // noinspection unchecked
- return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService);
- }
-
- // ------------------------------------------------------------------------------------------------------------------
- // SubscriptionAggregator implementation
- // ------------------------------------------------------------------------------------------------------------------
-
- @Override
- protected final void activateUnderlyingDataSource() {
- subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this);
- }
-
- @Override
- protected final void deactivateUnderlyingDataSource() {
- if (subscriptionToken != null) {
- subscriptionToken.cancel();
- subscriptionToken = null;
- }
- }
-
- @Override
- protected final boolean matchSubscriptionToken(final T token) {
- return token == subscriptionToken;
- }
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
new file mode 100644
index 00000000000..4a93d985af0
--- /dev/null
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
@@ -0,0 +1,65 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.iceberg.layout;
+
+import io.deephaven.engine.table.impl.locations.*;
+import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
+import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
+import io.deephaven.iceberg.util.IcebergCatalogAdapter;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ *
+ * Static {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key} discovery to
+ * a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a {@link TableLocationFactory}.
+ *
+ */
+public class IcebergStaticTableLocationProvider
+ extends IcebergTableLocationProviderBase {
+
+ private static final String IMPLEMENTATION_NAME = IcebergStaticTableLocationProvider.class.getSimpleName();
+
+ public IcebergStaticTableLocationProvider(
+ @NotNull final TK tableKey,
+ @NotNull final IcebergBaseLayout locationKeyFinder,
+ @NotNull final TableLocationFactory locationFactory,
+ @NotNull final IcebergCatalogAdapter adapter,
+ @NotNull final TableIdentifier tableIdentifier) {
+ super(tableKey, locationKeyFinder, locationFactory, null, adapter, tableIdentifier);
+ }
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // AbstractTableLocationProvider implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public String getImplementationName() {
+ return IMPLEMENTATION_NAME;
+ }
+
+ @Override
+ public void refresh() {
+ beginTransaction();
+ locationKeyFinder.findKeys(this::handleTableLocationKey);
+ endTransaction();
+ setInitialized();
+ }
+
+ @Override
+ public void update() {
+ throw new IllegalStateException("A static table location provider cannot be updated");
+ }
+
+ @Override
+ public void update(long snapshotId) {
+ throw new IllegalStateException("A static table location provider cannot be updated");
+ }
+
+ @Override
+ public void update(Snapshot snapshot) {
+ throw new IllegalStateException("A static table location provider cannot be updated");
+ }
+}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
new file mode 100644
index 00000000000..52c98718317
--- /dev/null
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
@@ -0,0 +1,97 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.iceberg.layout;
+
+import io.deephaven.engine.table.impl.locations.*;
+import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider;
+import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
+import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
+import io.deephaven.iceberg.util.IcebergCatalogAdapter;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public abstract class IcebergTableLocationProviderBase
+ extends AbstractTableLocationProvider {
+
+ final IcebergBaseLayout locationKeyFinder;
+ final TableLocationFactory locationFactory;
+ final TableDataRefreshService refreshService;
+ final IcebergCatalogAdapter adapter;
+ final TableIdentifier tableIdentifier;
+
+ private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken;
+
+ public IcebergTableLocationProviderBase(
+ @NotNull final TK tableKey,
+ @NotNull final IcebergBaseLayout locationKeyFinder,
+ @NotNull final TableLocationFactory locationFactory,
+ @Nullable final TableDataRefreshService refreshService,
+ @NotNull final IcebergCatalogAdapter adapter,
+ @NotNull final TableIdentifier tableIdentifier) {
+ super(tableKey, refreshService != null);
+ this.locationKeyFinder = locationKeyFinder;
+ this.locationFactory = locationFactory;
+ this.refreshService = refreshService;
+ this.adapter = adapter;
+ this.tableIdentifier = tableIdentifier;
+ }
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // AbstractTableLocationProvider implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ /**
+ * Update the table location provider with the latest snapshot from the catalog.
+ */
+ public abstract void update();
+
+ /**
+ * Update the table location provider with a specific snapshot from the catalog. If the {@code snapshotId} is not
+ * found in the list of snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot
+ * must also be newer (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException}
+ * is thrown.
+ *
+ * @param snapshotId The identifier of the snapshot to use when updating the table.
+ */
+ public abstract void update(final long snapshotId);
+
+ /**
+ * Update the table location provider with a specific snapshot from the catalog. The input snapshot must be newer
+ * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown.
+ *
+ * @param snapshot The snapshot to use when updating the table.
+ */
+ public abstract void update(final Snapshot snapshot);
+
+ @Override
+ @NotNull
+ protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) {
+ // noinspection unchecked
+ return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService);
+ }
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // SubscriptionAggregator implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ @Override
+ protected final void activateUnderlyingDataSource() {
+ subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this);
+ }
+
+ @Override
+ protected final void deactivateUnderlyingDataSource() {
+ if (subscriptionToken != null) {
+ subscriptionToken.cancel();
+ subscriptionToken = null;
+ }
+ }
+
+ @Override
+ protected final boolean matchSubscriptionToken(final T token) {
+ return token == subscriptionToken;
+ }
+}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index 76d099e9499..bf23dbd6e85 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -6,23 +6,17 @@
import io.deephaven.engine.context.ExecutionContext;
import io.deephaven.engine.rowset.RowSetFactory;
import io.deephaven.engine.table.*;
-import io.deephaven.engine.table.impl.PartitionAwareSourceTable;
import io.deephaven.engine.table.impl.QueryTable;
import io.deephaven.engine.table.impl.locations.TableDataException;
import io.deephaven.engine.table.impl.locations.TableKey;
-import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider;
import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder;
-import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider;
import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey;
import io.deephaven.engine.table.impl.locations.util.ExecutorTableDataRefreshService;
import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
import io.deephaven.engine.table.impl.sources.InMemoryColumnSource;
import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl;
import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
-import io.deephaven.iceberg.layout.IcebergBaseLayout;
-import io.deephaven.iceberg.layout.IcebergFlatLayout;
-import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout;
-import io.deephaven.iceberg.layout.IcebergRefreshingTableLocationProvider;
+import io.deephaven.iceberg.layout.*;
import io.deephaven.iceberg.location.IcebergTableLocationFactory;
import io.deephaven.iceberg.location.IcebergTableLocationKey;
import org.apache.iceberg.PartitionField;
@@ -354,11 +348,18 @@ public Table listSnapshotsAsTable(@NotNull final String tableIdentifier) {
return listSnapshotsAsTable(TableIdentifier.parse(tableIdentifier));
}
- Snapshot getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long snapshotId) {
+ /**
+ * Retrieve a specific {@link Snapshot snapshot} of an Iceberg table.
+ *
+ * @param tableIdentifier The identifier of the table from which to load the snapshot.
+ * @param snapshotId The identifier of the snapshot to load.
+ *
+ * @return An Optional containing the requested snapshot if it exists.
+ */
+ Optional getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long snapshotId) {
return listSnapshots(tableIdentifier).stream()
.filter(snapshot -> snapshot.snapshotId() == snapshotId)
- .findFirst()
- .orElse(null);
+ .findFirst();
}
/**
@@ -380,7 +381,7 @@ public Snapshot getCurrentSnapshot(@NotNull final TableIdentifier tableIdentifie
* @return The loaded table
*/
@SuppressWarnings("unused")
- public Table readTable(
+ public IcebergTable readTable(
@NotNull final TableIdentifier tableIdentifier,
@Nullable final IcebergInstructions instructions) {
return readTableInternal(tableIdentifier, null, instructions);
@@ -394,7 +395,7 @@ public Table readTable(
* @return The loaded table
*/
@SuppressWarnings("unused")
- public Table readTable(
+ public IcebergTable readTable(
@NotNull final String tableIdentifier,
@Nullable final IcebergInstructions instructions) {
return readTable(TableIdentifier.parse(tableIdentifier), instructions);
@@ -409,16 +410,15 @@ public Table readTable(
* @return The loaded table
*/
@SuppressWarnings("unused")
- public Table readTable(
+ public IcebergTable readTable(
@NotNull final TableIdentifier tableIdentifier,
final long tableSnapshotId,
@Nullable final IcebergInstructions instructions) {
// Find the snapshot with the given snapshot id
- final Snapshot tableSnapshot = getSnapshot(tableIdentifier, tableSnapshotId);
- if (tableSnapshot == null) {
- throw new IllegalArgumentException("Snapshot with id " + tableSnapshotId + " not found");
- }
+ final Snapshot tableSnapshot =
+ getSnapshot(tableIdentifier, tableSnapshotId).orElseThrow(() -> new IllegalArgumentException(
+ "Snapshot with id " + tableSnapshotId + " not found for table " + tableIdentifier));
return readTableInternal(tableIdentifier, tableSnapshot, instructions);
}
@@ -432,7 +432,7 @@ public Table readTable(
* @return The loaded table
*/
@SuppressWarnings("unused")
- public Table readTable(
+ public IcebergTable readTable(
@NotNull final String tableIdentifier,
final long tableSnapshotId,
@Nullable final IcebergInstructions instructions) {
@@ -448,14 +448,14 @@ public Table readTable(
* @return The loaded table
*/
@SuppressWarnings("unused")
- public Table readTable(
+ public IcebergTable readTable(
@NotNull final TableIdentifier tableIdentifier,
@NotNull final Snapshot tableSnapshot,
@Nullable final IcebergInstructions instructions) {
return readTableInternal(tableIdentifier, tableSnapshot, instructions);
}
- private Table readTableInternal(
+ private IcebergTable readTableInternal(
@NotNull final TableIdentifier tableIdentifier,
@Nullable final Snapshot tableSnapshot,
@Nullable final IcebergInstructions instructions) {
@@ -506,7 +506,6 @@ private Table readTableInternal(
tableDef = icebergTableDef;
}
- final String description;
final IcebergBaseLayout keyFinder;
if (partitionSpec.isUnpartitioned()) {
@@ -519,27 +518,25 @@ private Table readTableInternal(
}
if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.STATIC) {
- description = "Read static iceberg table with " + keyFinder;
-
- final AbstractTableLocationProvider locationProvider = new PollingTableLocationProvider<>(
- StandaloneTableKey.getInstance(),
- toKnownKeys(keyFinder),
- new IcebergTableLocationFactory(),
- null);
-
- return new PartitionAwareSourceTable(
+ final IcebergTableLocationProviderBase locationProvider =
+ new IcebergStaticTableLocationProvider<>(
+ StandaloneTableKey.getInstance(),
+ keyFinder,
+ new IcebergTableLocationFactory(),
+ this,
+ tableIdentifier);
+
+ return new IcebergTableStatic(
tableDef,
- description,
+ tableIdentifier.toString(),
RegionedTableComponentFactoryImpl.INSTANCE,
- locationProvider,
- null);
+ locationProvider);
}
final UpdateSourceRegistrar updateSourceRegistrar;
final IcebergRefreshingTableLocationProvider locationProvider;
if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) {
- description = "Read manual refreshing iceberg table with " + keyFinder;
updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
locationProvider = new IcebergRefreshingTableLocationProvider<>(
@@ -551,7 +548,6 @@ private Table readTableInternal(
tableIdentifier,
false);
} else {
- description = "Read automatic refreshing iceberg table with " + keyFinder;
updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
final TableDataRefreshService refreshService =
@@ -567,9 +563,9 @@ private Table readTableInternal(
true);
}
- return new IcebergTable(
+ return new IcebergTableRefreshing(
tableDef,
- description,
+ tableIdentifier.toString(),
RegionedTableComponentFactoryImpl.INSTANCE,
locationProvider,
updateSourceRegistrar);
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
index 9c422b96968..a0ae36a6eaa 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
@@ -3,58 +3,33 @@
//
package io.deephaven.iceberg.util;
-import io.deephaven.engine.table.TableDefinition;
-import io.deephaven.engine.table.impl.PartitionAwareSourceTable;
-import io.deephaven.engine.table.impl.SourceTableComponentFactory;
-import io.deephaven.engine.table.impl.locations.TableKey;
-import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
-import io.deephaven.iceberg.layout.IcebergRefreshingTableLocationProvider;
-import io.deephaven.iceberg.location.IcebergTableLocationKey;
+import io.deephaven.engine.table.Table;
import org.apache.iceberg.Snapshot;
-import org.apache.iceberg.catalog.TableIdentifier;
import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-/**
- * Table wrapper for refreshing Iceberg tables.
- */
-public class IcebergTable extends PartitionAwareSourceTable {
+public interface IcebergTable extends Table {
/**
- * Location discovery.
+ * Update the table with the latest snapshot from the catalog.
*/
- final IcebergRefreshingTableLocationProvider locationProvider;
+ @SuppressWarnings("unused")
+ void update();
/**
+ * Update the table with a specific snapshot from the catalog. If the {@code snapshotId} is not found in the list of
+ * snapshots for the table, an {@link IllegalArgumentException} is thrown. The input snapshot must also be newer
+ * (higher in sequence number) than the current snapshot or an {@link IllegalArgumentException} is thrown.
*
- *
- * @param tableDefinition A TableDefinition
- * @param description A human-readable description for this table
- * @param componentFactory A component factory for creating column source managers
- * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table
- * @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live
+ * @param snapshotId The identifier of the snapshot to use when updating the table.
*/
- IcebergTable(
- @NotNull TableDefinition tableDefinition,
- @NotNull String description,
- @NotNull SourceTableComponentFactory componentFactory,
- @NotNull IcebergRefreshingTableLocationProvider locationProvider,
- @Nullable UpdateSourceRegistrar updateSourceRegistrar) {
- super(tableDefinition, description, componentFactory, locationProvider, updateSourceRegistrar);
- this.locationProvider = locationProvider;
- }
-
- @SuppressWarnings("unused")
- public void update() {
- locationProvider.update();
- }
-
@SuppressWarnings("unused")
- public void update(final long snapshotId) {
- locationProvider.update(snapshotId);
- }
+ void update(final long snapshotId);
+ /**
+ * Update the table with a specific snapshot from the catalog. The input snapshot must be newer (higher in sequence
+ * number) than the current snapshot or an {@link IllegalArgumentException} is thrown.
+ *
+ * @param snapshot The snapshot to use when updating the table.
+ */
@SuppressWarnings("unused")
- public void update(final @NotNull Snapshot snapshot) {
- locationProvider.update(snapshot);
- }
+ void update(final @NotNull Snapshot snapshot);
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java
new file mode 100644
index 00000000000..72623b9e200
--- /dev/null
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java
@@ -0,0 +1,59 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.iceberg.util;
+
+import io.deephaven.engine.table.TableDefinition;
+import io.deephaven.engine.table.impl.PartitionAwareSourceTable;
+import io.deephaven.engine.table.impl.SourceTableComponentFactory;
+import io.deephaven.engine.table.impl.locations.TableKey;
+import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
+import io.deephaven.iceberg.layout.IcebergTableLocationProviderBase;
+import io.deephaven.iceberg.location.IcebergTableLocationKey;
+import org.apache.iceberg.Snapshot;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Table wrapper for refreshing Iceberg tables.
+ */
+public class IcebergTableRefreshing extends PartitionAwareSourceTable implements IcebergTable {
+ /**
+ * Location discovery.
+ */
+ final IcebergTableLocationProviderBase locationProvider;
+
+ /**
+ *
+ *
+ * @param tableDefinition A TableDefinition
+ * @param description A human-readable description for this table
+ * @param componentFactory A component factory for creating column source managers
+ * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table
+ * @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live
+ */
+ IcebergTableRefreshing(
+ @NotNull TableDefinition tableDefinition,
+ @NotNull String description,
+ @NotNull SourceTableComponentFactory componentFactory,
+ @NotNull IcebergTableLocationProviderBase locationProvider,
+ @Nullable UpdateSourceRegistrar updateSourceRegistrar) {
+ super(tableDefinition, description, componentFactory, locationProvider, updateSourceRegistrar);
+ this.locationProvider = locationProvider;
+ }
+
+ @Override
+ public void update() {
+ locationProvider.update();
+ }
+
+ @Override
+ public void update(final long snapshotId) {
+ locationProvider.update(snapshotId);
+ }
+
+ @Override
+ public void update(final @NotNull Snapshot snapshot) {
+ locationProvider.update(snapshot);
+ }
+}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java
new file mode 100644
index 00000000000..0884a6379c5
--- /dev/null
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java
@@ -0,0 +1,49 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.iceberg.util;
+
+import io.deephaven.engine.table.TableDefinition;
+import io.deephaven.engine.table.impl.PartitionAwareSourceTable;
+import io.deephaven.engine.table.impl.SourceTableComponentFactory;
+import io.deephaven.engine.table.impl.locations.TableKey;
+import io.deephaven.iceberg.layout.IcebergTableLocationProviderBase;
+import io.deephaven.iceberg.location.IcebergTableLocationKey;
+import org.apache.iceberg.Snapshot;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Table wrapper for refreshing Iceberg tables.
+ */
+public class IcebergTableStatic extends PartitionAwareSourceTable implements IcebergTable {
+ /**
+ *
+ *
+ * @param tableDefinition A TableDefinition
+ * @param description A human-readable description for this table
+ * @param componentFactory A component factory for creating column source managers
+ * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table
+ */
+ IcebergTableStatic(
+ @NotNull TableDefinition tableDefinition,
+ @NotNull String description,
+ @NotNull SourceTableComponentFactory componentFactory,
+ @NotNull IcebergTableLocationProviderBase locationProvider) {
+ super(tableDefinition, description, componentFactory, locationProvider, null);
+ }
+
+ @Override
+ public void update() {
+ throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description);
+ }
+
+ @Override
+ public void update(final long snapshotId) {
+ throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description);
+ }
+
+ @Override
+ public void update(final @NotNull Snapshot snapshot) {
+ throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description);
+ }
+}
From fb882e8e76956f940fc74362ae307b5d3afa6137 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Fri, 26 Jul 2024 12:11:48 -0700
Subject: [PATCH 06/56] WIP, some PR comments addressed.
---
.../util/datastructures/SubscriptionSet.java | 31 +++++++
.../impl/locations/TableLocationProvider.java | 49 +++++++++--
.../impl/AbstractTableLocationProvider.java | 86 ++++++++++++++++---
.../impl/FilteredTableDataService.java | 8 +-
.../impl/PollingTableLocationProvider.java | 2 +-
.../impl/TableLocationSubscriptionBuffer.java | 55 +++++++++---
.../TableBackedTableLocationProvider.java | 2 +-
...cebergRefreshingTableLocationProvider.java | 2 +-
.../IcebergStaticTableLocationProvider.java | 2 +-
9 files changed, 196 insertions(+), 41 deletions(-)
diff --git a/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java b/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java
index 0d9fc96a8c0..d5c1f711a93 100644
--- a/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java
+++ b/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java
@@ -7,6 +7,7 @@
import io.deephaven.base.reference.WeakReferenceWrapper;
import io.deephaven.base.verify.Assert;
import io.deephaven.base.verify.Require;
+import org.apache.commons.lang3.function.TriConsumer;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
@@ -219,6 +220,36 @@ public final boolean deliverNotification(
return initialSize > 0 && size == 0;
}
+ /**
+ * Dispatch a binary notification to all subscribers. Clean up any GC'd subscriptions.
+ *
+ * @param procedure The notification procedure to invoke
+ * @param notification The notification to deliver
+ * @param token An additional token to deliver (usually a transaction token)
+ * @param activeOnly Whether to restrict this notification to active subscriptions only
+ * @return Whether this operation caused the set to become empty
+ */
+ public final boolean deliverNotification(
+ @NotNull final TriConsumer procedure,
+ @Nullable final NOTIFICATION_TYPE notification,
+ @Nullable final Object token,
+ final boolean activeOnly) {
+ final int initialSize = size;
+ for (int si = 0; si < size;) {
+ final Entry currentEntry = subscriptions[si];
+ final LISTENER_TYPE currentListener = currentEntry.getListener();
+ if (currentListener == null) {
+ removeAt(si);
+ continue; // si is not incremented in this case - we'll reconsider the same slot if necessary.
+ }
+ if (!activeOnly || currentEntry.isActive()) {
+ procedure.accept(currentListener, notification, token);
+ }
+ ++si;
+ }
+ return initialSize > 0 && size == 0;
+ }
+
private void removeAt(final int subscriptionIndex) {
final int lastSubscriptionIndex = --size;
subscriptions[subscriptionIndex] = subscriptions[lastSubscriptionIndex];
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
index 1390bb3b1f4..9ba7a350d81 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
@@ -31,14 +31,29 @@ interface Listener extends BasicTableDataListener {
*
* @param token A token to identify the transaction.
*/
- void beginTransaction(final Object token);
+ void beginTransaction(@NotNull Object token);
+
+ /**
+ * Begin a transaction that collects location key additions and removals to be processed atomically. Uses
+ * {@code this} as the token.
+ */
+ default void beginTransaction() {
+ beginTransaction(this);
+ }
/**
* End the transaction and process the location changes.
*
* @param token A token to identify the transaction.
*/
- void endTransaction(final Object token);
+ void endTransaction(@NotNull Object token);
+
+ /**
+ * End the transaction and process the location changes. Uses {@code this} as the token.
+ */
+ default void endTransaction() {
+ endTransaction(this);
+ }
/**
* Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location
@@ -49,8 +64,19 @@ interface Listener extends BasicTableDataListener {
* @param transactionToken The token identifying the transaction.
*/
void handleTableLocationKeyAdded(
- @NotNull final ImmutableTableLocationKey tableLocationKey,
- final Object transactionToken);
+ @NotNull ImmutableTableLocationKey tableLocationKey,
+ @Nullable Object transactionToken);
+
+ /**
+ * Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location
+ * subscription. This should occur at most once per location, but the order of delivery is not
+ * guaranteed. Uses {@code this} as the token.
+ *
+ * @param tableLocationKey The new table location key.
+ */
+ default void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey) {
+ handleTableLocationKeyAdded(tableLocationKey, this);
+ }
/**
* Notify the listener of a {@link TableLocationKey} that has been removed.
@@ -59,8 +85,17 @@ void handleTableLocationKeyAdded(
* @param transactionToken The token identifying the transaction.
*/
void handleTableLocationKeyRemoved(
- @NotNull final ImmutableTableLocationKey tableLocationKey,
- final Object transactionToken);
+ @NotNull ImmutableTableLocationKey tableLocationKey,
+ @Nullable Object transactionToken);
+
+ /**
+ * Notify the listener of a {@link TableLocationKey} that has been removed. Uses {@code this} as the token.
+ *
+ * @param tableLocationKey The table location key that was removed.
+ */
+ default void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey) {
+ handleTableLocationKeyRemoved(tableLocationKey, this);
+ }
}
/**
@@ -126,7 +161,7 @@ void handleTableLocationKeyRemoved(
* @param tableLocationKey The key to test.
* @return Whether the key is known to this provider.
*/
- boolean hasTableLocationKey(@NotNull final TableLocationKey tableLocationKey);
+ boolean hasTableLocationKey(@NotNull TableLocationKey tableLocationKey);
/**
* @param tableLocationKey A {@link TableLocationKey} specifying the location to get.
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index dd3eab3ca54..1a3f7ead2da 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -16,7 +16,7 @@
* Partial {@link TableLocationProvider} implementation for standalone use or as part of a {@link TableDataService}.
*
* Presents an interface similar to {@link TableLocationProvider.Listener} for subclasses to use when communicating with
- * the parent; see {@link #handleTableLocationKey(TableLocationKey)}.
+ * the parent; see {@link #handleTableLocationKeyAdded(TableLocationKey)}.
*
* Note that subclasses are responsible for determining when it's appropriate to call {@link #setInitialized()} and/or
* override {@link #doInitialization()}.
@@ -84,18 +84,32 @@ public final ImmutableTableKey getKey() {
@Override
protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) {
- listener.beginTransaction(this);
- unmodifiableTableLocationKeys.forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, this));
- listener.endTransaction(this);
+ listener.beginTransaction();
+ unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded);
+ listener.endTransaction();
+ }
+
+ /**
+ * Deliver a possibly-new key. This method passes {@code this} as the transaction token.
+ *
+ * @param locationKey The new key
+ * @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools.
+ */
+ protected final void handleTableLocationKeyAdded(
+ @NotNull final TableLocationKey locationKey) {
+ handleTableLocationKeyAdded(locationKey, this);
}
/**
* Deliver a possibly-new key.
*
* @param locationKey The new key
+ * @param transactionToken The token identifying the transaction
* @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools.
*/
- protected final void handleTableLocationKey(@NotNull final TableLocationKey locationKey) {
+ protected final void handleTableLocationKeyAdded(
+ @NotNull final TableLocationKey locationKey,
+ @Nullable final Object transactionToken) {
if (!supportsSubscriptions()) {
tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable);
visitLocationKey(toKeyImmutable(locationKey));
@@ -113,30 +127,57 @@ protected final void handleTableLocationKey(@NotNull final TableLocationKey loca
if (locationCreatedRecorder) {
verifyPartitionKeys(locationKey);
if (subscriptions.deliverNotification(
- (listener, tlk) -> listener.handleTableLocationKeyAdded(tlk,
- AbstractTableLocationProvider.this),
- toKeyImmutable(result), true)) {
+ Listener::handleTableLocationKeyAdded,
+ toKeyImmutable(result),
+ transactionToken,
+ true)) {
onEmpty();
}
}
}
}
+ /**
+ * Internal method to begin an atomic transaction of location adds and removes. This method passes {@code this} as
+ * the transaction token.
+ */
protected final void beginTransaction() {
+ beginTransaction(this);
+ }
+
+ /**
+ * Internal method to begin an atomic transaction of location adds and removes.
+ *
+ * @param token A token to identify the transaction
+ */
+ protected final void beginTransaction(@NotNull final Object token) {
if (subscriptions != null) {
- subscriptions.deliverNotification(listener -> listener.beginTransaction(this), true);
+ subscriptions.deliverNotification(Listener::beginTransaction, token, true);
}
}
+ /**
+ * Internal method to end an atomic transaction of location adds and removes. This method passes {@code this} as the
+ * transaction token.
+ */
protected final void endTransaction() {
+ endTransaction(this);
+ }
+
+ /**
+ * Internal method to end an atomic transaction of location adds and removes.
+ *
+ * @param token A token to identify the transaction
+ */
+ protected final void endTransaction(@NotNull final Object token) {
if (subscriptions != null) {
- subscriptions.deliverNotification(listener -> listener.endTransaction(this), true);
+ subscriptions.deliverNotification(Listener::endTransaction, token, true);
}
}
/**
- * Called after a table location has been visited by {@link #handleTableLocationKey(TableLocationKey)}, but
- * before notifications have been delivered to any subscriptions, if applicable. The default implementation does
+ * Called after a table location has been visited by {@link #handleTableLocationKeyAdded(TableLocationKey)},
+ * but before notifications have been delivered to any subscriptions, if applicable. The default implementation does
* nothing, and may be overridden to implement additional features.
*
* @param locationKey The {@link TableLocationKey} that was visited.
@@ -273,16 +314,33 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey)
}
}
+ /**
+ * Notify subscribers that {@code locationKey} was removed. This method passes {@code this} as the transaction
+ * token.
+ *
+ * @param locationKey the TableLocation that was removed
+ */
+ protected final void handleTableLocationKeyRemoved(
+ @NotNull final ImmutableTableLocationKey locationKey) {
+ handleTableLocationKeyRemoved(locationKey, this);
+ }
+
/**
* Notify subscribers that {@code locationKey} was removed.
*
* @param locationKey the TableLocation that was removed
+ * @param transactionToken The token identifying the transaction
*/
- protected void handleTableLocationKeyRemoved(@NotNull final ImmutableTableLocationKey locationKey) {
+ protected void handleTableLocationKeyRemoved(
+ @NotNull final ImmutableTableLocationKey locationKey,
+ @Nullable final Object transactionToken) {
if (supportsSubscriptions()) {
synchronized (subscriptions) {
if (subscriptions.deliverNotification(
- (listener, tlk) -> listener.handleTableLocationKeyRemoved(tlk, this), locationKey, true)) {
+ Listener::handleTableLocationKeyRemoved,
+ locationKey,
+ transactionToken,
+ true)) {
onEmpty();
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
index 7c5c090f780..6592ea11a12 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
@@ -157,7 +157,7 @@ private FilteringListener(@NotNull final TableLocationProvider.Listener outputLi
}
@Override
- public void beginTransaction(final Object token) {
+ public void beginTransaction(@NotNull final Object token) {
// Delegate to the wrapped listener.
final TableLocationProvider.Listener outputListener = getWrapped();
if (outputListener != null) {
@@ -166,7 +166,7 @@ public void beginTransaction(final Object token) {
}
@Override
- public void endTransaction(final Object token) {
+ public void endTransaction(@NotNull final Object token) {
// Delegate to the wrapped listener.
final TableLocationProvider.Listener outputListener = getWrapped();
if (outputListener != null) {
@@ -177,7 +177,7 @@ public void endTransaction(final Object token) {
@Override
public void handleTableLocationKeyAdded(
@NotNull final ImmutableTableLocationKey tableLocationKey,
- final Object transactionToken) {
+ @Nullable final Object transactionToken) {
final TableLocationProvider.Listener outputListener = getWrapped();
// We can't try to clean up null listeners here, the underlying implementation may not allow concurrent
// unsubscribe operations.
@@ -189,7 +189,7 @@ public void handleTableLocationKeyAdded(
@Override
public void handleTableLocationKeyRemoved(
@NotNull final ImmutableTableLocationKey tableLocationKey,
- final Object transactionToken) {
+ @Nullable final Object transactionToken) {
final TableLocationProvider.Listener outputListener = getWrapped();
if (outputListener != null && locationKeyFilter.accept(tableLocationKey)) {
outputListener.handleTableLocationKeyRemoved(tableLocationKey, transactionToken);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
index 0f9302b7a77..5ac5c65c4e0 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
@@ -57,7 +57,7 @@ public void refresh() {
locationKeyFinder.findKeys(tableLocationKey -> {
// noinspection SuspiciousMethodCalls
missedKeys.remove(tableLocationKey);
- handleTableLocationKey(tableLocationKey);
+ handleTableLocationKeyAdded(tableLocationKey);
});
missedKeys.forEach(this::handleTableLocationKeyRemoved);
endTransaction();
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
index c9998af0ee6..29e654a3633 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
@@ -8,6 +8,7 @@
import io.deephaven.engine.table.impl.locations.TableDataException;
import io.deephaven.engine.table.impl.locations.TableLocationProvider;
import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
import java.util.*;
@@ -121,7 +122,7 @@ public synchronized void reset() {
// ------------------------------------------------------------------------------------------------------------------
@Override
- public void beginTransaction(final Object token) {
+ public void beginTransaction(@NotNull final Object token) {
synchronized (updateLock) {
// Assert that we can start a new transaction with this token.
Require.eqFalse(transactionTokens.contains(token), "transactionTokens.contains(token)");
@@ -137,7 +138,7 @@ public void beginTransaction(final Object token) {
}
@Override
- public void endTransaction(final Object token) {
+ public void endTransaction(@NotNull final Object token) {
synchronized (updateLock) {
// Assert that this transaction is open.
Require.eqTrue(transactionTokens.contains(token), "transactionTokens.contains(token)");
@@ -179,18 +180,31 @@ public void endTransaction(final Object token) {
@Override
public void handleTableLocationKeyAdded(
@NotNull final ImmutableTableLocationKey tableLocationKey,
- final Object transactionToken) {
+ @Nullable Object transactionToken) {
synchronized (updateLock) {
+ if (transactionToken == null) {
+ // If we're not in a transaction, modify the pending locations directly.
+ if (pendingLocationsRemoved.remove(tableLocationKey)) {
+ return;
+ }
+ pendingLocationsAdded.add(tableLocationKey);
+ return;
+ }
+
if (accumulatedLocationsAdded.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) {
accumulatedLocationsAdded.put(transactionToken, new HashSet<>());
}
final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken);
- final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken);
-
- // A single transaction should never add and remove the same location,
- Require.eqFalse(locationsRemoved.contains(tableLocationKey),
- "locationsRemoved.contains(tableLocationKey)");
+ // When adding a location in a transaction, check for logical consistency.
+ // 1. If the location was already added in this transaction, we have a problem. A transaction should not
+ // add the same location twice.
+ // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a
+ // logical error (although it may not be supported by all consumers).
+ if (accumulatedLocationsAdded.containsKey(tableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ + " was added twice in the same transaction.");
+ }
locationsAdded.add(tableLocationKey);
}
}
@@ -200,16 +214,33 @@ public void handleTableLocationKeyRemoved(
@NotNull final ImmutableTableLocationKey tableLocationKey,
final Object transactionToken) {
synchronized (updateLock) {
+ if (transactionToken == null) {
+ // If we're not in a transaction, modify the pending locations directly.
+ if (pendingLocationsRemoved.remove(tableLocationKey)) {
+ return;
+ }
+ pendingLocationsAdded.add(tableLocationKey);
+ return;
+ }
+
if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) {
accumulatedLocationsRemoved.put(transactionToken, new HashSet<>());
}
final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken);
final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken);
- // A single transaction should never add and remove the same location,
- Require.eqFalse(locationsAdded.contains(tableLocationKey),
- "locationsAdded.contains(tableLocationKey)");
-
+ // When removing a location in a transaction, check for logical consistency.
+ // 1. If the location was already removed in this transaction, we have a problem. A transaction should not
+ // remove the same location twice.
+ // 2. If the location was already added in this transaction, we have a problem. A transaction should not
+ // add then remove the same location.
+ if (accumulatedLocationsRemoved.containsKey(tableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ + " was removed twice in the same transaction.");
+ } else if (accumulatedLocationsAdded.containsKey(tableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ + " was removed after being added in the same transaction.");
+ }
locationsRemoved.add(tableLocationKey);
}
}
diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java
index 09687f81f1c..6d742706c2f 100644
--- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java
+++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocationProvider.java
@@ -43,7 +43,7 @@ private void processPending(@NotNull final Stream
tableStream) {
.withAttributes(Map.of(LOCATION_ID_ATTR, nextId.getAndIncrement())))
.peek(table -> Assert.assertion(table.isAppendOnly(), "table is append only"))
.map(TableBackedTableLocationKey::new)
- .forEach(this::handleTableLocationKey);
+ .forEach(this::handleTableLocationKeyAdded);
}
public synchronized void addPending(@NotNull final Table toAdd) {
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
index e6958dc63b3..58d7d83b7be 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
@@ -112,7 +112,7 @@ private void refreshSnapshot() {
final Set missedKeys = new HashSet<>(getTableLocationKeys());
locationKeyFinder.findKeys(tableLocationKey -> {
missedKeys.remove(tableLocationKey);
- handleTableLocationKey(tableLocationKey);
+ handleTableLocationKeyAdded(tableLocationKey);
});
missedKeys.forEach(this::handleTableLocationKeyRemoved);
endTransaction();
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
index 4a93d985af0..163952571a2 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
@@ -43,7 +43,7 @@ public String getImplementationName() {
@Override
public void refresh() {
beginTransaction();
- locationKeyFinder.findKeys(this::handleTableLocationKey);
+ locationKeyFinder.findKeys(this::handleTableLocationKeyAdded);
endTransaction();
setInitialized();
}
From 5bbdeb215058b1bb5609ac7a84cdcfe79a6b18a3 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Fri, 26 Jul 2024 20:38:00 -0700
Subject: [PATCH 07/56] WIP, even more PR comments addressed.
---
.../impl/CompositeTableDataService.java | 4 +-
.../impl/TableLocationSubscriptionBuffer.java | 84 ++++++++----
.../util/ExecutorTableDataRefreshService.java | 22 +++-
.../util/TableDataRefreshService.java | 18 +++
...ebergAutoRefreshTableLocationProvider.java | 120 ++++++++++++++++++
...rgManualRefreshTableLocationProvider.java} | 55 ++++----
.../IcebergStaticTableLocationProvider.java | 25 +++-
.../IcebergTableLocationProviderBase.java | 33 +----
.../iceberg/util/IcebergCatalogAdapter.java | 50 +++-----
.../iceberg/util/IcebergInstructions.java | 19 ++-
...eRefreshing.java => IcebergTableImpl.java} | 13 +-
.../iceberg/util/IcebergTableStatic.java | 49 -------
12 files changed, 306 insertions(+), 186 deletions(-)
create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
rename extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/{IcebergRefreshingTableLocationProvider.java => IcebergManualRefreshTableLocationProvider.java} (70%)
rename extensions/iceberg/src/main/java/io/deephaven/iceberg/util/{IcebergTableRefreshing.java => IcebergTableImpl.java} (79%)
delete mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
index a9e782496ca..3abcbf623e3 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
@@ -115,7 +115,9 @@ public void subscribe(@NotNull final Listener listener) {
p.subscribe(listener);
} else {
p.refresh();
- p.getTableLocationKeys().forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, this));
+ listener.beginTransaction(p);
+ p.getTableLocationKeys().forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, p));
+ listener.endTransaction(p);
}
});
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
index 29e654a3633..b9a455ce7af 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
@@ -76,8 +76,9 @@ public synchronized LocationUpdate processPending() {
// NB: Providers that don't support subscriptions don't tick - this single call to run is
// sufficient.
tableLocationProvider.refresh();
+ // TODO: cast this to AbstractTableLocationProvider and call begin/endTransaction?
tableLocationProvider.getTableLocationKeys()
- .forEach(tlk -> handleTableLocationKeyAdded(tlk, tableLocationProvider));
+ .forEach(tlk -> handleTableLocationKeyAdded(tlk, null));
}
subscribed = true;
}
@@ -146,28 +147,39 @@ public void endTransaction(@NotNull final Object token) {
final Set tokenLocationsAdded = accumulatedLocationsAdded.get(token);
final Set tokenLocationsRemoved = accumulatedLocationsRemoved.get(token);
- if (pendingLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) {
- // Handle any locations that were pending as adds but removed by this transaction.
+ if (tokenLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) {
for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsRemoved) {
+ // If we have a pending add that is removed by this transaction, we can remove it from the pending
+ // list because it is cancelled by this remove. This also covers the case where a `replace`
+ // operation has occurred in a previous transaction.
if (pendingLocationsAdded.remove(tableLocationKey)) {
continue;
}
+ // Verify that we don't have stacked removes (without intervening adds).
+ if (pendingLocationsRemoved.contains(tableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ + " was already removed by a previous transaction.");
+ }
+ if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
+ pendingLocationsRemoved = new HashSet<>();
+ }
pendingLocationsRemoved.add(tableLocationKey);
}
- } else {
- pendingLocationsRemoved = tokenLocationsRemoved;
}
- if (pendingLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) {
- // Handle any locations that were pending as removes but added again by this transaction.
+ if (tokenLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) {
for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsAdded) {
- if (pendingLocationsRemoved.remove(tableLocationKey)) {
- continue;
+ // Verify that we don't have stacked adds (without intervening removes).
+ if (pendingLocationsAdded.contains(tableLocationKey)
+ && !pendingLocationsRemoved.contains(tableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ + " was already added by a previous transaction.");
+ }
+ if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) {
+ pendingLocationsAdded = new HashSet<>();
}
pendingLocationsAdded.add(tableLocationKey);
}
- } else {
- pendingLocationsAdded = tokenLocationsAdded;
}
// Clear all the storage for this transaction.
@@ -182,10 +194,22 @@ public void handleTableLocationKeyAdded(
@NotNull final ImmutableTableLocationKey tableLocationKey,
@Nullable Object transactionToken) {
synchronized (updateLock) {
+ // When adding a location in a transaction, check for logical consistency.
+ // 1. If the location was already added in this transaction, we have a problem. A transaction should not
+ // add the same location twice.
+ // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a
+ // logical error (although it may not be supported by all consumers).
+
if (transactionToken == null) {
// If we're not in a transaction, modify the pending locations directly.
- if (pendingLocationsRemoved.remove(tableLocationKey)) {
- return;
+ // Need to verify that we don't have stacked adds (without intervening removes).
+ if (pendingLocationsAdded.contains(tableLocationKey)
+ && !pendingLocationsRemoved.contains(tableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ + " was already added by a previous transaction.");
+ }
+ if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) {
+ pendingLocationsAdded = new HashSet<>();
}
pendingLocationsAdded.add(tableLocationKey);
return;
@@ -196,14 +220,9 @@ public void handleTableLocationKeyAdded(
}
final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken);
- // When adding a location in a transaction, check for logical consistency.
- // 1. If the location was already added in this transaction, we have a problem. A transaction should not
- // add the same location twice.
- // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a
- // logical error (although it may not be supported by all consumers).
if (accumulatedLocationsAdded.containsKey(tableLocationKey)) {
throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was added twice in the same transaction.");
+ + " was added multiple times in the same transaction.");
}
locationsAdded.add(tableLocationKey);
}
@@ -214,29 +233,38 @@ public void handleTableLocationKeyRemoved(
@NotNull final ImmutableTableLocationKey tableLocationKey,
final Object transactionToken) {
synchronized (updateLock) {
+ // When removing a location in a transaction, check for logical consistency.
+ // 1. If the location was already removed in this transaction, we have a problem. A transaction should not
+ // remove the same location twice.
+ // 2. If the location was already added in this transaction, we have a problem. A transaction should not
+ // add then remove the same location.
+
if (transactionToken == null) {
// If we're not in a transaction, modify the pending locations directly.
- if (pendingLocationsRemoved.remove(tableLocationKey)) {
+ // If we have a pending add, it is being cancelled by this remove.
+ if (pendingLocationsAdded.remove(tableLocationKey)) {
return;
}
- pendingLocationsAdded.add(tableLocationKey);
+ // Verify that we don't have stacked removes (without intervening adds).
+ if (pendingLocationsRemoved.contains(tableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ + " was already removed by a previous transaction.");
+ }
+ if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
+ pendingLocationsRemoved = new HashSet<>();
+ }
+ pendingLocationsRemoved.add(tableLocationKey);
return;
}
if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) {
accumulatedLocationsRemoved.put(transactionToken, new HashSet<>());
}
- final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken);
final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken);
- // When removing a location in a transaction, check for logical consistency.
- // 1. If the location was already removed in this transaction, we have a problem. A transaction should not
- // remove the same location twice.
- // 2. If the location was already added in this transaction, we have a problem. A transaction should not
- // add then remove the same location.
if (accumulatedLocationsRemoved.containsKey(tableLocationKey)) {
throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was removed twice in the same transaction.");
+ + " was removed multiple times in the same transaction.");
} else if (accumulatedLocationsAdded.containsKey(tableLocationKey)) {
throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ " was removed after being added in the same transaction.");
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java
index 8c4e016607c..8db2b625354 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java
@@ -19,7 +19,6 @@
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
/**
* {@link TableDataRefreshService} implementation that uses a {@link ScheduledThreadPoolExecutor}.
@@ -76,7 +75,8 @@ private abstract class ScheduledSubscriptionTask {
- private ScheduledTableLocationProviderRefresh(@NotNull AbstractTableLocationProvider tableLocationProvider) {
+ private ScheduledTableLocationProviderRefresh(
+ @NotNull final AbstractTableLocationProvider tableLocationProvider) {
super(tableLocationProvider, tableLocationProviderRefreshIntervalMillis);
providerSubscriptions.increment(1);
}
+ private ScheduledTableLocationProviderRefresh(
+ @NotNull final AbstractTableLocationProvider tableLocationProvider,
+ final long refreshIntervalMillis) {
+ super(tableLocationProvider, refreshIntervalMillis);
+ providerSubscriptions.increment(1);
+ }
+
+
@Override
protected void refresh() {
final long startTimeNanos = System.nanoTime();
@@ -156,6 +165,13 @@ public CancellableSubscriptionToken scheduleTableLocationProviderRefresh(
return new ScheduledTableLocationProviderRefresh(tableLocationProvider);
}
+ @Override
+ public CancellableSubscriptionToken scheduleTableLocationProviderRefresh(
+ @NotNull final AbstractTableLocationProvider tableLocationProvider,
+ final long refreshIntervalMillis) {
+ return new ScheduledTableLocationProviderRefresh(tableLocationProvider, refreshIntervalMillis);
+ }
+
@Override
public CancellableSubscriptionToken scheduleTableLocationRefresh(
@NotNull final AbstractTableLocation tableLocation) {
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java
index c6b823c3139..093426d0bb6 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java
@@ -40,6 +40,17 @@ interface CancellableSubscriptionToken {
CancellableSubscriptionToken scheduleTableLocationProviderRefresh(
@NotNull AbstractTableLocationProvider tableLocationProvider);
+ /**
+ * Schedule run for an AbstractTableLocationProvider.
+ *
+ * @param tableLocationProvider The table location provider
+ * @param refreshIntervalMs The interval in milliseconds between refreshes
+ * @return A subscription token to be used for matching, which also supports cancellation
+ */
+ CancellableSubscriptionToken scheduleTableLocationProviderRefresh(
+ @NotNull AbstractTableLocationProvider tableLocationProvider,
+ long refreshIntervalMs);
+
/**
* Schedule run for an AbstractTableLocation.
*
@@ -126,6 +137,13 @@ public CancellableSubscriptionToken scheduleTableLocationProviderRefresh(
throw new UnsupportedOperationException();
}
+ @Override
+ public CancellableSubscriptionToken scheduleTableLocationProviderRefresh(
+ @NotNull final AbstractTableLocationProvider tableLocationProvider,
+ final long refreshIntervalMs) {
+ throw new UnsupportedOperationException();
+ }
+
@Override
public CancellableSubscriptionToken scheduleTableLocationRefresh(
@NotNull final AbstractTableLocation tableLocation) {
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
new file mode 100644
index 00000000000..bc9fb230eda
--- /dev/null
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
@@ -0,0 +1,120 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.iceberg.layout;
+
+import io.deephaven.base.verify.Assert;
+import io.deephaven.engine.table.impl.locations.*;
+import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
+import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
+import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
+import io.deephaven.iceberg.util.IcebergCatalogAdapter;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.jetbrains.annotations.NotNull;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ *
+ * Automatically refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location
+ * key} discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a
+ * {@link TableLocationFactory}.
+ *
+ */
+public class IcebergAutoRefreshTableLocationProvider
+ extends IcebergTableLocationProviderBase {
+
+ private static final String IMPLEMENTATION_NAME = IcebergAutoRefreshTableLocationProvider.class.getSimpleName();
+
+ private final long refreshIntervalMs;
+
+ private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken;
+
+ public IcebergAutoRefreshTableLocationProvider(
+ @NotNull final TK tableKey,
+ @NotNull final IcebergBaseLayout locationKeyFinder,
+ @NotNull final TableLocationFactory locationFactory,
+ @NotNull final TableDataRefreshService refreshService,
+ final long refreshIntervalMs,
+ @NotNull final IcebergCatalogAdapter adapter,
+ @NotNull final TableIdentifier tableIdentifier) {
+ super(tableKey, locationKeyFinder, locationFactory, refreshService, true, adapter, tableIdentifier);
+
+ Assert.neqNull(refreshService, "refreshService");
+ this.refreshIntervalMs = refreshIntervalMs;
+ }
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // AbstractTableLocationProvider implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public String getImplementationName() {
+ return IMPLEMENTATION_NAME;
+ }
+
+ @Override
+ public synchronized void refresh() {
+ final Snapshot latestSnapshot = adapter.getCurrentSnapshot(tableIdentifier);
+ if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) {
+ locationKeyFinder.snapshot = latestSnapshot;
+ refreshSnapshot();
+ }
+ }
+
+ @Override
+ public void update() {
+ throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated manually");
+ }
+
+ @Override
+ public void update(long snapshotId) {
+ throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated");
+ }
+
+ @Override
+ public void update(Snapshot snapshot) {
+ throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated");
+ }
+
+ /**
+ * Refresh the table location provider with the latest snapshot from the catalog. This method will identify new
+ * locations and removed locations.
+ */
+ private void refreshSnapshot() {
+ beginTransaction();
+ final Set missedKeys = new HashSet<>(getTableLocationKeys());
+ locationKeyFinder.findKeys(tableLocationKey -> {
+ missedKeys.remove(tableLocationKey);
+ handleTableLocationKeyAdded(tableLocationKey);
+ });
+ missedKeys.forEach(this::handleTableLocationKeyRemoved);
+ endTransaction();
+ setInitialized();
+ }
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // SubscriptionAggregator implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ @Override
+ protected final void activateUnderlyingDataSource() {
+ refreshSnapshot();
+ subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this, refreshIntervalMs);
+ }
+
+ @Override
+ protected final void deactivateUnderlyingDataSource() {
+ if (subscriptionToken != null) {
+ subscriptionToken.cancel();
+ subscriptionToken = null;
+ }
+ }
+
+ @Override
+ protected final boolean matchSubscriptionToken(final T token) {
+ return token == subscriptionToken;
+ }
+}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
similarity index 70%
rename from extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
rename to extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
index 58d7d83b7be..9287cfa64dc 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergRefreshingTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
@@ -6,12 +6,10 @@
import io.deephaven.engine.table.impl.locations.*;
import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
-import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
import io.deephaven.iceberg.util.IcebergCatalogAdapter;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.catalog.TableIdentifier;
import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
import java.util.HashSet;
import java.util.List;
@@ -19,33 +17,23 @@
/**
*
- * Refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key}
+ * Manually refreshing {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key}
* discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a
* {@link TableLocationFactory}.
*
- *
- * Supports both automatic and manual refreshing cases, distinguished by the {@code autoRefresh} parameter.
- *
*/
-public class IcebergRefreshingTableLocationProvider
+public class IcebergManualRefreshTableLocationProvider
extends IcebergTableLocationProviderBase {
- private static final String IMPLEMENTATION_NAME = IcebergRefreshingTableLocationProvider.class.getSimpleName();
-
- private final boolean autoRefresh;
+ private static final String IMPLEMENTATION_NAME = IcebergManualRefreshTableLocationProvider.class.getSimpleName();
- private TableDataRefreshService.CancellableSubscriptionToken subscriptionToken;
-
- public IcebergRefreshingTableLocationProvider(
+ public IcebergManualRefreshTableLocationProvider(
@NotNull final TK tableKey,
@NotNull final IcebergBaseLayout locationKeyFinder,
@NotNull final TableLocationFactory locationFactory,
- @Nullable final TableDataRefreshService refreshService,
@NotNull final IcebergCatalogAdapter adapter,
- @NotNull final TableIdentifier tableIdentifier,
- final boolean autoRefresh) {
- super(tableKey, locationKeyFinder, locationFactory, refreshService, adapter, tableIdentifier);
- this.autoRefresh = autoRefresh;
+ @NotNull final TableIdentifier tableIdentifier) {
+ super(tableKey, locationKeyFinder, locationFactory, null, true, adapter, tableIdentifier);
}
// ------------------------------------------------------------------------------------------------------------------
@@ -59,13 +47,8 @@ public String getImplementationName() {
@Override
public synchronized void refresh() {
- if (autoRefresh) {
- final Snapshot latestSnapshot = adapter.getCurrentSnapshot(tableIdentifier);
- if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) {
- locationKeyFinder.snapshot = latestSnapshot;
- }
- }
- refreshSnapshot();
+ // There should be no refresh service for this provider.
+ throw new UnsupportedOperationException();
}
@Override
@@ -95,7 +78,7 @@ public synchronized void update(final Snapshot snapshot) {
if (snapshot.sequenceNumber() <= locationKeyFinder.snapshot.sequenceNumber()) {
throw new IllegalArgumentException(
"Snapshot sequence number " + snapshot.sequenceNumber()
- + " is older than the current snapshot sequence number "
+ + " must be higher than the current snapshot sequence number "
+ locationKeyFinder.snapshot.sequenceNumber() + " for table " + tableIdentifier);
}
// Update the snapshot.
@@ -118,4 +101,24 @@ private void refreshSnapshot() {
endTransaction();
setInitialized();
}
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // SubscriptionAggregator implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ @Override
+ protected void activateUnderlyingDataSource() {
+ refreshSnapshot();
+ activationSuccessful(this);
+ }
+
+ @Override
+ protected void deactivateUnderlyingDataSource() {
+ // NOP for manually refreshing Iceberg table location provider.
+ }
+
+ @Override
+ protected boolean matchSubscriptionToken(T token) {
+ return token == this;
+ }
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
index 163952571a2..92798252dad 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
@@ -6,7 +6,6 @@
import io.deephaven.engine.table.impl.locations.*;
import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
-import io.deephaven.iceberg.util.IcebergCatalogAdapter;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.catalog.TableIdentifier;
import org.jetbrains.annotations.NotNull;
@@ -26,9 +25,8 @@ public IcebergStaticTableLocationProvider(
@NotNull final TK tableKey,
@NotNull final IcebergBaseLayout locationKeyFinder,
@NotNull final TableLocationFactory locationFactory,
- @NotNull final IcebergCatalogAdapter adapter,
@NotNull final TableIdentifier tableIdentifier) {
- super(tableKey, locationKeyFinder, locationFactory, null, adapter, tableIdentifier);
+ super(tableKey, locationKeyFinder, locationFactory, null, false, null, tableIdentifier);
}
// ------------------------------------------------------------------------------------------------------------------
@@ -62,4 +60,25 @@ public void update(long snapshotId) {
public void update(Snapshot snapshot) {
throw new IllegalStateException("A static table location provider cannot be updated");
}
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // SubscriptionAggregator implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ @Override
+ protected void activateUnderlyingDataSource() {
+ throw new IllegalStateException(
+ "activateUnderlyingDataSource() called on a static Iceberg table location provider");
+ }
+
+ @Override
+ protected void deactivateUnderlyingDataSource() {
+ throw new IllegalStateException(
+ "deactivateUnderlyingDataSource() called on a static Iceberg table location provider");
+ }
+
+ @Override
+ protected boolean matchSubscriptionToken(final T token) {
+ return false;
+ }
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
index 52c98718317..9e6577d500b 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
@@ -22,16 +22,15 @@ public abstract class IcebergTableLocationProviderBase locationFactory,
@Nullable final TableDataRefreshService refreshService,
- @NotNull final IcebergCatalogAdapter adapter,
+ final boolean isRefreshing,
+ @Nullable final IcebergCatalogAdapter adapter,
@NotNull final TableIdentifier tableIdentifier) {
- super(tableKey, refreshService != null);
+ super(tableKey, isRefreshing);
this.locationKeyFinder = locationKeyFinder;
this.locationFactory = locationFactory;
this.refreshService = refreshService;
@@ -39,10 +38,6 @@ public IcebergTableLocationProviderBase(
this.tableIdentifier = tableIdentifier;
}
- // ------------------------------------------------------------------------------------------------------------------
- // AbstractTableLocationProvider implementation
- // ------------------------------------------------------------------------------------------------------------------
-
/**
* Update the table location provider with the latest snapshot from the catalog.
*/
@@ -72,26 +67,4 @@ protected TableLocation makeTableLocation(@NotNull final TableLocationKey locati
// noinspection unchecked
return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService);
}
-
- // ------------------------------------------------------------------------------------------------------------------
- // SubscriptionAggregator implementation
- // ------------------------------------------------------------------------------------------------------------------
-
- @Override
- protected final void activateUnderlyingDataSource() {
- subscriptionToken = refreshService.scheduleTableLocationProviderRefresh(this);
- }
-
- @Override
- protected final void deactivateUnderlyingDataSource() {
- if (subscriptionToken != null) {
- subscriptionToken.cancel();
- subscriptionToken = null;
- }
- }
-
- @Override
- protected final boolean matchSubscriptionToken(final T token) {
- return token == subscriptionToken;
- }
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index bf23dbd6e85..e96cd2f8b43 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -11,7 +11,6 @@
import io.deephaven.engine.table.impl.locations.TableKey;
import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder;
import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey;
-import io.deephaven.engine.table.impl.locations.util.ExecutorTableDataRefreshService;
import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
import io.deephaven.engine.table.impl.sources.InMemoryColumnSource;
import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl;
@@ -370,6 +369,9 @@ Optional getSnapshot(@NotNull final TableIdentifier tableIdentifier, f
*/
public Snapshot getCurrentSnapshot(@NotNull final TableIdentifier tableIdentifier) {
final List snapshots = listSnapshots(tableIdentifier);
+ if (snapshots.isEmpty()) {
+ throw new IllegalStateException("No snapshots found for table " + tableIdentifier);
+ }
return snapshots.get(snapshots.size() - 1);
}
@@ -517,53 +519,44 @@ private IcebergTable readTableInternal(
userInstructions);
}
- if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.STATIC) {
+ if (instructions.updateMode() == IcebergInstructions.IcebergUpdateMode.STATIC) {
final IcebergTableLocationProviderBase locationProvider =
new IcebergStaticTableLocationProvider<>(
StandaloneTableKey.getInstance(),
keyFinder,
new IcebergTableLocationFactory(),
- this,
tableIdentifier);
- return new IcebergTableStatic(
+ return new IcebergTableImpl(
tableDef,
tableIdentifier.toString(),
RegionedTableComponentFactoryImpl.INSTANCE,
- locationProvider);
+ locationProvider,
+ null);
}
- final UpdateSourceRegistrar updateSourceRegistrar;
- final IcebergRefreshingTableLocationProvider locationProvider;
-
- if (instructions.refreshing() == IcebergInstructions.IcebergRefreshing.MANUAL_REFRESHING) {
- updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
+ final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
+ final IcebergTableLocationProviderBase locationProvider;
- locationProvider = new IcebergRefreshingTableLocationProvider<>(
+ if (instructions.updateMode() == IcebergInstructions.IcebergUpdateMode.MANUAL_REFRESHING) {
+ locationProvider = new IcebergManualRefreshTableLocationProvider<>(
StandaloneTableKey.getInstance(),
keyFinder,
new IcebergTableLocationFactory(),
- TableDataRefreshService.getSharedRefreshService(),
this,
- tableIdentifier,
- false);
+ tableIdentifier);
} else {
- updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
-
- final TableDataRefreshService refreshService =
- new ExecutorTableDataRefreshService("Local", instructions.autoRefreshMs(), 30_000L, 10);
-
- locationProvider = new IcebergRefreshingTableLocationProvider<>(
+ locationProvider = new IcebergAutoRefreshTableLocationProvider<>(
StandaloneTableKey.getInstance(),
keyFinder,
new IcebergTableLocationFactory(),
- refreshService,
+ TableDataRefreshService.getSharedRefreshService(),
+ instructions.autoRefreshMs(),
this,
- tableIdentifier,
- true);
+ tableIdentifier);
}
- return new IcebergTableRefreshing(
+ return new IcebergTableImpl(
tableDef,
tableIdentifier.toString(),
RegionedTableComponentFactoryImpl.INSTANCE,
@@ -583,13 +576,4 @@ private static KnownLocationKeyFinder toKnownKeys(
public Catalog catalog() {
return catalog;
}
-
- /**
- * Returns the underlying Iceberg {@link FileIO fileIO} used by this adapter.
- */
- @SuppressWarnings("unused")
- public FileIO fileIO() {
- return fileIO;
- }
-
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
index dc7515d7498..462ca838d2e 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
@@ -26,7 +26,7 @@ public abstract class IcebergInstructions {
public static final IcebergInstructions DEFAULT = builder().build();
@SuppressWarnings("unused")
- public enum IcebergRefreshing {
+ public enum IcebergUpdateMode {
STATIC, AUTO_REFRESHING, MANUAL_REFRESHING
}
@@ -52,16 +52,16 @@ public static Builder builder() {
public abstract Map columnRenames();
/**
- * The {@link IcebergRefreshing} mode to use when reading the Iceberg data files. Default is
- * {@link IcebergRefreshing#STATIC}.
+ * The {@link IcebergUpdateMode} mode to use when reading the Iceberg data files. Default is
+ * {@link IcebergUpdateMode#STATIC}.
*/
@Value.Default
- public IcebergRefreshing refreshing() {
- return IcebergRefreshing.STATIC;
+ public IcebergUpdateMode updateMode() {
+ return IcebergUpdateMode.STATIC;
}
/**
- * When {@link #refreshing()} is set to {@code IcebergRefreshing.AUTO_REFRESHING}, specifies the number of
+ * When {@link #updateMode()} is set to {@code IcebergUpdateMode.AUTO_REFRESHING}, specifies the number of
* milliseconds to wait before refreshing the Iceberg data files. Default is 60_000 milliseconds.
*/
@Value.Default
@@ -83,7 +83,12 @@ public interface Builder {
Builder putAllColumnRenames(Map entries);
@SuppressWarnings("unused")
- Builder refreshing(IcebergRefreshing refreshing);
+ Builder updateMode(IcebergUpdateMode refreshing);
+
+ @SuppressWarnings("unused")
+ default Builder updateMode(IcebergUpdateMode updateMode, long autoRefreshMs) {
+ return this.updateMode(updateMode).autoRefreshMs(autoRefreshMs);
+ }
@SuppressWarnings("unused")
Builder autoRefreshMs(long autoRefreshMs);
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java
similarity index 79%
rename from extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java
rename to extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java
index 72623b9e200..a73a55d40be 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableRefreshing.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java
@@ -15,24 +15,25 @@
import org.jetbrains.annotations.Nullable;
/**
- * Table wrapper for refreshing Iceberg tables.
+ * Table implementation for static and refreshing Iceberg tables.
*/
-public class IcebergTableRefreshing extends PartitionAwareSourceTable implements IcebergTable {
+public class IcebergTableImpl extends PartitionAwareSourceTable implements IcebergTable {
/**
* Location discovery.
*/
final IcebergTableLocationProviderBase locationProvider;
/**
+ * Create an instance of the class with the provided parameters.
*
- *
- * @param tableDefinition A TableDefinition
+ * @param tableDefinition The {@link TableDefinition} describing the table schema
* @param description A human-readable description for this table
* @param componentFactory A component factory for creating column source managers
- * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table
+ * @param locationProvider A {@link io.deephaven.engine.table.impl.locations.TableLocationProvider}, for use in
+ * discovering the locations that compose this table
* @param updateSourceRegistrar Callback for registering live tables for refreshes, null if this table is not live
*/
- IcebergTableRefreshing(
+ IcebergTableImpl(
@NotNull TableDefinition tableDefinition,
@NotNull String description,
@NotNull SourceTableComponentFactory componentFactory,
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java
deleted file mode 100644
index 0884a6379c5..00000000000
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableStatic.java
+++ /dev/null
@@ -1,49 +0,0 @@
-//
-// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
-//
-package io.deephaven.iceberg.util;
-
-import io.deephaven.engine.table.TableDefinition;
-import io.deephaven.engine.table.impl.PartitionAwareSourceTable;
-import io.deephaven.engine.table.impl.SourceTableComponentFactory;
-import io.deephaven.engine.table.impl.locations.TableKey;
-import io.deephaven.iceberg.layout.IcebergTableLocationProviderBase;
-import io.deephaven.iceberg.location.IcebergTableLocationKey;
-import org.apache.iceberg.Snapshot;
-import org.jetbrains.annotations.NotNull;
-
-/**
- * Table wrapper for refreshing Iceberg tables.
- */
-public class IcebergTableStatic extends PartitionAwareSourceTable implements IcebergTable {
- /**
- *
- *
- * @param tableDefinition A TableDefinition
- * @param description A human-readable description for this table
- * @param componentFactory A component factory for creating column source managers
- * @param locationProvider A TableLocationProvider, for use in discovering the locations that compose this table
- */
- IcebergTableStatic(
- @NotNull TableDefinition tableDefinition,
- @NotNull String description,
- @NotNull SourceTableComponentFactory componentFactory,
- @NotNull IcebergTableLocationProviderBase locationProvider) {
- super(tableDefinition, description, componentFactory, locationProvider, null);
- }
-
- @Override
- public void update() {
- throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description);
- }
-
- @Override
- public void update(final long snapshotId) {
- throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description);
- }
-
- @Override
- public void update(final @NotNull Snapshot snapshot) {
- throw new IllegalStateException("Static Iceberg tables cannot be updated, table: " + description);
- }
-}
From 3da205ccee7d2e8a2004e1ef05d9fffcbf6fd3fb Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Sat, 27 Jul 2024 13:25:37 -0700
Subject: [PATCH 08/56] Nearly all PR comments addressed.
---
.../impl/locations/TableLocationProvider.java | 12 ++++++-
.../impl/TableLocationSubscriptionBuffer.java | 26 +++++++-------
.../regioned/RegionedColumnSourceManager.java | 35 +++++++++++--------
...ebergAutoRefreshTableLocationProvider.java | 6 ++--
4 files changed, 48 insertions(+), 31 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
index 9ba7a350d81..eab5f6ad72f 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
@@ -56,10 +56,14 @@ default void endTransaction() {
}
/**
+ *
* Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location
* subscription. This should occur at most once per location, but the order of delivery is not
* guaranteed.
- *
+ *
+ *
+ * If transactionToken is {@code null}, the key will be added to the pending additions immediately.
+ *
* @param tableLocationKey The new table location key.
* @param transactionToken The token identifying the transaction.
*/
@@ -79,7 +83,12 @@ default void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tabl
}
/**
+ *
* Notify the listener of a {@link TableLocationKey} that has been removed.
+ *
+ *
+ * If transactionToken is {@code null}, the key will be added to the pending removals immediately.
+ *
*
* @param tableLocationKey The table location key that was removed.
* @param transactionToken The token identifying the transaction.
@@ -93,6 +102,7 @@ void handleTableLocationKeyRemoved(
*
* @param tableLocationKey The table location key that was removed.
*/
+ @SuppressWarnings("unused")
default void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey) {
handleTableLocationKeyRemoved(tableLocationKey, this);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
index b9a455ce7af..1962ae39ce1 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
@@ -3,6 +3,7 @@
//
package io.deephaven.engine.table.impl.locations.impl;
+import io.deephaven.base.verify.Assert;
import io.deephaven.base.verify.Require;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableDataException;
@@ -125,11 +126,13 @@ public synchronized void reset() {
@Override
public void beginTransaction(@NotNull final Object token) {
synchronized (updateLock) {
- // Assert that we can start a new transaction with this token.
- Require.eqFalse(transactionTokens.contains(token), "transactionTokens.contains(token)");
- Require.eqFalse(accumulatedLocationsAdded.containsKey(token),
+ // Verify that we can start a new transaction with this token.
+ if (transactionTokens.contains(token)) {
+ throw new IllegalStateException("A transaction with token " + token + " is currently open.");
+ }
+ Assert.eqFalse(accumulatedLocationsAdded.containsKey(token),
"accumulatedLocationsAdded.containsKey(token)");
- Require.eqFalse(accumulatedLocationsRemoved.containsKey(token),
+ Assert.eqFalse(accumulatedLocationsRemoved.containsKey(token),
"accumulatedLocationsRemoved.containsKey(token)");
transactionTokens.add(token);
@@ -141,11 +144,13 @@ public void beginTransaction(@NotNull final Object token) {
@Override
public void endTransaction(@NotNull final Object token) {
synchronized (updateLock) {
- // Assert that this transaction is open.
- Require.eqTrue(transactionTokens.contains(token), "transactionTokens.contains(token)");
+ // Verify that this transaction is open.
+ if (!transactionTokens.remove(token)) {
+ throw new IllegalStateException("No transaction with token " + token + " is currently open.");
+ }
- final Set tokenLocationsAdded = accumulatedLocationsAdded.get(token);
- final Set tokenLocationsRemoved = accumulatedLocationsRemoved.get(token);
+ final Set tokenLocationsAdded = accumulatedLocationsAdded.remove(token);
+ final Set tokenLocationsRemoved = accumulatedLocationsRemoved.remove(token);
if (tokenLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) {
for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsRemoved) {
@@ -181,11 +186,6 @@ public void endTransaction(@NotNull final Object token) {
pendingLocationsAdded.add(tableLocationKey);
}
}
-
- // Clear all the storage for this transaction.
- transactionTokens.remove(token);
- accumulatedLocationsAdded.remove(token);
- accumulatedLocationsRemoved.remove(token);
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
index 0bcb4e66724..5743c08f4b6 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
@@ -121,6 +121,9 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col
@ReferentialIntegrity
private Runnable delayedErrorReference;
+ final List invalidatedLocations;
+ final UpdateCommitter> invalidateCommitter;
+
/**
* Construct a column manager with the specified component factory and definitions.
*
@@ -179,6 +182,14 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col
rowSetModifiedColumnSet = null;
}
}
+
+ invalidatedLocations = new ArrayList<>();
+ invalidateCommitter = new UpdateCommitter<>(this,
+ ExecutionContext.getContext().getUpdateGraph(),
+ (ignored) -> {
+ invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate);
+ invalidatedLocations.clear();
+ });
}
@Override
@@ -223,11 +234,9 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati
orderedIncludedTableLocations.remove(includedLocation);
removedTableLocations.add(includedLocation);
- final UpdateCommitter> committer = new UpdateCommitter<>(this,
- ExecutionContext.getContext().getUpdateGraph(),
- (ignored) -> includedLocation.invalidate());
-
- committer.maybeActivate();
+ // Mark this location for invalidation.
+ invalidatedLocations.add(includedLocation);
+ invalidateCommitter.maybeActivate();
return true;
}
@@ -356,13 +365,13 @@ private TableUpdateImpl update(final boolean initializing) {
final RowSetBuilderSequential addedRegionBuilder = RowSetFactory.builderSequential();
- final int previousNumRegions = includedTableLocations.size();
- final int newNumRegions = previousNumRegions + (entriesToInclude == null ? 0 : entriesToInclude.size());
+ final int prevMaxIndex = nextRegionIndex.get();
+ final int maxIndex = nextRegionIndex.get() + (entriesToInclude == null ? 0 : entriesToInclude.size());
if (entriesToInclude != null) {
partitioningColumnValueSources.values().forEach(
- (final WritableColumnSource> wcs) -> wcs.ensureCapacity(newNumRegions));
- locationSource.ensureCapacity(newNumRegions);
- rowSetSource.ensureCapacity(newNumRegions);
+ (final WritableColumnSource> wcs) -> wcs.ensureCapacity(maxIndex));
+ locationSource.ensureCapacity(maxIndex);
+ rowSetSource.ensureCapacity(maxIndex);
for (final EmptyTableLocationEntry entryToInclude : entriesToInclude) {
final IncludedTableLocationEntry entry = new IncludedTableLocationEntry(entryToInclude);
@@ -389,7 +398,7 @@ private TableUpdateImpl update(final boolean initializing) {
}
if (initializing) {
- Assert.eqZero(previousNumRegions, "previousNumRegions");
+ Assert.eqZero(prevMaxIndex, "previousNumRegions");
if (isRefreshing) {
rowSetSource.startTrackingPrevValues();
includedLocationsTable.getRowSet().writableCast().initializePreviousValue();
@@ -401,9 +410,7 @@ private TableUpdateImpl update(final boolean initializing) {
final RowSet modifiedRegions = modifiedRegionBuilder.build();
final RowSet removedRegions = removedRegionBuilder.build();
if (addedRegions.isEmpty() && modifiedRegions.isEmpty() && removedRegions.isEmpty()) {
- addedRegions.close();
- modifiedRegions.close();
- removedRegions.close();
+ SafeCloseable.closeAll(addedRegions, modifiedRegions, removedRegions);
} else {
includedLocationsTable.getRowSet().writableCast().remove(removedRegions);
final TableUpdate update = new TableUpdateImpl(
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
index bc9fb230eda..ad7212e1c3f 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
@@ -66,17 +66,17 @@ public synchronized void refresh() {
@Override
public void update() {
- throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated manually");
+ throw new IllegalStateException("An automatically refreshing Iceberg table cannot be manually updated");
}
@Override
public void update(long snapshotId) {
- throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated");
+ throw new IllegalStateException("An automatically refreshing Iceberg table cannot be manually updated");
}
@Override
public void update(Snapshot snapshot) {
- throw new IllegalStateException("An automatically refreshing Iceberg table cannot be updated");
+ throw new IllegalStateException("An automatically refreshing Iceberg table cannot be manually updated");
}
/**
From 08dd329dbfbb2fb5b7b3c307fa21c0397ab12040 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Mon, 29 Jul 2024 12:07:50 -0700
Subject: [PATCH 09/56] Adjustment to IcebergInstructions update mode.
---
.../impl/locations/TableLocationProvider.java | 1 +
...ergManualRefreshTableLocationProvider.java | 6 +-
.../iceberg/util/IcebergCatalogAdapter.java | 7 ++-
.../iceberg/util/IcebergInstructions.java | 22 --------
.../iceberg/util/IcebergUpdateMode.java | 56 +++++++++++++++++++
5 files changed, 64 insertions(+), 28 deletions(-)
create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
index eab5f6ad72f..d6158090289 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
@@ -64,6 +64,7 @@ default void endTransaction() {
*
* If transactionToken is {@code null}, the key will be added to the pending additions immediately.
*
+ *
* @param tableLocationKey The new table location key.
* @param transactionToken The token identifying the transaction.
*/
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
index 9287cfa64dc..bfa5e99e60d 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
@@ -77,9 +77,9 @@ public synchronized void update(final Snapshot snapshot) {
// Verify that the input snapshot is newer (higher in sequence number) than the current snapshot.
if (snapshot.sequenceNumber() <= locationKeyFinder.snapshot.sequenceNumber()) {
throw new IllegalArgumentException(
- "Snapshot sequence number " + snapshot.sequenceNumber()
- + " must be higher than the current snapshot sequence number "
- + locationKeyFinder.snapshot.sequenceNumber() + " for table " + tableIdentifier);
+ "Update snapshot sequence number (" + snapshot.sequenceNumber()
+ + ") must be higher than the current snapshot sequence number ("
+ + locationKeyFinder.snapshot.sequenceNumber() + ") for table " + tableIdentifier);
}
// Update the snapshot.
locationKeyFinder.snapshot = snapshot;
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index 75b602813e4..8ba56e5cc70 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -550,7 +550,8 @@ private IcebergTable readTableInternal(
userInstructions);
}
- if (instructions.updateMode() == IcebergInstructions.IcebergUpdateMode.STATIC) {
+ if (instructions == null
+ || instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) {
final IcebergTableLocationProviderBase locationProvider =
new IcebergStaticTableLocationProvider<>(
StandaloneTableKey.getInstance(),
@@ -569,7 +570,7 @@ private IcebergTable readTableInternal(
final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
final IcebergTableLocationProviderBase locationProvider;
- if (instructions.updateMode() == IcebergInstructions.IcebergUpdateMode.MANUAL_REFRESHING) {
+ if (instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) {
locationProvider = new IcebergManualRefreshTableLocationProvider<>(
StandaloneTableKey.getInstance(),
keyFinder,
@@ -582,7 +583,7 @@ private IcebergTable readTableInternal(
keyFinder,
new IcebergTableLocationFactory(),
TableDataRefreshService.getSharedRefreshService(),
- instructions.autoRefreshMs(),
+ instructions.updateMode().autoRefreshMs(),
this,
tableIdentifier);
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
index 462ca838d2e..5b6438475b2 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
@@ -25,11 +25,6 @@ public abstract class IcebergInstructions {
@SuppressWarnings("unused")
public static final IcebergInstructions DEFAULT = builder().build();
- @SuppressWarnings("unused")
- public enum IcebergUpdateMode {
- STATIC, AUTO_REFRESHING, MANUAL_REFRESHING
- }
-
public static Builder builder() {
return ImmutableIcebergInstructions.builder();
}
@@ -60,15 +55,6 @@ public IcebergUpdateMode updateMode() {
return IcebergUpdateMode.STATIC;
}
- /**
- * When {@link #updateMode()} is set to {@code IcebergUpdateMode.AUTO_REFRESHING}, specifies the number of
- * milliseconds to wait before refreshing the Iceberg data files. Default is 60_000 milliseconds.
- */
- @Value.Default
- public long autoRefreshMs() {
- return 60_000L; // 60 second default
- }
-
public interface Builder {
@SuppressWarnings("unused")
Builder tableDefinition(TableDefinition tableDefinition);
@@ -85,14 +71,6 @@ public interface Builder {
@SuppressWarnings("unused")
Builder updateMode(IcebergUpdateMode refreshing);
- @SuppressWarnings("unused")
- default Builder updateMode(IcebergUpdateMode updateMode, long autoRefreshMs) {
- return this.updateMode(updateMode).autoRefreshMs(autoRefreshMs);
- }
-
- @SuppressWarnings("unused")
- Builder autoRefreshMs(long autoRefreshMs);
-
IcebergInstructions build();
}
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java
new file mode 100644
index 00000000000..32f67c742ec
--- /dev/null
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java
@@ -0,0 +1,56 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.iceberg.util;
+
+import io.deephaven.annotations.BuildableStyle;
+import org.immutables.value.Value;
+
+@Value.Immutable
+@BuildableStyle
+public abstract class IcebergUpdateMode {
+ public static final IcebergUpdateMode STATIC = builder()
+ .updateType(IcebergUpdateType.STATIC).build();
+ @SuppressWarnings("unused")
+ public static final IcebergUpdateMode AUTO_REFRESHING = builder()
+ .updateType(IcebergUpdateType.AUTO_REFRESHING).build();
+ @SuppressWarnings("unused")
+ public static final IcebergUpdateMode MANUAL_REFRESHING = builder()
+ .updateType(IcebergUpdateType.MANUAL_REFRESHING).build();
+
+ public enum IcebergUpdateType {
+ STATIC, AUTO_REFRESHING, MANUAL_REFRESHING
+ }
+
+ public static Builder builder() {
+ return ImmutableIcebergUpdateMode.builder();
+ }
+
+ @SuppressWarnings("unused")
+ public static IcebergUpdateMode autoRefreshing(final long refreshMs) {
+ return ImmutableIcebergUpdateMode.builder()
+ .updateType(IcebergUpdateType.AUTO_REFRESHING)
+ .autoRefreshMs(refreshMs)
+ .build();
+ }
+
+ @Value.Default
+ public IcebergUpdateType updateType() {
+ return IcebergUpdateType.STATIC;
+ }
+
+ @Value.Default
+ public long autoRefreshMs() {
+ return 60_000L;
+ }
+
+ public interface Builder {
+ @SuppressWarnings("unused")
+ Builder updateType(IcebergUpdateType updateType);
+
+ @SuppressWarnings("unused")
+ Builder autoRefreshMs(long autoRefreshMs);
+
+ IcebergUpdateMode build();
+ }
+}
From 7af0d1da546a6c10ee2bb3c6f22b1910c5bb2fad Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Mon, 29 Jul 2024 14:40:20 -0700
Subject: [PATCH 10/56] Added python wrapper for Iceberg refreshing tables.
---
.../iceberg/util/IcebergUpdateMode.java | 15 ++--
py/server/deephaven/experimental/iceberg.py | 80 ++++++++++++++++++-
2 files changed, 84 insertions(+), 11 deletions(-)
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java
index 32f67c742ec..246e81ac65e 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java
@@ -9,14 +9,15 @@
@Value.Immutable
@BuildableStyle
public abstract class IcebergUpdateMode {
- public static final IcebergUpdateMode STATIC = builder()
- .updateType(IcebergUpdateType.STATIC).build();
+ private static final long REFRESH_INTERVAL_MS = 60_000L;
+
+ public static final IcebergUpdateMode STATIC = builder().updateType(IcebergUpdateType.STATIC).build();
@SuppressWarnings("unused")
- public static final IcebergUpdateMode AUTO_REFRESHING = builder()
- .updateType(IcebergUpdateType.AUTO_REFRESHING).build();
+ public static final IcebergUpdateMode AUTO_REFRESHING =
+ builder().updateType(IcebergUpdateType.AUTO_REFRESHING).build();
@SuppressWarnings("unused")
- public static final IcebergUpdateMode MANUAL_REFRESHING = builder()
- .updateType(IcebergUpdateType.MANUAL_REFRESHING).build();
+ public static final IcebergUpdateMode MANUAL_REFRESHING =
+ builder().updateType(IcebergUpdateType.MANUAL_REFRESHING).build();
public enum IcebergUpdateType {
STATIC, AUTO_REFRESHING, MANUAL_REFRESHING
@@ -41,7 +42,7 @@ public IcebergUpdateType updateType() {
@Value.Default
public long autoRefreshMs() {
- return 60_000L;
+ return REFRESH_INTERVAL_MS;
}
public interface Builder {
diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py
index 7506bc95a25..42e2c8292bf 100644
--- a/py/server/deephaven/experimental/iceberg.py
+++ b/py/server/deephaven/experimental/iceberg.py
@@ -2,6 +2,7 @@
# Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
#
""" This module adds Iceberg table support into Deephaven. """
+from __future__ import annotations
from typing import List, Optional, Union, Dict, Sequence
import jpy
@@ -17,7 +18,9 @@
from deephaven.table import Table
_JIcebergInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergInstructions")
+_JIcebergUpdateMode = jpy.get_type("io.deephaven.iceberg.util.IcebergUpdateMode")
_JIcebergCatalogAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergCatalogAdapter")
+_JIcebergTable = jpy.get_type("io.deephaven.iceberg.util.IcebergTable")
# IcebergToolsS3 is an optional library
try:
@@ -30,6 +33,39 @@
_JSnapshot = jpy.get_type("org.apache.iceberg.Snapshot")
+class IcebergUpdateMode(JObjectWrapper):
+ """
+ This class specifies the update mode for the Iceberg table to be loaded into Deephaven. The modes are:
+
+ - `STATIC`: The table is loaded once and does not change
+ - `MANUAL_REFRESHING`: The table can be manually refreshed by the user.
+ - `AUTO_REFRESHING`: The table will be automatically refreshed at a specified interval (use
+ `auto_refreshing(auto_refresh_ms: int)` to specify an interval rather than use the system default
+ of 60 seconds).
+ """
+ j_object_type = _JIcebergUpdateMode
+
+ def __init__(self, mode: _JIcebergUpdateMode):
+ self._j_object = mode
+
+ @classmethod
+ def auto_refreshing(cls, auto_refresh_ms: int) -> IcebergUpdateMode:
+ """
+ Creates an IcebergUpdateMode with auto-refreshing mode enabled using the provided refresh interval.
+
+ :param auto_refresh_ms (int): the refresh interval in milliseconds.
+ """
+ return IcebergUpdateMode(_JIcebergUpdateMode.autoRefreshing(auto_refresh_ms))
+
+ @property
+ def j_object(self) -> jpy.JType:
+ return self._j_object
+
+IcebergUpdateMode.STATIC=IcebergUpdateMode(_JIcebergUpdateMode.STATIC)
+IcebergUpdateMode.MANUAL_REFRESHING=IcebergUpdateMode(_JIcebergUpdateMode.MANUAL_REFRESHING)
+IcebergUpdateMode.AUTO_REFRESHING=IcebergUpdateMode(_JIcebergUpdateMode.AUTO_REFRESHING)
+
+
class IcebergInstructions(JObjectWrapper):
"""
This class specifies the instructions for reading an Iceberg table into Deephaven. These include column rename
@@ -41,7 +77,8 @@ class IcebergInstructions(JObjectWrapper):
def __init__(self,
table_definition: Optional[Union[Dict[str, DType], List[Column]]] = None,
data_instructions: Optional[s3.S3Instructions] = None,
- column_renames: Optional[Dict[str, str]] = None):
+ column_renames: Optional[Dict[str, str]] = None,
+ update_mode: Optional[IcebergUpdateMode] = None):
"""
Initializes the instructions using the provided parameters.
@@ -53,6 +90,8 @@ def __init__(self,
reading files from a non-local file system, like S3.
column_renames (Optional[Dict[str, str]]): A dictionary of old to new column names that will be renamed in
the output table.
+ update_mode (Optional[IcebergUpdateMode]): The update mode for the table. If omitted, the default update
+ mode of `IcebergUpdateMode.STATIC` is used.
Raises:
DHError: If unable to build the instructions object.
@@ -71,6 +110,9 @@ def __init__(self,
for old_name, new_name in column_renames.items():
builder.putColumnRenames(old_name, new_name)
+ if update_mode is not None:
+ builder.updateMode(update_mode.j_object)
+
self._j_object = builder.build()
except Exception as e:
raise DHError(e, "Failed to build Iceberg instructions") from e
@@ -80,6 +122,36 @@ def j_object(self) -> jpy.JType:
return self._j_object
+class IcebergTable(Table):
+ """
+ IcebergTable is a subclass of Table that allows the users to dynamically update the table with new snapshots from
+ the Iceberg catalog.
+ """
+ j_object_type = _JIcebergTable
+
+ def __init__(self, j_table: jpy.JType):
+ super().__init__(j_table)
+
+ def update(self, snapshot_id:Optional[int] = None):
+ """
+ Updates the table with a specific snapshot. If no snapshot is provided, the most recent snapshot is used.
+
+ NOTE: this method is only valid when the table is in `MANUAL_REFRESHING` mode. `STATIC` and `AUTO_REFRESHING`
+ Iceberg tables cannot be updated manually and will throw an exception if this method is called.
+
+ Args:
+ snapshot_id (Optional[int]): the snapshot id to update to; if omitted the most recent snapshot will be used.
+ """
+ if snapshot_id is not None:
+ self.j_object.update(snapshot_id)
+ return
+ self.j_object.update()
+
+ @property
+ def j_object(self) -> jpy.JType:
+ return self.j_table
+
+
class IcebergCatalogAdapter(JObjectWrapper):
"""
This class provides an interface for interacting with Iceberg catalogs. It allows listing namespaces, tables and
@@ -135,7 +207,7 @@ def snapshots(self, table_identifier: str) -> Table:
return self.j_object.listSnapshotsAsTable(table_identifier)
- def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> Table:
+ def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> IcebergTable:
"""
Reads the table from the catalog using the provided instructions. Optionally, a snapshot id can be provided to
read a specific snapshot of the table.
@@ -157,8 +229,8 @@ def read_table(self, table_identifier: str, instructions: Optional[IcebergInstru
instructions_object = _JIcebergInstructions.DEFAULT
if snapshot_id is not None:
- return Table(self.j_object.readTable(table_identifier, snapshot_id, instructions_object))
- return Table(self.j_object.readTable(table_identifier, instructions_object))
+ return IcebergTable(self.j_object.readTable(table_identifier, snapshot_id, instructions_object))
+ return IcebergTable(self.j_object.readTable(table_identifier, instructions_object))
@property
def j_object(self) -> jpy.JType:
From 2d79c38b93a85fd96fdd4654d9eb2ad72bb2202d Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Mon, 29 Jul 2024 15:20:15 -0700
Subject: [PATCH 11/56] Changes to mocked tests for ColumnSourceManager and
PartitionAwareSourceTable.
---
.../impl/TestPartitionAwareSourceTable.java | 16 ++++++++++------
.../TestRegionedColumnSourceManager.java | 14 +++++++-------
2 files changed, 17 insertions(+), 13 deletions(-)
diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java
index db0142c1c9b..e5933d866bb 100644
--- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java
+++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java
@@ -6,9 +6,7 @@
import io.deephaven.base.Pair;
import io.deephaven.base.verify.Assert;
import io.deephaven.engine.context.ExecutionContext;
-import io.deephaven.engine.rowset.WritableRowSet;
-import io.deephaven.engine.rowset.RowSet;
-import io.deephaven.engine.rowset.RowSetFactory;
+import io.deephaven.engine.rowset.*;
import io.deephaven.engine.table.*;
import io.deephaven.engine.table.impl.perf.PerformanceEntry;
import io.deephaven.engine.table.vectors.ColumnVectors;
@@ -24,7 +22,6 @@
import io.deephaven.chunk.ChunkType;
import io.deephaven.chunk.WritableChunk;
import io.deephaven.chunk.WritableIntChunk;
-import io.deephaven.engine.rowset.RowSequence;
import io.deephaven.qst.column.Column;
import io.deephaven.util.type.ArrayTypeUtils;
import io.deephaven.vector.ObjectVector;
@@ -250,7 +247,9 @@ private void doInitializeCheck(final ImmutableTableLocationKey[] tableLocationKe
@Override
public Object invoke(Invocation invocation) {
subscriptionBuffer = (TableLocationSubscriptionBuffer) invocation.getParameter(0);
+ subscriptionBuffer.beginTransaction();
Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded);
+ subscriptionBuffer.endTransaction();
return null;
}
});
@@ -320,7 +319,8 @@ private void doRefreshChangedCheck() {
checking(new Expectations() {
{
oneOf(columnSourceManager).refresh();
- will(returnValue(toAdd.copy()));
+ will(returnValue(new TableUpdateImpl(toAdd.copy(), RowSetFactory.empty(), RowSetFactory.empty(),
+ RowSetShiftData.EMPTY, ModifiedColumnSet.ALL)));
checking(new Expectations() {
{
oneOf(listener).getNotification(with(any(TableUpdateImpl.class)));
@@ -360,7 +360,9 @@ private void doRefreshUnchangedCheck() {
checking(new Expectations() {
{
oneOf(columnSourceManager).refresh();
- will(returnValue(RowSetFactory.empty()));
+ will(returnValue(
+ new TableUpdateImpl(RowSetFactory.empty(), RowSetFactory.empty(), RowSetFactory.empty(),
+ RowSetShiftData.EMPTY, ModifiedColumnSet.ALL)));
}
});
@@ -407,7 +409,9 @@ public Object invoke(Invocation invocation) {
private void doAddLocationsRefreshCheck(final ImmutableTableLocationKey[] tableLocationKeys,
final Set expectPassFilters) {
+ subscriptionBuffer.beginTransaction();
Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded);
+ subscriptionBuffer.endTransaction();
expectPassFilters.forEach(tl -> checking(new Expectations() {
{
diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java
index 55ffd9960b5..ff44314501a 100644
--- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java
+++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java
@@ -525,19 +525,19 @@ public void testRefreshing() {
// Refresh them
setSizeExpectations(true, true, 5, 1000);
- updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh()));
+ updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added()));
checkIndexes();
assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations());
// Refresh them with no change
setSizeExpectations(true, true, 5, 1000);
- updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh()));
+ updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added()));
checkIndexes();
assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations());
// Refresh them with a change for the subscription-supporting one
setSizeExpectations(true, true, 5, 1001);
- updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh()));
+ updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added()));
checkIndexes();
assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations());
@@ -568,26 +568,26 @@ public void testRefreshing() {
// Test run with new locations included
setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE);
- updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh()));
+ updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added()));
checkIndexes();
assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B), SUT.includedLocations());
// Test no-op run
setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE);
- updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh()));
+ updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added()));
checkIndexes();
assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B), SUT.includedLocations());
// Test run with a location updated from null to not
setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 2);
- updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh()));
+ updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added()));
checkIndexes();
assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B),
SUT.includedLocations());
// Test run with a location updated
setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 10000002);
- updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh()));
+ updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh().added()));
checkIndexes();
assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B),
SUT.includedLocations());
From 3809f212672266b8e98b663709f590c2235304a9 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Tue, 30 Jul 2024 10:11:29 -0700
Subject: [PATCH 12/56] Added DHError handler and add'l documentation to python
`snapshots()` call.
---
.../iceberg/util/IcebergCatalogAdapter.java | 11 +++++++--
py/server/deephaven/experimental/iceberg.py | 23 +++++++++++++++----
2 files changed, 27 insertions(+), 7 deletions(-)
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index 8ba56e5cc70..e57b984e05f 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -291,8 +291,15 @@ public List listSnapshots(@NotNull final TableIdentifier tableIdentifi
/**
* List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting
- * table will be static and contain the same information as {@link #listSnapshots(TableIdentifier)}.
- *
+ * table will be static and contain contains the following columns:
+ *
+ *
Id: the snapshot identifier (can be used for updating the table or loading a specific snapshot).
+ *
TimestampMs: the timestamp of the snapshot.
+ *
Operation: the data operation that created this snapshot.
+ *
Summary: additional information about the snapshot from the Iceberg metadata.
+ *
SnapshotObject: a Java object containing the Iceberg API snapshot.
+ *
+ *
* @param tableIdentifier The identifier of the table from which to gather snapshots.
* @return A list of all tables in the given namespace.
*/
diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py
index 42e2c8292bf..9a3b1a9a1ce 100644
--- a/py/server/deephaven/experimental/iceberg.py
+++ b/py/server/deephaven/experimental/iceberg.py
@@ -141,11 +141,18 @@ def update(self, snapshot_id:Optional[int] = None):
Args:
snapshot_id (Optional[int]): the snapshot id to update to; if omitted the most recent snapshot will be used.
+
+ Raises:
+ DHError: If unable to update the Iceberg table.
+
"""
- if snapshot_id is not None:
- self.j_object.update(snapshot_id)
- return
- self.j_object.update()
+ try:
+ if snapshot_id is not None:
+ self.j_object.update(snapshot_id)
+ return
+ self.j_object.update()
+ except Exception as e:
+ raise DHError(e, "Failed to update Iceberg table") from e
@property
def j_object(self) -> jpy.JType:
@@ -196,7 +203,13 @@ def tables(self, namespace: str) -> Table:
def snapshots(self, table_identifier: str) -> Table:
"""
- Returns information on the snapshots of the specified table as a Deephaven table.
+ Returns information on the snapshots of the specified table as a Deephaven table. The table contains the
+ following columns:
+ - `Id`: the snapshot identifier (can be used for updating the table or loading a specific snapshot).
+ - `TimestampMs`: the timestamp of the snapshot.
+ - `Operation`: the data operation that created this snapshot.
+ - `Summary`: additional information about this snapshot from the Iceberg metadata.
+ - `SnapshotObject`: a Java object containing the Iceberg API snapshot.
Args:
table_identifier (str): the table from which to list snapshots.
From 5273a15e4847f262fcb972ac04cc80e5a57cc184 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Tue, 30 Jul 2024 10:18:25 -0700
Subject: [PATCH 13/56] Fixed typo in JavaDoc
---
.../java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index e57b984e05f..c7458288165 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -220,6 +220,7 @@ public Table listNamespacesAsTable(@NotNull final Namespace namespace) {
*
* @return A {@link Table table} of all namespaces.
*/
+ @SuppressWarnings("unused")
public Table listNamespacesAsTable(@NotNull final String... namespace) {
return listNamespacesAsTable(Namespace.of(namespace));
}
@@ -291,7 +292,7 @@ public List listSnapshots(@NotNull final TableIdentifier tableIdentifi
/**
* List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting
- * table will be static and contain contains the following columns:
+ * table will be static and contain the following columns:
*
*
Id: the snapshot identifier (can be used for updating the table or loading a specific snapshot).
*
TimestampMs: the timestamp of the snapshot.
From b9e2c6e3ba9f43e2ec69abe9176433503c87d054 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Wed, 31 Jul 2024 11:15:26 -0700
Subject: [PATCH 14/56] WIP
---
.../impl/locations/TableLocationProvider.java | 4 +--
.../impl/AbstractTableLocationProvider.java | 28 ++-----------------
2 files changed, 4 insertions(+), 28 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
index d6158090289..d23b057ec13 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
@@ -80,7 +80,7 @@ void handleTableLocationKeyAdded(
* @param tableLocationKey The new table location key.
*/
default void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey) {
- handleTableLocationKeyAdded(tableLocationKey, this);
+ handleTableLocationKeyAdded(tableLocationKey, null);
}
/**
@@ -105,7 +105,7 @@ void handleTableLocationKeyRemoved(
*/
@SuppressWarnings("unused")
default void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey) {
- handleTableLocationKeyRemoved(tableLocationKey, this);
+ handleTableLocationKeyRemoved(tableLocationKey, null);
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index 1a3f7ead2da..c9d8c7c0087 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -16,7 +16,7 @@
* Partial {@link TableLocationProvider} implementation for standalone use or as part of a {@link TableDataService}.
*
* Presents an interface similar to {@link TableLocationProvider.Listener} for subclasses to use when communicating with
- * the parent; see {@link #handleTableLocationKeyAdded(TableLocationKey)}.
+ * the parent; see {@link #handleTableLocationKeyAdded(TableLocationKey, Object).
*
* Note that subclasses are responsible for determining when it's appropriate to call {@link #setInitialized()} and/or
* override {@link #doInitialization()}.
@@ -84,20 +84,7 @@ public final ImmutableTableKey getKey() {
@Override
protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) {
- listener.beginTransaction();
unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded);
- listener.endTransaction();
- }
-
- /**
- * Deliver a possibly-new key. This method passes {@code this} as the transaction token.
- *
- * @param locationKey The new key
- * @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools.
- */
- protected final void handleTableLocationKeyAdded(
- @NotNull final TableLocationKey locationKey) {
- handleTableLocationKeyAdded(locationKey, this);
}
/**
@@ -176,7 +163,7 @@ protected final void endTransaction(@NotNull final Object token) {
}
/**
- * Called after a table location has been visited by {@link #handleTableLocationKeyAdded(TableLocationKey)},
+ * Called after a table location has been visited by {@link #handleTableLocationKeyAdded(TableLocationKey, Object)},
* but before notifications have been delivered to any subscriptions, if applicable. The default implementation does
* nothing, and may be overridden to implement additional features.
*
@@ -314,17 +301,6 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey)
}
}
- /**
- * Notify subscribers that {@code locationKey} was removed. This method passes {@code this} as the transaction
- * token.
- *
- * @param locationKey the TableLocation that was removed
- */
- protected final void handleTableLocationKeyRemoved(
- @NotNull final ImmutableTableLocationKey locationKey) {
- handleTableLocationKeyRemoved(locationKey, this);
- }
-
/**
* Notify subscribers that {@code locationKey} was removed.
*
From 9937f79627ba626febf8b97454015922b66938d5 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Wed, 31 Jul 2024 12:37:19 -0700
Subject: [PATCH 15/56] Suggestion from review
---
.../impl/AbstractTableLocationProvider.java | 44 +++++++++++--------
1 file changed, 26 insertions(+), 18 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index c9d8c7c0087..d996e35d053 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -163,9 +163,10 @@ protected final void endTransaction(@NotNull final Object token) {
}
/**
- * Called after a table location has been visited by {@link #handleTableLocationKeyAdded(TableLocationKey, Object)},
- * but before notifications have been delivered to any subscriptions, if applicable. The default implementation does
- * nothing, and may be overridden to implement additional features.
+ * Called after a table location has been visited by
+ * {@link #handleTableLocationKeyAdded(TableLocationKey, Object)}, but before notifications have been delivered to
+ * any subscriptions, if applicable. The default implementation does nothing, and may be overridden to implement
+ * additional features.
*
* @param locationKey The {@link TableLocationKey} that was visited.
*/
@@ -289,32 +290,31 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t
* @param locationKey The {@link TableLocationKey} to remove
*/
public void removeTableLocationKey(@NotNull final TableLocationKey locationKey) {
- final Object removedLocation = tableLocations.remove(locationKey);
-
- if (removedLocation != null) {
- handleTableLocationKeyRemoved(locationKey.makeImmutable());
- if (removedLocation instanceof AbstractTableLocation) {
- final AbstractTableLocation abstractLocation = (AbstractTableLocation) removedLocation;
- abstractLocation.handleUpdate(null, System.currentTimeMillis());
- abstractLocation.clearColumnLocations();
- }
- }
+ handleTableLocationKeyRemoved(locationKey, null);
}
/**
- * Notify subscribers that {@code locationKey} was removed.
+ * Handle a removal, optionally as part of a transaction. Notify subscribers that {@code locationKey} was removed if
+ * necessary. See {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics.
*
* @param locationKey the TableLocation that was removed
* @param transactionToken The token identifying the transaction
*/
protected void handleTableLocationKeyRemoved(
- @NotNull final ImmutableTableLocationKey locationKey,
+ @NotNull final TableLocationKey locationKey,
@Nullable final Object transactionToken) {
- if (supportsSubscriptions()) {
- synchronized (subscriptions) {
+ if (!supportsSubscriptions()) {
+ maybeClearLocationForRemoval(tableLocations.remove(locationKey));
+ return;
+ }
+
+ synchronized (subscriptions) {
+ final Object removedLocation = tableLocations.remove(locationKey);
+ if (removedLocation != null) {
+ maybeClearLocationForRemoval(removedLocation);
if (subscriptions.deliverNotification(
Listener::handleTableLocationKeyRemoved,
- locationKey,
+ locationKey.makeImmutable(),
transactionToken,
true)) {
onEmpty();
@@ -323,6 +323,14 @@ protected void handleTableLocationKeyRemoved(
}
}
+ private static void maybeClearLocationForRemoval(@Nullable final Object removedLocation) {
+ if (removedLocation instanceof AbstractTableLocation) {
+ final AbstractTableLocation abstractLocation = (AbstractTableLocation) removedLocation;
+ abstractLocation.handleUpdate(null, System.currentTimeMillis());
+ abstractLocation.clearColumnLocations();
+ }
+ }
+
private void verifyPartitionKeys(@NotNull final TableLocationKey locationKey) {
if (partitionKeys == null) {
partitionKeys = new ArrayList<>(locationKey.getPartitionKeys());
From cd08038aecfc010aa95701b1add618253bb1b834 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Wed, 31 Jul 2024 13:25:34 -0700
Subject: [PATCH 16/56] WIP, changes to revert some transaction token code.
---
.../impl/locations/TableLocationProvider.java | 15 ------
.../impl/AbstractTableLocationProvider.java | 49 +++++++++++--------
.../impl/PollingTableLocationProvider.java | 2 -
.../impl/TestPartitionAwareSourceTable.java | 4 --
...ebergAutoRefreshTableLocationProvider.java | 8 +--
...ergManualRefreshTableLocationProvider.java | 12 ++---
.../IcebergStaticTableLocationProvider.java | 6 +--
7 files changed, 42 insertions(+), 54 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
index d23b057ec13..86ba191fba2 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
@@ -33,14 +33,6 @@ interface Listener extends BasicTableDataListener {
*/
void beginTransaction(@NotNull Object token);
- /**
- * Begin a transaction that collects location key additions and removals to be processed atomically. Uses
- * {@code this} as the token.
- */
- default void beginTransaction() {
- beginTransaction(this);
- }
-
/**
* End the transaction and process the location changes.
*
@@ -48,13 +40,6 @@ default void beginTransaction() {
*/
void endTransaction(@NotNull Object token);
- /**
- * End the transaction and process the location changes. Uses {@code this} as the token.
- */
- default void endTransaction() {
- endTransaction(this);
- }
-
/**
*
* Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index d996e35d053..21ae98fdb9e 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -3,8 +3,11 @@
//
package io.deephaven.engine.table.impl.locations.impl;
+import io.deephaven.engine.context.ExecutionContext;
import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.impl.locations.*;
+import io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSourceManager;
+import io.deephaven.engine.updategraph.UpdateCommitter;
import io.deephaven.hash.KeyedObjectHashMap;
import io.deephaven.hash.KeyedObjectKey;
import org.jetbrains.annotations.NotNull;
@@ -43,6 +46,9 @@ public abstract class AbstractTableLocationProvider
(Collection) (Collection extends TableLocationKey>) Collections
.unmodifiableCollection(tableLocations.keySet());
+ final List locationsToClear;
+ final UpdateCommitter> locationClearCommitter;
+
private volatile boolean initialized;
private List partitionKeys;
@@ -58,6 +64,18 @@ protected AbstractTableLocationProvider(@NotNull final TableKey tableKey, final
super(supportsSubscriptions);
this.tableKey = tableKey.makeImmutable();
this.partitionKeys = null;
+
+ locationsToClear = new ArrayList<>();
+ locationClearCommitter = new UpdateCommitter<>(this,
+ ExecutionContext.getContext().getUpdateGraph(),
+ (ignored) -> {
+ locationsToClear.forEach(location -> {
+ location.handleUpdate(null, System.currentTimeMillis());
+ location.clearColumnLocations();
+
+ });
+ locationsToClear.clear();
+ });
}
/**
@@ -87,6 +105,10 @@ protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider
unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded);
}
+ protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey locationKey) {
+ handleTableLocationKeyAdded(locationKey, null);
+ }
+
/**
* Deliver a possibly-new key.
*
@@ -124,14 +146,6 @@ protected final void handleTableLocationKeyAdded(
}
}
- /**
- * Internal method to begin an atomic transaction of location adds and removes. This method passes {@code this} as
- * the transaction token.
- */
- protected final void beginTransaction() {
- beginTransaction(this);
- }
-
/**
* Internal method to begin an atomic transaction of location adds and removes.
*
@@ -143,14 +157,6 @@ protected final void beginTransaction(@NotNull final Object token) {
}
}
- /**
- * Internal method to end an atomic transaction of location adds and removes. This method passes {@code this} as the
- * transaction token.
- */
- protected final void endTransaction() {
- endTransaction(this);
- }
-
/**
* Internal method to end an atomic transaction of location adds and removes.
*
@@ -293,6 +299,10 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey)
handleTableLocationKeyRemoved(locationKey, null);
}
+ protected final void handleTableLocationKeyRemoved(@NotNull final TableLocationKey locationKey) {
+ handleTableLocationKeyRemoved(locationKey, null);
+ }
+
/**
* Handle a removal, optionally as part of a transaction. Notify subscribers that {@code locationKey} was removed if
* necessary. See {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics.
@@ -323,11 +333,10 @@ protected void handleTableLocationKeyRemoved(
}
}
- private static void maybeClearLocationForRemoval(@Nullable final Object removedLocation) {
+ private void maybeClearLocationForRemoval(@Nullable final Object removedLocation) {
if (removedLocation instanceof AbstractTableLocation) {
- final AbstractTableLocation abstractLocation = (AbstractTableLocation) removedLocation;
- abstractLocation.handleUpdate(null, System.currentTimeMillis());
- abstractLocation.clearColumnLocations();
+ locationsToClear.add((AbstractTableLocation) removedLocation);
+ locationClearCommitter.maybeActivate();
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
index 5ac5c65c4e0..c9d213713d5 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
@@ -52,7 +52,6 @@ public String getImplementationName() {
// polling. We do need a mechanism to avoid going backwards, probably.
@Override
public void refresh() {
- beginTransaction();
final Set missedKeys = new HashSet<>(getTableLocationKeys());
locationKeyFinder.findKeys(tableLocationKey -> {
// noinspection SuspiciousMethodCalls
@@ -60,7 +59,6 @@ public void refresh() {
handleTableLocationKeyAdded(tableLocationKey);
});
missedKeys.forEach(this::handleTableLocationKeyRemoved);
- endTransaction();
setInitialized();
}
diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java
index e5933d866bb..6bed409a747 100644
--- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java
+++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java
@@ -247,9 +247,7 @@ private void doInitializeCheck(final ImmutableTableLocationKey[] tableLocationKe
@Override
public Object invoke(Invocation invocation) {
subscriptionBuffer = (TableLocationSubscriptionBuffer) invocation.getParameter(0);
- subscriptionBuffer.beginTransaction();
Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded);
- subscriptionBuffer.endTransaction();
return null;
}
});
@@ -409,9 +407,7 @@ public Object invoke(Invocation invocation) {
private void doAddLocationsRefreshCheck(final ImmutableTableLocationKey[] tableLocationKeys,
final Set expectPassFilters) {
- subscriptionBuffer.beginTransaction();
Arrays.stream(tableLocationKeys).forEach(subscriptionBuffer::handleTableLocationKeyAdded);
- subscriptionBuffer.endTransaction();
expectPassFilters.forEach(tl -> checking(new Expectations() {
{
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
index ad7212e1c3f..4a826d460f5 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
@@ -84,14 +84,14 @@ public void update(Snapshot snapshot) {
* locations and removed locations.
*/
private void refreshSnapshot() {
- beginTransaction();
+ beginTransaction(this);
final Set missedKeys = new HashSet<>(getTableLocationKeys());
locationKeyFinder.findKeys(tableLocationKey -> {
missedKeys.remove(tableLocationKey);
- handleTableLocationKeyAdded(tableLocationKey);
+ handleTableLocationKeyAdded(tableLocationKey, this);
});
- missedKeys.forEach(this::handleTableLocationKeyRemoved);
- endTransaction();
+ missedKeys.forEach(tlk -> handleTableLocationKeyRemoved(tlk, this));
+ endTransaction(this);
setInitialized();
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
index bfa5e99e60d..c4639a3edf5 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
@@ -91,14 +91,14 @@ public synchronized void update(final Snapshot snapshot) {
* locations and removed locations.
*/
private void refreshSnapshot() {
- beginTransaction();
+ beginTransaction(this);
final Set missedKeys = new HashSet<>(getTableLocationKeys());
- locationKeyFinder.findKeys(tableLocationKey -> {
- missedKeys.remove(tableLocationKey);
- handleTableLocationKeyAdded(tableLocationKey);
+ locationKeyFinder.findKeys(tlk -> {
+ missedKeys.remove(tlk);
+ handleTableLocationKeyAdded(tlk, this);
});
- missedKeys.forEach(this::handleTableLocationKeyRemoved);
- endTransaction();
+ missedKeys.forEach(tlk -> handleTableLocationKeyRemoved(tlk, this));
+ endTransaction(this);
setInitialized();
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
index 92798252dad..a9f52487681 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
@@ -40,9 +40,9 @@ public String getImplementationName() {
@Override
public void refresh() {
- beginTransaction();
- locationKeyFinder.findKeys(this::handleTableLocationKeyAdded);
- endTransaction();
+ beginTransaction(this);
+ locationKeyFinder.findKeys(tlk -> handleTableLocationKeyAdded(tlk, this));
+ endTransaction(this);
setInitialized();
}
From f28325f46ace6059d1a755f6f9dac6aa4f61ed83 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Wed, 21 Aug 2024 12:28:46 -0700
Subject: [PATCH 17/56] Correct logic across multiple transactions.
---
.../locations/impl/TableLocationSubscriptionBuffer.java | 6 ++----
1 file changed, 2 insertions(+), 4 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
index 1962ae39ce1..44ac8119c11 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
@@ -175,8 +175,7 @@ public void endTransaction(@NotNull final Object token) {
if (tokenLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) {
for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsAdded) {
// Verify that we don't have stacked adds (without intervening removes).
- if (pendingLocationsAdded.contains(tableLocationKey)
- && !pendingLocationsRemoved.contains(tableLocationKey)) {
+ if (pendingLocationsAdded.contains(tableLocationKey)) {
throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ " was already added by a previous transaction.");
}
@@ -203,8 +202,7 @@ public void handleTableLocationKeyAdded(
if (transactionToken == null) {
// If we're not in a transaction, modify the pending locations directly.
// Need to verify that we don't have stacked adds (without intervening removes).
- if (pendingLocationsAdded.contains(tableLocationKey)
- && !pendingLocationsRemoved.contains(tableLocationKey)) {
+ if (pendingLocationsAdded.contains(tableLocationKey)) {
throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ " was already added by a previous transaction.");
}
From cd31d82a4818e862498d613e7caccd582dd76ff8 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Fri, 23 Aug 2024 11:18:26 -0700
Subject: [PATCH 18/56] Moved transaction accumulation to
AbstractTableLocationProvider
---
.../util/datastructures/SubscriptionSet.java | 14 +-
.../impl/locations/TableLocationProvider.java | 16 +-
.../impl/AbstractTableLocationProvider.java | 185 ++++++++++++++----
.../impl/FilteredTableDataService.java | 10 +
.../impl/TableLocationSubscriptionBuffer.java | 184 ++++++-----------
5 files changed, 242 insertions(+), 167 deletions(-)
diff --git a/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java b/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java
index 1024f255ca2..cc76ffe0c44 100644
--- a/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java
+++ b/Util/src/main/java/io/deephaven/util/datastructures/SubscriptionSet.java
@@ -221,18 +221,18 @@ public final boolean deliverNotification(
}
/**
- * Dispatch a binary notification to all subscribers. Clean up any GC'd subscriptions.
+ * Dispatch a notification to all subscribers. Clean up any GC'd subscriptions.
*
* @param procedure The notification procedure to invoke
- * @param notification The notification to deliver
- * @param token An additional token to deliver (usually a transaction token)
+ * @param firstNotification The first item to deliver
+ * @param secondNotification The second item to deliver (must be of the same type as {@code firstNotification})
* @param activeOnly Whether to restrict this notification to active subscriptions only
* @return Whether this operation caused the set to become empty
*/
public final boolean deliverNotification(
- @NotNull final TriConsumer procedure,
- @Nullable final NOTIFICATION_TYPE notification,
- @Nullable final Object token,
+ @NotNull final TriConsumer procedure,
+ @Nullable final NOTIFICATION_TYPE firstNotification,
+ @Nullable final NOTIFICATION_TYPE secondNotification,
final boolean activeOnly) {
final int initialSize = size;
for (int si = 0; si < size;) {
@@ -243,7 +243,7 @@ public final boolean deliverNotification(
continue; // si is not incremented in this case - we'll reconsider the same slot if necessary.
}
if (!activeOnly || currentEntry.isActive()) {
- procedure.accept(currentListener, notification, token);
+ procedure.accept(currentListener, firstNotification, secondNotification);
}
++si;
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
index 86ba191fba2..2339ec2efaf 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
@@ -49,7 +49,7 @@ interface Listener extends BasicTableDataListener {
*
* If transactionToken is {@code null}, the key will be added to the pending additions immediately.
*
+ * Notify the listener of collections of {@link TableLocationKey TableLocationKeys} added or removed while
+ * initiating or maintaining the location subscription. This should occur at most once per location, but the
+ * order of delivery is not guaranteed.
+ *
+ *
+ * @param addedKeys Collection of table location keys that were added.
+ * @param removedKeys Collection of table location keys that were removed.
+ */
+ void handleTableLocationKeysUpdate(
+ @Nullable Collection addedKeys,
+ @Nullable Collection removedKeys);
}
/**
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index 21ae98fdb9e..8df894bd253 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -3,10 +3,10 @@
//
package io.deephaven.engine.table.impl.locations.impl;
+import io.deephaven.base.verify.Assert;
import io.deephaven.engine.context.ExecutionContext;
import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.impl.locations.*;
-import io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSourceManager;
import io.deephaven.engine.updategraph.UpdateCommitter;
import io.deephaven.hash.KeyedObjectHashMap;
import io.deephaven.hash.KeyedObjectKey;
@@ -28,8 +28,17 @@ public abstract class AbstractTableLocationProvider
extends SubscriptionAggregator
implements TableLocationProvider {
+ private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet();
+
private final ImmutableTableKey tableKey;
+ // These sets represent open transactions that are being accumulated.
+ private final Set transactionTokens = new HashSet<>();
+ private final Map> accumulatedLocationsAdded = new HashMap<>();
+ private final Map> accumulatedLocationsRemoved = new HashMap<>();
+
+ private final Object transactionLock = new Object();
+
/**
* Map from {@link TableLocationKey} to itself, or to a {@link TableLocation}. The values are {@link TableLocation}s
* if:
@@ -102,13 +111,95 @@ public final ImmutableTableKey getKey() {
@Override
protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider.Listener listener) {
- unmodifiableTableLocationKeys.forEach(listener::handleTableLocationKeyAdded);
+ listener.handleTableLocationKeysUpdate(unmodifiableTableLocationKeys, null);
}
protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey locationKey) {
handleTableLocationKeyAdded(locationKey, null);
}
+ /**
+ * Internal method to begin an atomic transaction of location adds and removes.
+ *
+ * @param token A token to identify the transaction
+ */
+ protected void beginTransaction(@NotNull final Object token) {
+ synchronized (transactionLock) {
+ // Verify that we can start a new transaction with this token.
+ if (transactionTokens.contains(token)) {
+ throw new IllegalStateException("A transaction with token " + token + " is currently open.");
+ }
+ Assert.eqFalse(accumulatedLocationsAdded.containsKey(token),
+ "accumulatedLocationsAdded.containsKey(token)");
+ Assert.eqFalse(accumulatedLocationsRemoved.containsKey(token),
+ "accumulatedLocationsRemoved.containsKey(token)");
+
+ transactionTokens.add(token);
+ accumulatedLocationsAdded.put(token, EMPTY_TABLE_LOCATION_KEYS);
+ accumulatedLocationsRemoved.put(token, EMPTY_TABLE_LOCATION_KEYS);
+ }
+ }
+
+ /**
+ * Internal method to end an atomic transaction of location adds and removes.
+ *
+ * @param token A token to identify the transaction
+ */
+ protected void endTransaction(@NotNull final Object token) {
+ final Set locationsAdded;
+ final Set locationsRemoved;
+ synchronized (transactionLock) {
+ // Verify that this transaction is open.
+ if (!transactionTokens.remove(token)) {
+ throw new IllegalStateException("No transaction with token " + token + " is currently open.");
+ }
+
+ locationsAdded = accumulatedLocationsAdded.remove(token);
+ locationsRemoved = accumulatedLocationsRemoved.remove(token);
+ }
+
+ final Collection addedImmutableKeys = new ArrayList<>(locationsAdded.size());
+ final Collection removedImmutableKeys = new ArrayList<>(locationsRemoved.size());
+
+ // Process the accumulated adds and removes under a lock on `tableLocations` to keep modifications atomic to
+ // other holders of this lock.
+ synchronized (tableLocations) {
+ if (locationsAdded != EMPTY_TABLE_LOCATION_KEYS || locationsRemoved != EMPTY_TABLE_LOCATION_KEYS) {
+ for (TableLocationKey locationKey : locationsAdded) {
+ locationCreatedRecorder = false;
+ final Object result = tableLocations.putIfAbsent(locationKey, this::observeInsert);
+ visitLocationKey(locationKey);
+ if (locationCreatedRecorder) {
+ verifyPartitionKeys(locationKey);
+ addedImmutableKeys.add(toKeyImmutable(result));
+ }
+ }
+
+ for (TableLocationKey locationKey : locationsRemoved) {
+ final Object removedLocation = tableLocations.remove(locationKey);
+ if (removedLocation != null) {
+ maybeClearLocationForRemoval(removedLocation);
+ removedImmutableKeys.add(toKeyImmutable(locationKey));
+ }
+ }
+ }
+ }
+
+ if (subscriptions != null) {
+ synchronized (subscriptions) {
+ // Push the notifications to the subscribers.
+ if ((!addedImmutableKeys.isEmpty() || !removedImmutableKeys.isEmpty())
+ && subscriptions.deliverNotification(
+ Listener::handleTableLocationKeysUpdate,
+ addedImmutableKeys,
+ removedImmutableKeys,
+ true)) {
+ onEmpty();
+ }
+ }
+ }
+ }
+
/**
* Deliver a possibly-new key.
*
@@ -119,12 +210,35 @@ protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey
protected final void handleTableLocationKeyAdded(
@NotNull final TableLocationKey locationKey,
@Nullable final Object transactionToken) {
+
if (!supportsSubscriptions()) {
tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable);
visitLocationKey(toKeyImmutable(locationKey));
return;
}
+ if (transactionToken != null) {
+ // When adding a location in a transaction, check for logical consistency.
+ // 1. If the location was already added in this transaction, we have a problem. A transaction should not
+ // add the same location twice.
+ // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a
+ // logical error (although it may not be supported by all consumers).
+ synchronized (transactionLock) {
+ if (accumulatedLocationsAdded.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) {
+ accumulatedLocationsAdded.put(transactionToken, new HashSet<>());
+ }
+ final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken);
+
+ if (accumulatedLocationsAdded.containsKey(locationKey)) {
+ throw new IllegalStateException("TableLocationKey " + locationKey
+ + " was added multiple times in the same transaction.");
+ }
+ locationsAdded.add(locationKey);
+ }
+ return;
+ }
+
+ // If we're not in a transaction, we should push this key immediately.
synchronized (subscriptions) {
// Since we're holding the lock on subscriptions, the following code is overly complicated - we could
// certainly just deliver the notification in observeInsert. That said, I'm happier with this approach,
@@ -138,7 +252,6 @@ protected final void handleTableLocationKeyAdded(
if (subscriptions.deliverNotification(
Listener::handleTableLocationKeyAdded,
toKeyImmutable(result),
- transactionToken,
true)) {
onEmpty();
}
@@ -146,28 +259,6 @@ protected final void handleTableLocationKeyAdded(
}
}
- /**
- * Internal method to begin an atomic transaction of location adds and removes.
- *
- * @param token A token to identify the transaction
- */
- protected final void beginTransaction(@NotNull final Object token) {
- if (subscriptions != null) {
- subscriptions.deliverNotification(Listener::beginTransaction, token, true);
- }
- }
-
- /**
- * Internal method to end an atomic transaction of location adds and removes.
- *
- * @param token A token to identify the transaction
- */
- protected final void endTransaction(@NotNull final Object token) {
- if (subscriptions != null) {
- subscriptions.deliverNotification(Listener::endTransaction, token, true);
- }
- }
-
/**
* Called after a table location has been visited by
* {@link #handleTableLocationKeyAdded(TableLocationKey, Object)}, but before notifications have been delivered to
@@ -232,14 +323,12 @@ protected void doInitialization() {
@Override
@NotNull
public final Collection getTableLocationKeys() {
- // We need to prevent reading the map (and maybe mutating it?) during a transaction.
- // We could transition between two maps, a stable copy and a shadow copy that is being mutated.
- // Or we could hold a bigger lock while mutating the map, and hold the same lock here. Sounds like a job for a
- // read-write lock (e.g. ReentrantReadWriteLock), maybe. If you want `FunctionalLock`, the pattern (but mostly
- // not the code) from io.deephaven.engine.updategraph.UpdateGraphLock could help.
- // I think we need the read-write lock for correctness, and I think we need to make it explicit. That is, the
- // user needs to be able to get a read lock and hold it while it's operating on the returned collection.
- return unmodifiableTableLocationKeys;
+ // This lock is held while `endTransaction()` updates `tableLocations` with the accumulated adds/removes.
+ // Locking here ensures that this call won't return while `tableLocations` (and `unmodifiableTableLocationKeys`)
+ // contain a partial transaction.
+ synchronized (tableLocations) {
+ return unmodifiableTableLocationKeys;
+ }
}
@Override
@@ -274,8 +363,8 @@ public TableLocation getTableLocationIfPresent(@NotNull final TableLocationKey t
/**
* Remove a {@link TableLocationKey} and its corresponding {@link TableLocation} (if it was created). All
* subscribers to this TableLocationProvider will be
- * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(ImmutableTableLocationKey, Object)}
- * notified}. If the TableLocation was created, all of its subscribers will additionally be
+ * {@link TableLocationProvider.Listener#handleTableLocationKeyRemoved(ImmutableTableLocationKey) notified}. If the
+ * TableLocation was created, all of its subscribers will additionally be
* {@link TableLocation.Listener#handleUpdate() notified} that it no longer exists. This TableLocationProvider will
* continue to update other locations and will no longer provide or request information about the removed location.
*
@@ -318,6 +407,31 @@ protected void handleTableLocationKeyRemoved(
return;
}
+ // When removing a location in a transaction, check for logical consistency.
+ // 1. If the location was already removed in this transaction, we have a problem. A transaction should not
+ // remove the same location twice.
+ // 2. If the location was already added in this transaction, we have a problem. A transaction should not
+ // add then remove the same location.
+ if (transactionToken != null) {
+ synchronized (transactionLock) {
+ if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) {
+ accumulatedLocationsRemoved.put(transactionToken, new HashSet<>());
+ }
+ final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken);
+
+ if (accumulatedLocationsRemoved.containsKey(locationKey)) {
+ throw new IllegalStateException("TableLocationKey " + locationKey
+ + " was removed multiple times in the same transaction.");
+ } else if (accumulatedLocationsAdded.containsKey(locationKey)) {
+ throw new IllegalStateException("TableLocationKey " + locationKey
+ + " was removed after being added in the same transaction.");
+ }
+ locationsRemoved.add(locationKey);
+ return;
+ }
+ }
+
+ // If we're not in a transaction, we should push this key immediately.
synchronized (subscriptions) {
final Object removedLocation = tableLocations.remove(locationKey);
if (removedLocation != null) {
@@ -325,7 +439,6 @@ protected void handleTableLocationKeyRemoved(
if (subscriptions.deliverNotification(
Listener::handleTableLocationKeyRemoved,
locationKey.makeImmutable(),
- transactionToken,
true)) {
onEmpty();
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
index 6592ea11a12..948fb1fe38b 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/FilteredTableDataService.java
@@ -196,6 +196,16 @@ public void handleTableLocationKeyRemoved(
}
}
+ @Override
+ public void handleTableLocationKeysUpdate(
+ @Nullable Collection addedKeys,
+ @Nullable Collection removedKeys) {
+ final TableLocationProvider.Listener outputListener = getWrapped();
+ if (outputListener != null) {
+ outputListener.handleTableLocationKeysUpdate(addedKeys, removedKeys);
+ }
+ }
+
@Override
public void handleException(@NotNull final TableDataException exception) {
final TableLocationProvider.Listener outputListener = getWrapped();
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
index 44ac8119c11..c73658910d2 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
@@ -3,7 +3,6 @@
//
package io.deephaven.engine.table.impl.locations.impl;
-import io.deephaven.base.verify.Assert;
import io.deephaven.base.verify.Require;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableDataException;
@@ -30,11 +29,6 @@ public class TableLocationSubscriptionBuffer implements TableLocationProvider.Li
private Set pendingLocationsAdded = EMPTY_TABLE_LOCATION_KEYS;
private Set pendingLocationsRemoved = EMPTY_TABLE_LOCATION_KEYS;
- // These sets represent open transactions that are being accumulated.
- private final Set transactionTokens = new HashSet<>();
- private final Map> accumulatedLocationsAdded = new HashMap<>();
- private final Map> accumulatedLocationsRemoved = new HashMap<>();
-
private TableDataException pendingException = null;
public TableLocationSubscriptionBuffer(@NotNull final TableLocationProvider tableLocationProvider) {
@@ -77,9 +71,7 @@ public synchronized LocationUpdate processPending() {
// NB: Providers that don't support subscriptions don't tick - this single call to run is
// sufficient.
tableLocationProvider.refresh();
- // TODO: cast this to AbstractTableLocationProvider and call begin/endTransaction?
- tableLocationProvider.getTableLocationKeys()
- .forEach(tlk -> handleTableLocationKeyAdded(tlk, null));
+ handleTableLocationKeysUpdate(tableLocationProvider.getTableLocationKeys(), null);
}
subscribed = true;
}
@@ -125,67 +117,12 @@ public synchronized void reset() {
@Override
public void beginTransaction(@NotNull final Object token) {
- synchronized (updateLock) {
- // Verify that we can start a new transaction with this token.
- if (transactionTokens.contains(token)) {
- throw new IllegalStateException("A transaction with token " + token + " is currently open.");
- }
- Assert.eqFalse(accumulatedLocationsAdded.containsKey(token),
- "accumulatedLocationsAdded.containsKey(token)");
- Assert.eqFalse(accumulatedLocationsRemoved.containsKey(token),
- "accumulatedLocationsRemoved.containsKey(token)");
-
- transactionTokens.add(token);
- accumulatedLocationsAdded.put(token, EMPTY_TABLE_LOCATION_KEYS);
- accumulatedLocationsRemoved.put(token, EMPTY_TABLE_LOCATION_KEYS);
- }
+ throw new UnsupportedOperationException("Transactions are not supported by this provider.");
}
@Override
public void endTransaction(@NotNull final Object token) {
- synchronized (updateLock) {
- // Verify that this transaction is open.
- if (!transactionTokens.remove(token)) {
- throw new IllegalStateException("No transaction with token " + token + " is currently open.");
- }
-
- final Set tokenLocationsAdded = accumulatedLocationsAdded.remove(token);
- final Set tokenLocationsRemoved = accumulatedLocationsRemoved.remove(token);
-
- if (tokenLocationsRemoved != EMPTY_TABLE_LOCATION_KEYS) {
- for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsRemoved) {
- // If we have a pending add that is removed by this transaction, we can remove it from the pending
- // list because it is cancelled by this remove. This also covers the case where a `replace`
- // operation has occurred in a previous transaction.
- if (pendingLocationsAdded.remove(tableLocationKey)) {
- continue;
- }
- // Verify that we don't have stacked removes (without intervening adds).
- if (pendingLocationsRemoved.contains(tableLocationKey)) {
- throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was already removed by a previous transaction.");
- }
- if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
- pendingLocationsRemoved = new HashSet<>();
- }
- pendingLocationsRemoved.add(tableLocationKey);
- }
- }
-
- if (tokenLocationsAdded != EMPTY_TABLE_LOCATION_KEYS) {
- for (final ImmutableTableLocationKey tableLocationKey : tokenLocationsAdded) {
- // Verify that we don't have stacked adds (without intervening removes).
- if (pendingLocationsAdded.contains(tableLocationKey)) {
- throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was already added by a previous transaction.");
- }
- if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) {
- pendingLocationsAdded = new HashSet<>();
- }
- pendingLocationsAdded.add(tableLocationKey);
- }
- }
- }
+ throw new UnsupportedOperationException("Transactions are not supported by this provider.");
}
@Override
@@ -193,36 +130,19 @@ public void handleTableLocationKeyAdded(
@NotNull final ImmutableTableLocationKey tableLocationKey,
@Nullable Object transactionToken) {
synchronized (updateLock) {
- // When adding a location in a transaction, check for logical consistency.
- // 1. If the location was already added in this transaction, we have a problem. A transaction should not
- // add the same location twice.
- // 2. If the location was already removed in this transaction, we have a `replace` operation which is not a
- // logical error (although it may not be supported by all consumers).
-
- if (transactionToken == null) {
- // If we're not in a transaction, modify the pending locations directly.
- // Need to verify that we don't have stacked adds (without intervening removes).
- if (pendingLocationsAdded.contains(tableLocationKey)) {
- throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was already added by a previous transaction.");
- }
- if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) {
- pendingLocationsAdded = new HashSet<>();
- }
- pendingLocationsAdded.add(tableLocationKey);
- return;
+ if (transactionToken != null) {
+ throw new UnsupportedOperationException("Transactions are not supported by this provider.");
}
- if (accumulatedLocationsAdded.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) {
- accumulatedLocationsAdded.put(transactionToken, new HashSet<>());
- }
- final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken);
-
- if (accumulatedLocationsAdded.containsKey(tableLocationKey)) {
+ // Need to verify that we don't have stacked adds (without intervening removes).
+ if (pendingLocationsAdded.contains(tableLocationKey)) {
throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was added multiple times in the same transaction.");
+ + " was already added by a previous transaction.");
}
- locationsAdded.add(tableLocationKey);
+ if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) {
+ pendingLocationsAdded = new HashSet<>();
+ }
+ pendingLocationsAdded.add(tableLocationKey);
}
}
@@ -231,43 +151,61 @@ public void handleTableLocationKeyRemoved(
@NotNull final ImmutableTableLocationKey tableLocationKey,
final Object transactionToken) {
synchronized (updateLock) {
- // When removing a location in a transaction, check for logical consistency.
- // 1. If the location was already removed in this transaction, we have a problem. A transaction should not
- // remove the same location twice.
- // 2. If the location was already added in this transaction, we have a problem. A transaction should not
- // add then remove the same location.
+ if (transactionToken != null) {
+ throw new UnsupportedOperationException("Transactions are not supported by this provider.");
+ }
- if (transactionToken == null) {
- // If we're not in a transaction, modify the pending locations directly.
- // If we have a pending add, it is being cancelled by this remove.
- if (pendingLocationsAdded.remove(tableLocationKey)) {
- return;
- }
- // Verify that we don't have stacked removes (without intervening adds).
- if (pendingLocationsRemoved.contains(tableLocationKey)) {
- throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was already removed by a previous transaction.");
- }
- if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
- pendingLocationsRemoved = new HashSet<>();
- }
- pendingLocationsRemoved.add(tableLocationKey);
+ // If we have a pending add, it is being cancelled by this remove.
+ if (pendingLocationsAdded.remove(tableLocationKey)) {
return;
}
-
- if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) {
- accumulatedLocationsRemoved.put(transactionToken, new HashSet<>());
+ // Verify that we don't have stacked removes (without intervening adds).
+ if (pendingLocationsRemoved.contains(tableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + tableLocationKey
+ + " was already removed by a previous transaction.");
+ }
+ if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
+ pendingLocationsRemoved = new HashSet<>();
}
- final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken);
+ pendingLocationsRemoved.add(tableLocationKey);
+ }
+ }
- if (accumulatedLocationsRemoved.containsKey(tableLocationKey)) {
- throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was removed multiple times in the same transaction.");
- } else if (accumulatedLocationsAdded.containsKey(tableLocationKey)) {
- throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was removed after being added in the same transaction.");
+ @Override
+ public void handleTableLocationKeysUpdate(
+ @Nullable Collection addedKeys,
+ @Nullable Collection removedKeys) {
+ synchronized (updateLock) {
+ if (removedKeys != null) {
+ for (final ImmutableTableLocationKey removedTableLocationKey : removedKeys) {
+ // If we have a pending add, it is being cancelled by this remove.
+ if (pendingLocationsAdded.remove(removedTableLocationKey)) {
+ continue;
+ }
+ // Verify that we don't have stacked removes.
+ if (pendingLocationsRemoved.contains(removedTableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + removedTableLocationKey
+ + " was already removed by a previous transaction.");
+ }
+ if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
+ pendingLocationsRemoved = new HashSet<>();
+ }
+ pendingLocationsRemoved.add(removedTableLocationKey);
+ }
+ }
+ if (addedKeys != null) {
+ for (final ImmutableTableLocationKey addedTableLocationKey : addedKeys) {
+ // Need to verify that we don't have stacked adds.
+ if (pendingLocationsAdded.contains(addedTableLocationKey)) {
+ throw new IllegalStateException("TableLocationKey " + addedTableLocationKey
+ + " was already added by a previous transaction.");
+ }
+ if (pendingLocationsAdded == EMPTY_TABLE_LOCATION_KEYS) {
+ pendingLocationsAdded = new HashSet<>();
+ }
+ pendingLocationsAdded.add(addedTableLocationKey);
+ }
}
- locationsRemoved.add(tableLocationKey);
}
}
From d680c0c66b25c053352834de65a9e91a78b7097e Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Mon, 26 Aug 2024 12:25:28 -0700
Subject: [PATCH 19/56] Moved transaction accumulation to
AbstractTableLocationProvider
---
.../impl/locations/TableLocationProvider.java | 12 +-
.../impl/AbstractTableLocationProvider.java | 134 ++++++++++--------
.../impl/CompositeTableDataService.java | 4 +-
.../impl/PollingTableLocationProvider.java | 15 +-
.../impl/TableLocationSubscriptionBuffer.java | 4 +-
.../util/ExecutorTableDataRefreshService.java | 36 ++---
.../util/TableDataRefreshService.java | 22 ++-
.../regioned/RegionedColumnSourceManager.java | 59 ++++----
8 files changed, 165 insertions(+), 121 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
index 2339ec2efaf..b6205431a59 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocationProvider.java
@@ -51,7 +51,8 @@ interface Listener extends BasicTableDataListener {
*
*
* @param tableLocationKey The new table location key.
- * @param transactionToken The token identifying the transaction.
+ * @param transactionToken The token identifying the transaction, or {@code null} if this addition is not part
+ * of a transaction.
*/
void handleTableLocationKeyAdded(
@NotNull ImmutableTableLocationKey tableLocationKey,
@@ -60,7 +61,8 @@ void handleTableLocationKeyAdded(
/**
* Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location
* subscription. This should occur at most once per location, but the order of delivery is not
- * guaranteed. Uses {@code this} as the token.
+ * guaranteed. This addition is not part of any transaction, and is equivalent to
+ * {@code handleTableLocationKeyAdded(tableLocationKey, null);} by default.
*
* @param tableLocationKey The new table location key.
*/
@@ -77,14 +79,16 @@ default void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tabl
*
*
* @param tableLocationKey The table location key that was removed.
- * @param transactionToken The token identifying the transaction.
+ * @param transactionToken The token identifying the transaction, or {@code null} if this addition is not part
+ * of a transaction.
*/
void handleTableLocationKeyRemoved(
@NotNull ImmutableTableLocationKey tableLocationKey,
@Nullable Object transactionToken);
/**
- * Notify the listener of a {@link TableLocationKey} that has been removed. Uses {@code this} as the token.
+ * Notify the listener of a {@link TableLocationKey} that has been removed. This addition is not part of any
+ * transaction, and is equivalent to {@code handleTableLocationKeyAdded(tableLocationKey, null);} by default.
*
* @param tableLocationKey The table location key that was removed.
*/
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index 8df894bd253..4afbb770116 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -3,7 +3,6 @@
//
package io.deephaven.engine.table.impl.locations.impl;
-import io.deephaven.base.verify.Assert;
import io.deephaven.engine.context.ExecutionContext;
import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.impl.locations.*;
@@ -30,12 +29,41 @@ public abstract class AbstractTableLocationProvider
private static final Set EMPTY_TABLE_LOCATION_KEYS = Collections.emptySet();
+ /**
+ * Helper class to manage a transaction of added and removed location keys.
+ */
+ private static class Transaction {
+ Set locationsAdded = EMPTY_TABLE_LOCATION_KEYS;
+ Set locationsRemoved = EMPTY_TABLE_LOCATION_KEYS;
+
+ void addlocationKey(TableLocationKey locationKey) {
+ if (locationsAdded == EMPTY_TABLE_LOCATION_KEYS) {
+ locationsAdded = new HashSet<>();
+ } else if (locationsAdded.contains(locationKey)) {
+ throw new IllegalStateException("TableLocationKey " + locationKey
+ + " was already added in this transaction.");
+ }
+ locationsAdded.add(locationKey);
+ }
+
+ void removeLocationKey(TableLocationKey locationKey) {
+ if (locationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
+ locationsRemoved = new HashSet<>();
+ } else if (locationsRemoved.contains(locationKey)) {
+ throw new IllegalStateException("TableLocationKey " + locationKey
+ + " was already removed and has not been replaced.");
+ } else if (locationsAdded.contains(locationKey)) {
+ throw new IllegalStateException("TableLocationKey " + locationKey
+ + " was removed after being added in the same transaction.");
+ }
+ locationsRemoved.add(locationKey);
+ }
+ }
+
private final ImmutableTableKey tableKey;
// These sets represent open transactions that are being accumulated.
- private final Set transactionTokens = new HashSet<>();
- private final Map> accumulatedLocationsAdded = new HashMap<>();
- private final Map> accumulatedLocationsRemoved = new HashMap<>();
+ private final Map transactions = new HashMap<>();
private final Object transactionLock = new Object();
@@ -114,10 +142,6 @@ protected final void deliverInitialSnapshot(@NotNull final TableLocationProvider
listener.handleTableLocationKeysUpdate(unmodifiableTableLocationKeys, null);
}
- protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey locationKey) {
- handleTableLocationKeyAdded(locationKey, null);
- }
-
/**
* Internal method to begin an atomic transaction of location adds and removes.
*
@@ -126,17 +150,10 @@ protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey
protected void beginTransaction(@NotNull final Object token) {
synchronized (transactionLock) {
// Verify that we can start a new transaction with this token.
- if (transactionTokens.contains(token)) {
+ if (transactions.containsKey(token)) {
throw new IllegalStateException("A transaction with token " + token + " is currently open.");
}
- Assert.eqFalse(accumulatedLocationsAdded.containsKey(token),
- "accumulatedLocationsAdded.containsKey(token)");
- Assert.eqFalse(accumulatedLocationsRemoved.containsKey(token),
- "accumulatedLocationsRemoved.containsKey(token)");
-
- transactionTokens.add(token);
- accumulatedLocationsAdded.put(token, EMPTY_TABLE_LOCATION_KEYS);
- accumulatedLocationsRemoved.put(token, EMPTY_TABLE_LOCATION_KEYS);
+ transactions.put(token, new Transaction());
}
}
@@ -146,26 +163,26 @@ protected void beginTransaction(@NotNull final Object token) {
* @param token A token to identify the transaction
*/
protected void endTransaction(@NotNull final Object token) {
- final Set locationsAdded;
- final Set locationsRemoved;
+ final Transaction transaction;
synchronized (transactionLock) {
// Verify that this transaction is open.
- if (!transactionTokens.remove(token)) {
+ transaction = transactions.remove(token);
+ if (transaction == null) {
throw new IllegalStateException("No transaction with token " + token + " is currently open.");
}
-
- locationsAdded = accumulatedLocationsAdded.remove(token);
- locationsRemoved = accumulatedLocationsRemoved.remove(token);
}
- final Collection addedImmutableKeys = new ArrayList<>(locationsAdded.size());
- final Collection removedImmutableKeys = new ArrayList<>(locationsRemoved.size());
+ final Collection addedImmutableKeys =
+ new ArrayList<>(transaction.locationsAdded.size());
+ final Collection removedImmutableKeys =
+ new ArrayList<>(transaction.locationsRemoved.size());
// Process the accumulated adds and removes under a lock on `tableLocations` to keep modifications atomic to
// other holders of this lock.
synchronized (tableLocations) {
- if (locationsAdded != EMPTY_TABLE_LOCATION_KEYS || locationsRemoved != EMPTY_TABLE_LOCATION_KEYS) {
- for (TableLocationKey locationKey : locationsAdded) {
+ if (transaction.locationsAdded != EMPTY_TABLE_LOCATION_KEYS
+ || transaction.locationsRemoved != EMPTY_TABLE_LOCATION_KEYS) {
+ for (TableLocationKey locationKey : transaction.locationsAdded) {
locationCreatedRecorder = false;
final Object result = tableLocations.putIfAbsent(locationKey, this::observeInsert);
visitLocationKey(locationKey);
@@ -175,7 +192,7 @@ protected void endTransaction(@NotNull final Object token) {
}
}
- for (TableLocationKey locationKey : locationsRemoved) {
+ for (TableLocationKey locationKey : transaction.locationsRemoved) {
final Object removedLocation = tableLocations.remove(locationKey);
if (removedLocation != null) {
maybeClearLocationForRemoval(removedLocation);
@@ -190,10 +207,10 @@ protected void endTransaction(@NotNull final Object token) {
// Push the notifications to the subscribers.
if ((!addedImmutableKeys.isEmpty() || !removedImmutableKeys.isEmpty())
&& subscriptions.deliverNotification(
- Listener::handleTableLocationKeysUpdate,
- addedImmutableKeys,
- removedImmutableKeys,
- true)) {
+ Listener::handleTableLocationKeysUpdate,
+ addedImmutableKeys,
+ removedImmutableKeys,
+ true)) {
onEmpty();
}
}
@@ -204,7 +221,17 @@ protected void endTransaction(@NotNull final Object token) {
* Deliver a possibly-new key.
*
* @param locationKey The new key
- * @param transactionToken The token identifying the transaction
+ * @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools.
+ */
+ protected final void handleTableLocationKeyAdded(@NotNull final TableLocationKey locationKey) {
+ handleTableLocationKeyAdded(locationKey, null);
+ }
+
+ /**
+ * Deliver a possibly-new key, optionally as part of a transaction.
+ *
+ * @param locationKey The new key
+ * @param transactionToken The token identifying the transaction (or null if not part of a transaction)
* @apiNote This method is intended to be used by subclasses or by tightly-coupled discovery tools.
*/
protected final void handleTableLocationKeyAdded(
@@ -213,7 +240,7 @@ protected final void handleTableLocationKeyAdded(
if (!supportsSubscriptions()) {
tableLocations.putIfAbsent(locationKey, TableLocationKey::makeImmutable);
- visitLocationKey(toKeyImmutable(locationKey));
+ visitLocationKey(locationKey);
return;
}
@@ -224,16 +251,12 @@ protected final void handleTableLocationKeyAdded(
// 2. If the location was already removed in this transaction, we have a `replace` operation which is not a
// logical error (although it may not be supported by all consumers).
synchronized (transactionLock) {
- if (accumulatedLocationsAdded.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) {
- accumulatedLocationsAdded.put(transactionToken, new HashSet<>());
- }
- final Set locationsAdded = accumulatedLocationsAdded.get(transactionToken);
-
- if (accumulatedLocationsAdded.containsKey(locationKey)) {
- throw new IllegalStateException("TableLocationKey " + locationKey
- + " was added multiple times in the same transaction.");
+ final Transaction transaction = transactions.get(transactionToken);
+ if (transaction == null) {
+ throw new IllegalStateException(
+ "No transaction with token " + transactionToken + " is currently open.");
}
- locationsAdded.add(locationKey);
+ transaction.addlocationKey(locationKey);
}
return;
}
@@ -388,6 +411,12 @@ public void removeTableLocationKey(@NotNull final TableLocationKey locationKey)
handleTableLocationKeyRemoved(locationKey, null);
}
+ /**
+ * Handle a removal. Notify subscribers that {@code locationKey} was removed if necessary. See
+ * {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics.
+ *
+ * @param locationKey the TableLocation that was removed
+ */
protected final void handleTableLocationKeyRemoved(@NotNull final TableLocationKey locationKey) {
handleTableLocationKeyRemoved(locationKey, null);
}
@@ -397,7 +426,7 @@ protected final void handleTableLocationKeyRemoved(@NotNull final TableLocationK
* necessary. See {@link #removeTableLocationKey(TableLocationKey)} for additional discussions of semantics.
*
* @param locationKey the TableLocation that was removed
- * @param transactionToken The token identifying the transaction
+ * @param transactionToken The token identifying the transaction (or null if not part of a transaction)
*/
protected void handleTableLocationKeyRemoved(
@NotNull final TableLocationKey locationKey,
@@ -414,19 +443,12 @@ protected void handleTableLocationKeyRemoved(
// add then remove the same location.
if (transactionToken != null) {
synchronized (transactionLock) {
- if (accumulatedLocationsRemoved.get(transactionToken) == EMPTY_TABLE_LOCATION_KEYS) {
- accumulatedLocationsRemoved.put(transactionToken, new HashSet<>());
- }
- final Set locationsRemoved = accumulatedLocationsRemoved.get(transactionToken);
-
- if (accumulatedLocationsRemoved.containsKey(locationKey)) {
- throw new IllegalStateException("TableLocationKey " + locationKey
- + " was removed multiple times in the same transaction.");
- } else if (accumulatedLocationsAdded.containsKey(locationKey)) {
- throw new IllegalStateException("TableLocationKey " + locationKey
- + " was removed after being added in the same transaction.");
+ final Transaction transaction = transactions.get(transactionToken);
+ if (transaction == null) {
+ throw new IllegalStateException(
+ "No transaction with token " + transactionToken + " is currently open.");
}
- locationsRemoved.add(locationKey);
+ transaction.removeLocationKey(locationKey);
return;
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
index 3abcbf623e3..a9157da4b0f 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/CompositeTableDataService.java
@@ -115,9 +115,7 @@ public void subscribe(@NotNull final Listener listener) {
p.subscribe(listener);
} else {
p.refresh();
- listener.beginTransaction(p);
- p.getTableLocationKeys().forEach(tlk -> listener.handleTableLocationKeyAdded(tlk, p));
- listener.endTransaction(p);
+ p.getTableLocationKeys().forEach(listener::handleTableLocationKeyAdded);
}
});
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
index c9d213713d5..a350b93096e 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/PollingTableLocationProvider.java
@@ -8,9 +8,6 @@
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
-import java.util.HashSet;
-import java.util.Set;
-
/**
* Polling-driven {@link TableLocationProvider} implementation that delegates {@link TableLocationKey location key}
* discovery to a {@link TableLocationKeyFinder} and {@link TableLocation location} creation to a
@@ -46,19 +43,9 @@ public String getImplementationName() {
return IMPLEMENTATION_NAME;
}
- // The simplest way to support "push" of new data availability is to provide a callback to the user that just calls
- // `refresh`, which would need to become synchronized. Alternatively, we could make an Iceberg-specific aTLP
- // implementation that exposes a more specific callback, e.g. with a snapshot ID, as well as the option to disable
- // polling. We do need a mechanism to avoid going backwards, probably.
@Override
public void refresh() {
- final Set missedKeys = new HashSet<>(getTableLocationKeys());
- locationKeyFinder.findKeys(tableLocationKey -> {
- // noinspection SuspiciousMethodCalls
- missedKeys.remove(tableLocationKey);
- handleTableLocationKeyAdded(tableLocationKey);
- });
- missedKeys.forEach(this::handleTableLocationKeyRemoved);
+ locationKeyFinder.findKeys(this::handleTableLocationKeyAdded);
setInitialized();
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
index c73658910d2..33a121c95ca 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/TableLocationSubscriptionBuffer.java
@@ -162,7 +162,7 @@ public void handleTableLocationKeyRemoved(
// Verify that we don't have stacked removes (without intervening adds).
if (pendingLocationsRemoved.contains(tableLocationKey)) {
throw new IllegalStateException("TableLocationKey " + tableLocationKey
- + " was already removed by a previous transaction.");
+ + " was already removed and has not been replaced.");
}
if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
pendingLocationsRemoved = new HashSet<>();
@@ -185,7 +185,7 @@ public void handleTableLocationKeysUpdate(
// Verify that we don't have stacked removes.
if (pendingLocationsRemoved.contains(removedTableLocationKey)) {
throw new IllegalStateException("TableLocationKey " + removedTableLocationKey
- + " was already removed by a previous transaction.");
+ + " was already removed and has not been replaced.");
}
if (pendingLocationsRemoved == EMPTY_TABLE_LOCATION_KEYS) {
pendingLocationsRemoved = new HashSet<>();
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java
index 8db2b625354..3f0a4a862fb 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/ExecutorTableDataRefreshService.java
@@ -28,8 +28,8 @@ public class ExecutorTableDataRefreshService implements TableDataRefreshService
private static final String NAME_PREFIX = "TableDataRefreshService-";
private final String name;
- private final long tableLocationProviderRefreshIntervalMillis;
- private final long tableLocationRefreshIntervalMillis;
+ private final long tableLocationProviderDefaultRefreshIntervalMillis;
+ private final long tableLocationDefaultRefreshIntervalMillis;
private final ScheduledThreadPoolExecutor scheduler;
@@ -43,9 +43,10 @@ public ExecutorTableDataRefreshService(@NotNull final String name,
final long tableLocationRefreshIntervalMillis,
final int threadPoolSize) {
this.name = Require.neqNull(name, "name");
- this.tableLocationProviderRefreshIntervalMillis = Require.gtZero(tableLocationProviderRefreshIntervalMillis,
- "tableLocationProviderRefreshIntervalMillis");
- this.tableLocationRefreshIntervalMillis =
+ this.tableLocationProviderDefaultRefreshIntervalMillis =
+ Require.gtZero(tableLocationProviderRefreshIntervalMillis,
+ "tableLocationProviderRefreshIntervalMillis");
+ this.tableLocationDefaultRefreshIntervalMillis =
Require.gtZero(tableLocationRefreshIntervalMillis, "tableLocationRefreshIntervalMillis");
NamingThreadFactory threadFactory = new NamingThreadFactory(TableDataRefreshService.class, "refreshThread");
@@ -110,12 +111,6 @@ public void cancel() {
private class ScheduledTableLocationProviderRefresh
extends ScheduledSubscriptionTask {
- private ScheduledTableLocationProviderRefresh(
- @NotNull final AbstractTableLocationProvider tableLocationProvider) {
- super(tableLocationProvider, tableLocationProviderRefreshIntervalMillis);
- providerSubscriptions.increment(1);
- }
-
private ScheduledTableLocationProviderRefresh(
@NotNull final AbstractTableLocationProvider tableLocationProvider,
final long refreshIntervalMillis) {
@@ -123,7 +118,6 @@ private ScheduledTableLocationProviderRefresh(
providerSubscriptions.increment(1);
}
-
@Override
protected void refresh() {
final long startTimeNanos = System.nanoTime();
@@ -140,8 +134,10 @@ public void cancel() {
private class ScheduledTableLocationRefresh extends ScheduledSubscriptionTask {
- private ScheduledTableLocationRefresh(@NotNull AbstractTableLocation tableLocation) {
- super(tableLocation, tableLocationRefreshIntervalMillis);
+ private ScheduledTableLocationRefresh(
+ @NotNull final AbstractTableLocation tableLocation,
+ final long refreshIntervalMillis) {
+ super(tableLocation, refreshIntervalMillis);
locationSubscriptions.increment(1);
}
@@ -162,7 +158,8 @@ public void cancel() {
@Override
public CancellableSubscriptionToken scheduleTableLocationProviderRefresh(
@NotNull final AbstractTableLocationProvider tableLocationProvider) {
- return new ScheduledTableLocationProviderRefresh(tableLocationProvider);
+ return new ScheduledTableLocationProviderRefresh(tableLocationProvider,
+ tableLocationProviderDefaultRefreshIntervalMillis);
}
@Override
@@ -175,6 +172,13 @@ public CancellableSubscriptionToken scheduleTableLocationProviderRefresh(
@Override
public CancellableSubscriptionToken scheduleTableLocationRefresh(
@NotNull final AbstractTableLocation tableLocation) {
- return new ScheduledTableLocationRefresh(tableLocation);
+ return new ScheduledTableLocationRefresh(tableLocation, tableLocationDefaultRefreshIntervalMillis);
+ }
+
+ @Override
+ public CancellableSubscriptionToken scheduleTableLocationRefresh(
+ @NotNull final AbstractTableLocation tableLocation,
+ final long refreshIntervalMillis) {
+ return new ScheduledTableLocationRefresh(tableLocation, refreshIntervalMillis);
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java
index 093426d0bb6..dbdb942282d 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/util/TableDataRefreshService.java
@@ -32,7 +32,7 @@ interface CancellableSubscriptionToken {
void submitOneTimeAsyncTask(@NotNull Runnable task);
/**
- * Schedule run for an AbstractTableLocationProvider.
+ * Schedule run for an AbstractTableLocationProvider using an implementation-defined default refresh interval
*
* @param tableLocationProvider The table location provider
* @return A subscription token to be used for matching, which also supports cancellation
@@ -52,13 +52,24 @@ CancellableSubscriptionToken scheduleTableLocationProviderRefresh(
long refreshIntervalMs);
/**
- * Schedule run for an AbstractTableLocation.
+ * Schedule run for an AbstractTableLocation using an implementation-defined default refresh interval
*
* @param tableLocation The table location
* @return A subscription token to be used for matching, which also supports cancellation
*/
CancellableSubscriptionToken scheduleTableLocationRefresh(@NotNull AbstractTableLocation tableLocation);
+ /**
+ * Schedule run for an AbstractTableLocation.
+ *
+ * @param tableLocation The table location
+ * @param refreshIntervalMs The interval in milliseconds between refreshes
+ * @return A subscription token to be used for matching, which also supports cancellation
+ */
+ CancellableSubscriptionToken scheduleTableLocationRefresh(
+ @NotNull AbstractTableLocation tableLocation,
+ long refreshIntervalMs);
+
/**
* Get (and possibly construct) a shared instance.
*
@@ -149,5 +160,12 @@ public CancellableSubscriptionToken scheduleTableLocationRefresh(
@NotNull final AbstractTableLocation tableLocation) {
throw new UnsupportedOperationException();
}
+
+ @Override
+ public CancellableSubscriptionToken scheduleTableLocationRefresh(
+ @NotNull AbstractTableLocation tableLocation,
+ final long refreshIntervalMs) {
+ throw new UnsupportedOperationException();
+ }
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
index 3383d7c78a1..710dcc663a7 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
@@ -26,7 +26,6 @@
import io.deephaven.io.logger.Logger;
import io.deephaven.util.SafeCloseable;
import io.deephaven.util.annotations.ReferentialIntegrity;
-import io.deephaven.util.mutable.MutableInt;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
@@ -75,17 +74,6 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col
private final KeyedObjectHashMap includedTableLocations =
new KeyedObjectHashMap<>(INCLUDED_TABLE_LOCATION_ENTRY_KEY);
- /**
- * List of locations that were removed this cycle. Will be cleared after each update.
- */
- private final List removedTableLocations = new ArrayList<>();
-
- /**
- * The next region index to assign to a location. We increment for each new location and will not reuse indices from
- * regions that were removed.
- */
- private final MutableInt nextRegionIndex = new MutableInt(0);
-
/**
* Table locations that provide the regions backing our column sources, in insertion order.
*/
@@ -122,7 +110,27 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col
@ReferentialIntegrity
private Runnable delayedErrorReference;
- final List invalidatedLocations;
+ /**
+ * The next region index to assign to a location. We increment for each new location and will not reuse indices from
+ * regions that were removed.
+ */
+ private int nextRegionIndex = 0;
+
+ /**
+ * List of locations that were removed this cycle. Will be swapped each cycle with {@code invalidatedLocations} and
+ * cleared.
+ */
+ private List removedTableLocations = new ArrayList<>();
+
+ /**
+ * List of locations to invalidate at the end of the cycle. Swapped with {@code removedTableLocations} each cycle to
+ * avoid reallocating.
+ */
+ private List invalidatedLocations = new ArrayList<>();
+
+ /**
+ * Will invalidate the locations at the end of the cycle after all downstream updates are complete.
+ */
final UpdateCommitter> invalidateCommitter;
/**
@@ -184,12 +192,13 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col
}
}
- invalidatedLocations = new ArrayList<>();
invalidateCommitter = new UpdateCommitter<>(this,
ExecutionContext.getContext().getUpdateGraph(),
(ignored) -> {
- invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate);
- invalidatedLocations.clear();
+ synchronized (this) {
+ invalidatedLocations.forEach(IncludedTableLocationEntry::invalidate);
+ invalidatedLocations.clear();
+ }
});
}
@@ -223,7 +232,7 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation)
}
@Override
- public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locationKey) {
+ public synchronized boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locationKey) {
final IncludedTableLocationEntry includedLocation = includedTableLocations.remove(locationKey);
final EmptyTableLocationEntry emptyLocation = emptyTableLocations.remove(locationKey);
@@ -234,9 +243,6 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati
} else if (includedLocation != null) {
orderedIncludedTableLocations.remove(includedLocation);
removedTableLocations.add(includedLocation);
-
- // Mark this location for invalidation.
- invalidatedLocations.add(includedLocation);
invalidateCommitter.maybeActivate();
return true;
}
@@ -327,7 +333,12 @@ private TableUpdateImpl update(final boolean initializing) {
removedRowSetBuilder.appendRowSequenceWithOffset(removedLocation.location.getRowSet(), regionFirstKey);
removedRegionBuilder.appendKey(removedLocation.regionIndex);
}
- removedTableLocations.clear();
+
+ // Swap invalidatedLocations with removedTableLocations.
+ final List tmpTableLocations = removedTableLocations;
+ removedTableLocations = invalidatedLocations;
+ invalidatedLocations = tmpTableLocations;
+ Assert.eqTrue(removedTableLocations.isEmpty(), "removedTableLocations.isEmpty()");
final RowSetBuilderSequential modifiedRegionBuilder = initializing ? null : RowSetFactory.builderSequential();
@@ -370,8 +381,8 @@ private TableUpdateImpl update(final boolean initializing) {
final RowSetBuilderSequential addedRegionBuilder = RowSetFactory.builderSequential();
- final int prevMaxIndex = nextRegionIndex.get();
- final int maxIndex = nextRegionIndex.get() + (entriesToInclude.isEmpty() ? 0 : entriesToInclude.size());
+ final int prevMaxIndex = nextRegionIndex;
+ final int maxIndex = nextRegionIndex + (entriesToInclude.isEmpty() ? 0 : entriesToInclude.size());
if (!entriesToInclude.isEmpty()) {
partitioningColumnValueSources.values().forEach(
(final WritableColumnSource> wcs) -> wcs.ensureCapacity(maxIndex));
@@ -528,7 +539,7 @@ private class IncludedTableLocationEntry implements Comparable> columnLocationStates = new ArrayList<>();
/**
From 6607fc3ccb130d1bdb2a5dabeaf3747174686ea7 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Wed, 28 Aug 2024 14:30:57 -0700
Subject: [PATCH 20/56] PR comments addressed.
---
.../impl/AbstractTableLocationProvider.java | 2 +-
.../regioned/RegionedColumnSourceManager.java | 24 +++++++----
.../iceberg/util/IcebergToolsS3.java | 2 +
...ebergAutoRefreshTableLocationProvider.java | 4 +-
...ergManualRefreshTableLocationProvider.java | 11 +++--
.../IcebergStaticTableLocationProvider.java | 2 +-
.../IcebergTableLocationProviderBase.java | 6 +--
.../iceberg/util/IcebergCatalogAdapter.java | 41 +++++++++++++------
.../iceberg/util/IcebergInstructions.java | 6 +--
.../iceberg/util/IcebergTableImpl.java | 11 +++++
.../iceberg/util/IcebergUpdateMode.java | 23 +++++++----
py/server/deephaven/experimental/iceberg.py | 32 ++++++++++-----
12 files changed, 112 insertions(+), 52 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index 4afbb770116..b4afbe12cd1 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -468,7 +468,7 @@ protected void handleTableLocationKeyRemoved(
}
}
- private void maybeClearLocationForRemoval(@Nullable final Object removedLocation) {
+ private synchronized void maybeClearLocationForRemoval(@Nullable final Object removedLocation) {
if (removedLocation instanceof AbstractTableLocation) {
locationsToClear.add((AbstractTableLocation) removedLocation);
locationClearCommitter.maybeActivate();
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
index 710dcc663a7..e2f2714e688 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
@@ -322,9 +322,11 @@ public void deliverError(@NotNull final Throwable error, @Nullable final TableLi
private TableUpdateImpl update(final boolean initializing) {
final RowSetBuilderSequential addedRowSetBuilder = RowSetFactory.builderSequential();
- final RowSetBuilderSequential removedRowSetBuilder = RowSetFactory.builderSequential();
- final RowSetBuilderSequential removedRegionBuilder = RowSetFactory.builderSequential();
+ final RowSetBuilderSequential removedRowSetBuilder =
+ removedTableLocations.isEmpty() ? null : RowSetFactory.builderSequential();
+ final RowSetBuilderSequential removedRegionBuilder =
+ removedTableLocations.isEmpty() ? null : RowSetFactory.builderSequential();
// Sort the removed locations by region index, so that we can process them in order.
removedTableLocations.sort(Comparator.comparingInt(e -> e.regionIndex));
@@ -379,7 +381,8 @@ private TableUpdateImpl update(final boolean initializing) {
emptyTableLocations.removeAll(entriesToInclude);
- final RowSetBuilderSequential addedRegionBuilder = RowSetFactory.builderSequential();
+ final RowSetBuilderSequential addedRegionBuilder =
+ entriesToInclude.isEmpty() ? null : RowSetFactory.builderSequential();
final int prevMaxIndex = nextRegionIndex;
final int maxIndex = nextRegionIndex + (entriesToInclude.isEmpty() ? 0 : entriesToInclude.size());
@@ -407,14 +410,14 @@ private TableUpdateImpl update(final boolean initializing) {
addedRegionBuilder.appendKey(entry.regionIndex);
});
}
- final RowSet addedRegions = addedRegionBuilder.build();
+ final RowSet addedRegions = addedRegionBuilder == null ? RowSetFactory.empty() : addedRegionBuilder.build();
if (addedRegions.isNonempty()) {
includedLocationsTable.getRowSet().writableCast().insert(addedRegions);
}
if (initializing) {
- Assert.eqZero(prevMaxIndex, "previousNumRegions");
+ Assert.eqZero(prevMaxIndex, "prevMaxIndex");
if (isRefreshing) {
rowSetSource.startTrackingPrevValues();
includedLocationsTable.getRowSet().writableCast().initializePreviousValue();
@@ -424,7 +427,8 @@ private TableUpdateImpl update(final boolean initializing) {
}
} else {
final RowSet modifiedRegions = modifiedRegionBuilder.build();
- final RowSet removedRegions = removedRegionBuilder.build();
+ final RowSet removedRegions =
+ removedRegionBuilder == null ? RowSetFactory.empty() : removedRegionBuilder.build();
if (addedRegions.isEmpty() && modifiedRegions.isEmpty() && removedRegions.isEmpty()) {
SafeCloseable.closeAll(addedRegions, modifiedRegions, removedRegions);
} else {
@@ -438,8 +442,12 @@ private TableUpdateImpl update(final boolean initializing) {
includedLocationsTable.notifyListeners(update);
}
}
- return new TableUpdateImpl(addedRowSetBuilder.build(), removedRowSetBuilder.build(), RowSetFactory.empty(),
- RowSetShiftData.EMPTY, ModifiedColumnSet.EMPTY);
+ return new TableUpdateImpl(
+ addedRowSetBuilder.build(),
+ removedRowSetBuilder == null ? RowSetFactory.empty() : removedRowSetBuilder.build(),
+ RowSetFactory.empty(),
+ RowSetShiftData.EMPTY,
+ ModifiedColumnSet.EMPTY);
}
@Override
diff --git a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java
index 166b47e5d28..a297bfb0964 100644
--- a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java
+++ b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java
@@ -68,6 +68,8 @@ public static IcebergCatalogAdapter createS3Rest(
properties.put(S3FileIOProperties.ENDPOINT, endpointOverride);
}
+ // final org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration();
+
final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null);
final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI;
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
index 4a826d460f5..81b38edd70c 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
@@ -28,6 +28,7 @@ public class IcebergAutoRefreshTableLocationProvider locationFactory,
@NotNull final IcebergCatalogAdapter adapter,
@NotNull final TableIdentifier tableIdentifier) {
- super(tableKey, locationKeyFinder, locationFactory, null, true, adapter, tableIdentifier);
+ super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier);
}
// ------------------------------------------------------------------------------------------------------------------
@@ -108,8 +110,11 @@ private void refreshSnapshot() {
@Override
protected void activateUnderlyingDataSource() {
- refreshSnapshot();
- activationSuccessful(this);
+ if (!initialized) {
+ refreshSnapshot();
+ activationSuccessful(this);
+ initialized = true;
+ }
}
@Override
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
index a9f52487681..53a5e9fc172 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergStaticTableLocationProvider.java
@@ -26,7 +26,7 @@ public IcebergStaticTableLocationProvider(
@NotNull final IcebergBaseLayout locationKeyFinder,
@NotNull final TableLocationFactory locationFactory,
@NotNull final TableIdentifier tableIdentifier) {
- super(tableKey, locationKeyFinder, locationFactory, null, false, null, tableIdentifier);
+ super(tableKey, locationKeyFinder, locationFactory, false, null, tableIdentifier);
}
// ------------------------------------------------------------------------------------------------------------------
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
index 9e6577d500b..3e9fa7d5fa5 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
@@ -6,7 +6,6 @@
import io.deephaven.engine.table.impl.locations.*;
import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider;
import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
-import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
import io.deephaven.iceberg.util.IcebergCatalogAdapter;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.catalog.TableIdentifier;
@@ -18,7 +17,6 @@ public abstract class IcebergTableLocationProviderBase locationFactory;
- final TableDataRefreshService refreshService;
final IcebergCatalogAdapter adapter;
final TableIdentifier tableIdentifier;
@@ -26,14 +24,12 @@ public IcebergTableLocationProviderBase(
@NotNull final TK tableKey,
@NotNull final IcebergBaseLayout locationKeyFinder,
@NotNull final TableLocationFactory locationFactory,
- @Nullable final TableDataRefreshService refreshService,
final boolean isRefreshing,
@Nullable final IcebergCatalogAdapter adapter,
@NotNull final TableIdentifier tableIdentifier) {
super(tableKey, isRefreshing);
this.locationKeyFinder = locationKeyFinder;
this.locationFactory = locationFactory;
- this.refreshService = refreshService;
this.adapter = adapter;
this.tableIdentifier = tableIdentifier;
}
@@ -65,6 +61,6 @@ public IcebergTableLocationProviderBase(
@NotNull
protected TableLocation makeTableLocation(@NotNull final TableLocationKey locationKey) {
// noinspection unchecked
- return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, refreshService);
+ return locationFactory.makeLocation((TK) getKey(), (TLK) locationKey, null);
}
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index c4ed048c82e..6c0ecaee6cf 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -10,7 +10,6 @@
import io.deephaven.engine.table.impl.QueryTable;
import io.deephaven.engine.table.impl.locations.TableDataException;
import io.deephaven.engine.table.impl.locations.TableKey;
-import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder;
import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey;
import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
import io.deephaven.engine.table.impl.sources.InMemoryColumnSource;
@@ -342,13 +341,34 @@ public List listSnapshots(@NotNull final TableIdentifier tableIdentifi
/**
* List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting
* table will be static and contain the following columns:
- *
- *
Id: the snapshot identifier (can be used for updating the table or loading a specific snapshot).
- *
TimestampMs: the timestamp of the snapshot.
- *
Operation: the data operation that created this snapshot.
- *
Summary: additional information about the snapshot from the Iceberg metadata.
- *
SnapshotObject: a Java object containing the Iceberg API snapshot.
- *
+ *
+ *
+ *
Column Name
+ *
Description
+ *
+ *
+ *
Id
+ *
The snapshot identifier (can be used for updating the table or loading a specific snapshot)
+ *
+ *
+ *
TimestampMs
+ *
The timestamp of the snapshot
+ *
+ *
+ *
Operation
+ *
The data operation that created this snapshot
+ *
+ *
+ *
Summary
+ *
Additional information about the snapshot from the Iceberg metadata
+ *
+ *
+ *
SnapshotObject
+ *
A Java object containing the Iceberg API snapshot
+ *
+ *
+ *
+ *
*
* @param tableIdentifier The identifier of the table from which to gather snapshots.
* @return A list of all tables in the given namespace.
@@ -808,11 +828,6 @@ private IcebergTable readTableInternal(
updateSourceRegistrar);
}
- private static KnownLocationKeyFinder toKnownKeys(
- final IcebergBaseLayout keyFinder) {
- return KnownLocationKeyFinder.copyFrom(keyFinder, Comparator.naturalOrder());
- }
-
/**
* Returns the underlying Iceberg {@link Catalog catalog} used by this adapter.
*/
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
index 5b6438475b2..9bc776c43b1 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergInstructions.java
@@ -48,11 +48,11 @@ public static Builder builder() {
/**
* The {@link IcebergUpdateMode} mode to use when reading the Iceberg data files. Default is
- * {@link IcebergUpdateMode#STATIC}.
+ * {@link IcebergUpdateMode#staticMode()}.
*/
@Value.Default
public IcebergUpdateMode updateMode() {
- return IcebergUpdateMode.STATIC;
+ return IcebergUpdateMode.staticMode();
}
public interface Builder {
@@ -69,7 +69,7 @@ public interface Builder {
Builder putAllColumnRenames(Map entries);
@SuppressWarnings("unused")
- Builder updateMode(IcebergUpdateMode refreshing);
+ Builder updateMode(IcebergUpdateMode updateMode);
IcebergInstructions build();
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java
index a73a55d40be..02e3db6e115 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java
@@ -57,4 +57,15 @@ public void update(final long snapshotId) {
public void update(final @NotNull Snapshot snapshot) {
locationProvider.update(snapshot);
}
+
+ @Override
+ public int currentSnapshotId() {
+ // TODO: implement the TableAdapter
+ return 0;
+ }
+
+ @Override
+ public Snapshot currentSnapshot() {
+ return null;
+ }
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java
index 246e81ac65e..43da976fce9 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergUpdateMode.java
@@ -11,12 +11,10 @@
public abstract class IcebergUpdateMode {
private static final long REFRESH_INTERVAL_MS = 60_000L;
- public static final IcebergUpdateMode STATIC = builder().updateType(IcebergUpdateType.STATIC).build();
- @SuppressWarnings("unused")
- public static final IcebergUpdateMode AUTO_REFRESHING =
+ private static final IcebergUpdateMode STATIC = builder().updateType(IcebergUpdateType.STATIC).build();
+ private static final IcebergUpdateMode AUTO_REFRESHING =
builder().updateType(IcebergUpdateType.AUTO_REFRESHING).build();
- @SuppressWarnings("unused")
- public static final IcebergUpdateMode MANUAL_REFRESHING =
+ private static final IcebergUpdateMode MANUAL_REFRESHING =
builder().updateType(IcebergUpdateType.MANUAL_REFRESHING).build();
public enum IcebergUpdateType {
@@ -27,8 +25,19 @@ public static Builder builder() {
return ImmutableIcebergUpdateMode.builder();
}
- @SuppressWarnings("unused")
- public static IcebergUpdateMode autoRefreshing(final long refreshMs) {
+ public static IcebergUpdateMode staticMode() {
+ return STATIC;
+ }
+
+ public static IcebergUpdateMode manualRefreshingMode() {
+ return MANUAL_REFRESHING;
+ }
+
+ public static IcebergUpdateMode autoRefreshingMode() {
+ return AUTO_REFRESHING;
+ }
+
+ public static IcebergUpdateMode autoRefreshingMode(final long refreshMs) {
return ImmutableIcebergUpdateMode.builder()
.updateType(IcebergUpdateType.AUTO_REFRESHING)
.autoRefreshMs(refreshMs)
diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py
index 1080d2ccad9..623cccc6c16 100644
--- a/py/server/deephaven/experimental/iceberg.py
+++ b/py/server/deephaven/experimental/iceberg.py
@@ -32,9 +32,9 @@ class IcebergUpdateMode(JObjectWrapper):
"""
This class specifies the update mode for the Iceberg table to be loaded into Deephaven. The modes are:
- - `STATIC`: The table is loaded once and does not change
- - `MANUAL_REFRESHING`: The table can be manually refreshed by the user.
- - `AUTO_REFRESHING`: The table will be automatically refreshed at a specified interval (use
+ - `static()`: The table is loaded once and does not change
+ - `manual_refresh()`: The table can be manually refreshed by the user.
+ - `auto_refresh()`: The table will be automatically refreshed at a system-defined interval (also can call
`auto_refreshing(auto_refresh_ms: int)` to specify an interval rather than use the system default
of 60 seconds).
"""
@@ -44,22 +44,34 @@ def __init__(self, mode: _JIcebergUpdateMode):
self._j_object = mode
@classmethod
- def auto_refreshing(cls, auto_refresh_ms: int) -> IcebergUpdateMode:
+ def static(cls) -> IcebergUpdateMode:
+ """
+ Creates an IcebergUpdateMode with no refreshing supported.
+ """
+ return IcebergUpdateMode(_JIcebergUpdateMode.staticMode())
+
+ @classmethod
+ def manual_refresh(cls) -> IcebergUpdateMode:
+ """
+ Creates an IcebergUpdateMode with manual refreshing enabled.
+ """
+ return IcebergUpdateMode(_JIcebergUpdateMode.manualRefreshingMode())
+
+ @classmethod
+ def auto_refresh(cls, auto_refresh_ms:Optional[int] = None) -> IcebergUpdateMode:
"""
Creates an IcebergUpdateMode with auto-refreshing mode enabled using the provided refresh interval.
:param auto_refresh_ms (int): the refresh interval in milliseconds.
"""
- return IcebergUpdateMode(_JIcebergUpdateMode.autoRefreshing(auto_refresh_ms))
+ if auto_refresh_ms is None:
+ return IcebergUpdateMode(_JIcebergUpdateMode.autoRefreshingMode())
+ return IcebergUpdateMode(_JIcebergUpdateMode.autoRefreshingMode(auto_refresh_ms))
@property
def j_object(self) -> jpy.JType:
return self._j_object
-IcebergUpdateMode.STATIC=IcebergUpdateMode(_JIcebergUpdateMode.STATIC)
-IcebergUpdateMode.MANUAL_REFRESHING=IcebergUpdateMode(_JIcebergUpdateMode.MANUAL_REFRESHING)
-IcebergUpdateMode.AUTO_REFRESHING=IcebergUpdateMode(_JIcebergUpdateMode.AUTO_REFRESHING)
-
class IcebergInstructions(JObjectWrapper):
"""
@@ -119,7 +131,7 @@ def j_object(self) -> jpy.JType:
class IcebergTable(Table):
"""
- IcebergTable is a subclass of Table that allows the users to dynamically update the table with new snapshots from
+ IcebergTable is a subclass of Table that allows users to dynamically update the table with new snapshots from
the Iceberg catalog.
"""
j_object_type = _JIcebergTable
From 893336f0ed9f4d80044be882849df12f849b9282 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Fri, 30 Aug 2024 14:31:02 -0700
Subject: [PATCH 21/56] Updated to use IcebergTableAdapter and exposed in
python. Addressed PR comments.
---
.../locations/impl/AbstractTableLocation.java | 44 +-
.../impl/AbstractTableLocationProvider.java | 27 -
.../regioned/RegionedColumnSourceManager.java | 23 +
extensions/iceberg/s3/build.gradle | 1 +
.../iceberg/util/IcebergToolsS3.java | 13 +-
.../iceberg/util/IcebergToolsTest.java | 64 +-
...ebergAutoRefreshTableLocationProvider.java | 7 +-
.../iceberg/layout/IcebergBaseLayout.java | 39 +-
.../iceberg/layout/IcebergFlatLayout.java | 15 +-
.../IcebergKeyValuePartitionedLayout.java | 17 +-
...ergManualRefreshTableLocationProvider.java | 10 +-
.../IcebergTableLocationProviderBase.java | 6 +-
.../iceberg/util/IcebergCatalogAdapter.java | 516 +++------------
.../deephaven/iceberg/util/IcebergTable.java | 3 -
.../iceberg/util/IcebergTableAdapter.java | 587 ++++++++++++++++++
.../iceberg/util/IcebergTableImpl.java | 11 -
.../deephaven/iceberg/util/IcebergTools.java | 9 +-
py/server/deephaven/experimental/iceberg.py | 103 ++-
18 files changed, 925 insertions(+), 570 deletions(-)
create mode 100644 extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java
index 8391c0a774f..1c3f429fad7 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java
@@ -12,6 +12,7 @@
import io.deephaven.hash.KeyedObjectHashMap;
import io.deephaven.hash.KeyedObjectKey;
import io.deephaven.util.annotations.InternalUseOnly;
+import io.deephaven.util.referencecounting.ReferenceCounted;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
@@ -35,6 +36,8 @@ public abstract class AbstractTableLocation
private final KeyedObjectHashMap columnLocations =
new KeyedObjectHashMap<>(StringUtils.charSequenceKey());
+ private final ReferenceCounted referenceCounted;
+
@SuppressWarnings("rawtypes")
private static final AtomicReferenceFieldUpdater CACHED_DATA_INDEXES_UPDATER =
AtomicReferenceFieldUpdater.newUpdater(
@@ -58,6 +61,15 @@ protected AbstractTableLocation(@NotNull final TableKey tableKey,
super(supportsSubscriptions);
this.tableKey = Require.neqNull(tableKey, "tableKey").makeImmutable();
this.tableLocationKey = Require.neqNull(tableLocationKey, "tableLocationKey").makeImmutable();
+
+ referenceCounted = new ReferenceCounted() {
+ @Override
+ protected void onReferenceCountAtZero() {
+ // Call the location's onReferenceCountAtZero method
+ AbstractTableLocation.this.onReferenceCountAtZero();
+ }
+ };
+
}
@Override
@@ -158,7 +170,7 @@ public final ColumnLocation getColumnLocation(@NotNull final CharSequence name)
* Clear all column locations (usually because a truncated location was observed).
*/
@SuppressWarnings("unused")
- protected final void clearColumnLocations() {
+ public final void clearColumnLocations() {
columnLocations.clear();
}
@@ -229,4 +241,34 @@ public final BasicDataIndex getDataIndex(@NotNull final String... columns) {
@InternalUseOnly
@Nullable
public abstract BasicDataIndex loadDataIndex(@NotNull String... columns);
+
+ // ------------------------------------------------------------------------------------------------------------------
+ // Reference counting implementation
+ // ------------------------------------------------------------------------------------------------------------------
+
+ /**
+ * Increment the reference count by one.
+ *
+ * @throws IllegalStateException If the reference count was not successfully incremented
+ */
+ public final void incrementReferenceCount() {
+ referenceCounted.incrementReferenceCount();
+ }
+
+ /**
+ * Decrement the reference count by one, when the reference count reaches zero this location will be cleared.
+ *
+ * @throws IllegalStateException If the reference count was not successfully incremented
+ */
+ public void decrementReferenceCount() {
+ referenceCounted.decrementReferenceCount();
+ }
+
+ /**
+ * The reference count has reached zero, we can clear this location and release any resources.
+ */
+ private void onReferenceCountAtZero() {
+ handleUpdate(null, System.currentTimeMillis());
+ clearColumnLocations();
+ }
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
index b4afbe12cd1..3b49fa56634 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocationProvider.java
@@ -3,10 +3,8 @@
//
package io.deephaven.engine.table.impl.locations.impl;
-import io.deephaven.engine.context.ExecutionContext;
import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.impl.locations.*;
-import io.deephaven.engine.updategraph.UpdateCommitter;
import io.deephaven.hash.KeyedObjectHashMap;
import io.deephaven.hash.KeyedObjectKey;
import org.jetbrains.annotations.NotNull;
@@ -83,9 +81,6 @@ void removeLocationKey(TableLocationKey locationKey) {
(Collection) (Collection extends TableLocationKey>) Collections
.unmodifiableCollection(tableLocations.keySet());
- final List locationsToClear;
- final UpdateCommitter> locationClearCommitter;
-
private volatile boolean initialized;
private List partitionKeys;
@@ -101,18 +96,6 @@ protected AbstractTableLocationProvider(@NotNull final TableKey tableKey, final
super(supportsSubscriptions);
this.tableKey = tableKey.makeImmutable();
this.partitionKeys = null;
-
- locationsToClear = new ArrayList<>();
- locationClearCommitter = new UpdateCommitter<>(this,
- ExecutionContext.getContext().getUpdateGraph(),
- (ignored) -> {
- locationsToClear.forEach(location -> {
- location.handleUpdate(null, System.currentTimeMillis());
- location.clearColumnLocations();
-
- });
- locationsToClear.clear();
- });
}
/**
@@ -195,7 +178,6 @@ protected void endTransaction(@NotNull final Object token) {
for (TableLocationKey locationKey : transaction.locationsRemoved) {
final Object removedLocation = tableLocations.remove(locationKey);
if (removedLocation != null) {
- maybeClearLocationForRemoval(removedLocation);
removedImmutableKeys.add(toKeyImmutable(locationKey));
}
}
@@ -432,7 +414,6 @@ protected void handleTableLocationKeyRemoved(
@NotNull final TableLocationKey locationKey,
@Nullable final Object transactionToken) {
if (!supportsSubscriptions()) {
- maybeClearLocationForRemoval(tableLocations.remove(locationKey));
return;
}
@@ -457,7 +438,6 @@ protected void handleTableLocationKeyRemoved(
synchronized (subscriptions) {
final Object removedLocation = tableLocations.remove(locationKey);
if (removedLocation != null) {
- maybeClearLocationForRemoval(removedLocation);
if (subscriptions.deliverNotification(
Listener::handleTableLocationKeyRemoved,
locationKey.makeImmutable(),
@@ -468,13 +448,6 @@ protected void handleTableLocationKeyRemoved(
}
}
- private synchronized void maybeClearLocationForRemoval(@Nullable final Object removedLocation) {
- if (removedLocation instanceof AbstractTableLocation) {
- locationsToClear.add((AbstractTableLocation) removedLocation);
- locationClearCommitter.maybeActivate();
- }
- }
-
private void verifyPartitionKeys(@NotNull final TableLocationKey locationKey) {
if (partitionKeys == null) {
partitionKeys = new ArrayList<>(locationKey.getPartitionKeys());
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
index e2f2714e688..274f603835a 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
@@ -15,6 +15,7 @@
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableDataException;
import io.deephaven.engine.table.impl.locations.TableLocation;
+import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation;
import io.deephaven.engine.table.impl.locations.impl.TableLocationUpdateSubscriptionBuffer;
import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource;
import io.deephaven.engine.table.impl.sources.ObjectArraySource;
@@ -103,6 +104,9 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col
@ReferentialIntegrity
private final Collection retainedDataIndexes = new ArrayList<>();
+ private final List locationsToClear;
+ private final UpdateCommitter> locationClearCommitter;
+
/**
* A reference to a delayed error notifier for the {@link #includedLocationsTable}, if one is pending.
*/
@@ -192,6 +196,19 @@ public class RegionedColumnSourceManager extends LivenessArtifact implements Col
}
}
+
+ locationsToClear = new ArrayList<>();
+ locationClearCommitter = new UpdateCommitter<>(this,
+ ExecutionContext.getContext().getUpdateGraph(),
+ (ignored) -> {
+ locationsToClear.forEach(location -> {
+ location.handleUpdate(null, System.currentTimeMillis());
+ location.clearColumnLocations();
+
+ });
+ locationsToClear.clear();
+ });
+
invalidateCommitter = new UpdateCommitter<>(this,
ExecutionContext.getContext().getUpdateGraph(),
(ignored) -> {
@@ -212,6 +229,9 @@ public synchronized void addLocation(@NotNull final TableLocation tableLocation)
log.debug().append("LOCATION_ADDED:").append(tableLocation.toString()).endl();
}
emptyTableLocations.add(new EmptyTableLocationEntry(tableLocation));
+ if (tableLocation instanceof AbstractTableLocation) {
+ ((AbstractTableLocation) tableLocation).incrementReferenceCount();
+ }
} else {
// Duplicate location - not allowed
final TableLocation duplicateLocation =
@@ -656,6 +676,9 @@ private boolean pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) {
private void invalidate() {
columnLocationStates.forEach(cls -> cls.source.invalidateRegion(regionIndex));
+ if (location instanceof AbstractTableLocation) {
+ ((AbstractTableLocation) location).decrementReferenceCount();
+ }
}
@Override
diff --git a/extensions/iceberg/s3/build.gradle b/extensions/iceberg/s3/build.gradle
index bde1e84bc7f..019906dd767 100644
--- a/extensions/iceberg/s3/build.gradle
+++ b/extensions/iceberg/s3/build.gradle
@@ -1,6 +1,7 @@
plugins {
id 'java-library'
id 'io.deephaven.project.register'
+ id 'io.deephaven.hadoop-common-dependencies'
}
evaluationDependsOn Docker.registryProject('localstack')
diff --git a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java
index a297bfb0964..a552a7f3a05 100644
--- a/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java
+++ b/extensions/iceberg/s3/src/main/java/io/deephaven/iceberg/util/IcebergToolsS3.java
@@ -4,6 +4,7 @@
package io.deephaven.iceberg.util;
import com.google.common.base.Strings;
+import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.CatalogUtil;
import org.apache.iceberg.aws.AwsClientProperties;
@@ -68,14 +69,11 @@ public static IcebergCatalogAdapter createS3Rest(
properties.put(S3FileIOProperties.ENDPOINT, endpointOverride);
}
- // final org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration();
-
- final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null);
-
final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI;
+ catalog.setConf(new Configuration());
catalog.initialize(catalogName, properties);
- return new IcebergCatalogAdapter(catalog, fileIO);
+ return new IcebergCatalogAdapter(catalog);
}
/**
@@ -103,11 +101,10 @@ public static IcebergCatalogAdapter createGlue(
properties.put(CatalogProperties.URI, catalogURI);
properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation);
- final FileIO fileIO = CatalogUtil.loadFileIO(S3_FILE_IO_CLASS, properties, null);
-
final String catalogName = name != null ? name : "IcebergCatalog-" + catalogURI;
+ catalog.setConf(new Configuration());
catalog.initialize(catalogName, properties);
- return new IcebergCatalogAdapter(catalog, fileIO);
+ return new IcebergCatalogAdapter(catalog);
}
}
diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java
index 44a17942cdf..07b6b127cdb 100644
--- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java
+++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java
@@ -44,8 +44,8 @@
import java.util.stream.Collectors;
import static io.deephaven.iceberg.util.IcebergCatalogAdapter.NAMESPACE_DEFINITION;
-import static io.deephaven.iceberg.util.IcebergCatalogAdapter.SNAPSHOT_DEFINITION;
import static io.deephaven.iceberg.util.IcebergCatalogAdapter.TABLES_DEFINITION;
+import static io.deephaven.iceberg.util.IcebergTableAdapter.SNAPSHOT_DEFINITION;
public abstract class IcebergToolsTest {
@@ -187,7 +187,7 @@ public void tearDown() throws ExecutionException, InterruptedException {
@Test
public void testListNamespaces() {
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
final Collection namespaces = adapter.listNamespaces();
final Collection namespaceNames =
@@ -197,14 +197,14 @@ public void testListNamespaces() {
Assert.eqTrue(namespaceNames.contains("sales"), "namespaceNames.contains(sales)");
Assert.eqTrue(namespaceNames.contains("sample"), "namespaceNames.contains(sample)");
- final Table table = adapter.listNamespacesAsTable();
+ final Table table = adapter.namespaces();
Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog");
Assert.equals(table.getDefinition(), "table.getDefinition()", NAMESPACE_DEFINITION);
}
@Test
public void testListTables() {
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
@@ -216,19 +216,19 @@ public void testListTables() {
Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_single")), "tables.contains(sales_single)");
Assert.eqTrue(tables.contains(TableIdentifier.of(ns, "sales_renamed")), "tables.contains(sales_renamed)");
- Table table = adapter.listTablesAsTable(ns);
+ Table table = adapter.tables(ns);
Assert.eq(table.size(), "table.size()", 4, "4 tables in the namespace");
Assert.equals(table.getDefinition(), "table.getDefinition()", TABLES_DEFINITION);
// Test the string versions of the methods
- table = adapter.listTablesAsTable("sales");
+ table = adapter.tables("sales");
Assert.eq(table.size(), "table.size()", 4, "4 tables in the namespace");
Assert.equals(table.getDefinition(), "table.getDefinition()", TABLES_DEFINITION);
}
@Test
public void testListSnapshots() {
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
final TLongArrayList snapshotIds = new TLongArrayList();
final TableIdentifier tableIdentifier = TableIdentifier.of("sales", "sales_multi");
@@ -242,12 +242,12 @@ public void testListSnapshots() {
Assert.eqTrue(snapshotIds.contains(3247344357341484163L), "snapshots.contains(3247344357341484163L)");
Assert.eqTrue(snapshotIds.contains(1792185872197984875L), "snapshots.contains(1792185872197984875L)");
- Table table = adapter.listSnapshotsAsTable(tableIdentifier);
+ Table table = adapter.snapshots(tableIdentifier);
Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi");
Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION);
// Test the string versions of the methods
- table = adapter.listSnapshotsAsTable("sales.sales_multi");
+ table = adapter.snapshots("sales.sales_multi");
Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi");
Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION);
}
@@ -257,7 +257,7 @@ public void testOpenTableA() throws ExecutionException, InterruptedException, Ti
uploadSalesPartitioned();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -280,7 +280,7 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti
uploadSalesMulti();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi");
@@ -302,7 +302,7 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti
uploadSalesSingle();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_single");
@@ -324,7 +324,7 @@ public void testOpenTableC() throws ExecutionException, InterruptedException, Ti
public void testOpenTableS3Only() throws ExecutionException, InterruptedException, TimeoutException {
uploadSalesPartitioned();
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -345,7 +345,7 @@ public void testOpenTableDefinition() throws ExecutionException, InterruptedExce
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -373,7 +373,7 @@ public void testOpenTablePartitionTypeException() {
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -417,7 +417,7 @@ public void testOpenTableDefinitionRename() throws ExecutionException, Interrupt
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -447,7 +447,7 @@ public void testSkippedPartitioningColumn() throws ExecutionException, Interrupt
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -477,7 +477,7 @@ public void testReorderedPartitioningColumn() throws ExecutionException, Interru
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -498,7 +498,7 @@ public void testZeroPartitioningColumns() throws ExecutionException, Interrupted
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -526,7 +526,7 @@ public void testIncorrectPartitioningColumns() throws ExecutionException, Interr
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -562,7 +562,7 @@ public void testMissingPartitioningColumns() {
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -591,7 +591,7 @@ public void testOpenTableColumnRename() throws ExecutionException, InterruptedEx
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -610,7 +610,7 @@ public void testOpenTableColumnLegalization() throws ExecutionException, Interru
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_renamed");
@@ -633,7 +633,7 @@ public void testOpenTableColumnLegalizationRename()
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_renamed");
@@ -665,7 +665,7 @@ public void testOpenTableColumnLegalizationPartitionException() {
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -696,7 +696,7 @@ public void testOpenTableColumnRenamePartitioningColumns()
.build();
final IcebergCatalogAdapter adapter =
- IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_partitioned");
@@ -720,7 +720,7 @@ public void testOpenTableColumnRenamePartitioningColumns()
public void testOpenTableSnapshot() throws ExecutionException, InterruptedException, TimeoutException {
uploadSalesMulti();
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi");
@@ -752,7 +752,7 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept
public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedException, TimeoutException {
uploadSalesMulti();
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi");
@@ -799,7 +799,7 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx
public void testOpenAllTypesTable() throws ExecutionException, InterruptedException, TimeoutException {
uploadAllTypes();
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sample");
final TableIdentifier tableId = TableIdentifier.of(ns, "all_types");
@@ -812,7 +812,7 @@ public void testOpenAllTypesTable() throws ExecutionException, InterruptedExcept
@Test
public void testTableDefinition() {
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi");
@@ -837,7 +837,7 @@ public void testTableDefinition() {
@Test
public void testTableDefinitionTable() {
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
final Namespace ns = Namespace.of("sales");
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi");
@@ -870,7 +870,7 @@ public void testTableDefinitionTable() {
@Test
public void testTableDefinitionWithInstructions() {
- final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog, resourceFileIO);
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
IcebergInstructions localInstructions = IcebergInstructions.builder()
.dataInstructions(instructions.dataInstructions().get())
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
index 81b38edd70c..bafbcdff872 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergAutoRefreshTableLocationProvider.java
@@ -8,7 +8,7 @@
import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
-import io.deephaven.iceberg.util.IcebergCatalogAdapter;
+import io.deephaven.iceberg.util.IcebergTableAdapter;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.catalog.TableIdentifier;
import org.jetbrains.annotations.NotNull;
@@ -39,7 +39,7 @@ public IcebergAutoRefreshTableLocationProvider(
@NotNull final TableLocationFactory locationFactory,
@NotNull final TableDataRefreshService refreshService,
final long refreshIntervalMs,
- @NotNull final IcebergCatalogAdapter adapter,
+ @NotNull final IcebergTableAdapter adapter,
@NotNull final TableIdentifier tableIdentifier) {
super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier);
@@ -59,7 +59,8 @@ public String getImplementationName() {
@Override
public synchronized void refresh() {
- final Snapshot latestSnapshot = adapter.getCurrentSnapshot(tableIdentifier);
+ adapter.refresh();
+ final Snapshot latestSnapshot = adapter.currentSnapshot();
if (latestSnapshot.sequenceNumber() > locationKeyFinder.snapshot.sequenceNumber()) {
locationKeyFinder.snapshot = latestSnapshot;
refreshSnapshot();
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java
index 70a65230f29..1f1d9ba0546 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergBaseLayout.java
@@ -10,9 +10,9 @@
import io.deephaven.iceberg.location.IcebergTableLocationKey;
import io.deephaven.iceberg.location.IcebergTableParquetLocationKey;
import io.deephaven.iceberg.util.IcebergInstructions;
+import io.deephaven.iceberg.util.IcebergTableAdapter;
import io.deephaven.parquet.table.ParquetInstructions;
import org.apache.iceberg.*;
-import org.apache.iceberg.io.FileIO;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
@@ -24,19 +24,15 @@
public abstract class IcebergBaseLayout implements TableLocationKeyFinder {
/**
- * The {@link TableDefinition} that will be used for the table.
+ * The {@link IcebergTableAdapter} that will be used to access the table.
*/
- final TableDefinition tableDef;
-
- /**
- * The Iceberg {@link Table} to discover locations for.
- */
- final Table table;
+ final IcebergTableAdapter tableAdapter;
/**
- * The {@link FileIO} to use for passing to the catalog reading manifest data files.
+ * The {@link TableDefinition} that will be used for life of this table. Although Iceberg table schema may change,
+ * schema changes are not supported in Deephaven.
*/
- final FileIO fileIO;
+ final TableDefinition tableDef;
/**
* The instructions for customizations while reading.
@@ -87,28 +83,24 @@ protected IcebergTableLocationKey locationKey(
return new IcebergTableParquetLocationKey(fileUri, 0, partitions, parquetInstructions);
}
throw new UnsupportedOperationException(String.format("%s:%d - an unsupported file format %s for URI '%s'",
- table, snapshot.snapshotId(), format, fileUri));
+ tableAdapter, snapshot.snapshotId(), format, fileUri));
}
/**
- * @param tableDef The {@link TableDefinition} that will be used for the table.
- * @param table The {@link Table} to discover locations for.
+ * @param tableAdapter The {@link IcebergTableAdapter} that will be used to access the table.
* @param tableSnapshot The {@link Snapshot} from which to discover data files.
- * @param fileIO The file IO to use for reading manifest data files.
* @param instructions The instructions for customizations while reading.
*/
public IcebergBaseLayout(
- @NotNull final TableDefinition tableDef,
- @NotNull final Table table,
+ @NotNull final IcebergTableAdapter tableAdapter,
@NotNull final Snapshot tableSnapshot,
- @NotNull final FileIO fileIO,
@NotNull final IcebergInstructions instructions) {
- this.tableDef = tableDef;
- this.table = table;
+ this.tableAdapter = tableAdapter;
this.snapshot = tableSnapshot;
- this.fileIO = fileIO;
this.instructions = instructions;
+ this.tableDef = tableAdapter.definition(tableSnapshot, instructions);
+
this.cache = new HashMap<>();
}
@@ -116,9 +108,10 @@ public IcebergBaseLayout(
@Override
public synchronized void findKeys(@NotNull final Consumer locationKeyObserver) {
+ final Table table = tableAdapter.icebergTable();
try {
// Retrieve the manifest files from the snapshot
- final List manifestFiles = snapshot.allManifests(fileIO);
+ final List manifestFiles = snapshot.allManifests(table.io());
for (final ManifestFile manifestFile : manifestFiles) {
// Currently only can process manifest files with DATA content type.
if (manifestFile.content() != ManifestContent.DATA) {
@@ -126,7 +119,7 @@ public synchronized void findKeys(@NotNull final Consumer reader = ManifestFiles.read(manifestFile, fileIO)) {
+ try (final ManifestReader reader = ManifestFiles.read(manifestFile, table.io())) {
for (DataFile df : reader) {
final URI fileUri = FileUtils.convertToURI(df.path().toString(), false);
final IcebergTableLocationKey locationKey =
@@ -139,7 +132,7 @@ public synchronized void findKeys(@NotNull final Consumer type;
final int index;
@@ -39,21 +38,17 @@ public ColumnData(String name, Class> type, int index) {
private final List outputPartitioningColumns;
/**
- * @param tableDef The {@link TableDefinition} that will be used for the table.
- * @param table The {@link Table} to discover locations for.
+ * @param tableAdapter The {@link IcebergTableAdapter} that will be used to access the table.
* @param tableSnapshot The {@link Snapshot} from which to discover data files.
- * @param fileIO The file IO to use for reading manifest data files.
* @param partitionSpec The Iceberg {@link PartitionSpec partition spec} for the table.
* @param instructions The instructions for customizations while reading.
*/
public IcebergKeyValuePartitionedLayout(
- @NotNull final TableDefinition tableDef,
- @NotNull final org.apache.iceberg.Table table,
+ @NotNull final IcebergTableAdapter tableAdapter,
@NotNull final org.apache.iceberg.Snapshot tableSnapshot,
- @NotNull final FileIO fileIO,
@NotNull final PartitionSpec partitionSpec,
@NotNull final IcebergInstructions instructions) {
- super(tableDef, table, tableSnapshot, fileIO, instructions);
+ super(tableAdapter, tableSnapshot, instructions);
// We can assume due to upstream validation that there are no duplicate names (after renaming) that are included
// in the output definition, so we can ignore duplicates.
@@ -81,7 +76,7 @@ public IcebergKeyValuePartitionedLayout(
@Override
public String toString() {
- return IcebergKeyValuePartitionedLayout.class.getSimpleName() + '[' + table.name() + ']';
+ return IcebergKeyValuePartitionedLayout.class.getSimpleName() + '[' + tableAdapter + ']';
}
@Override
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
index bb1f4b50fdc..f2e5e8fb7e2 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergManualRefreshTableLocationProvider.java
@@ -6,7 +6,7 @@
import io.deephaven.engine.table.impl.locations.*;
import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder;
-import io.deephaven.iceberg.util.IcebergCatalogAdapter;
+import io.deephaven.iceberg.util.IcebergTableAdapter;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.catalog.TableIdentifier;
import org.jetbrains.annotations.NotNull;
@@ -33,7 +33,7 @@ public IcebergManualRefreshTableLocationProvider(
@NotNull final TK tableKey,
@NotNull final IcebergBaseLayout locationKeyFinder,
@NotNull final TableLocationFactory locationFactory,
- @NotNull final IcebergCatalogAdapter adapter,
+ @NotNull final IcebergTableAdapter adapter,
@NotNull final TableIdentifier tableIdentifier) {
super(tableKey, locationKeyFinder, locationFactory, true, adapter, tableIdentifier);
}
@@ -55,12 +55,14 @@ public synchronized void refresh() {
@Override
public synchronized void update() {
- update(adapter.getCurrentSnapshot(tableIdentifier));
+ adapter.refresh();
+ update(adapter.currentSnapshot());
}
@Override
public synchronized void update(final long snapshotId) {
- final List snapshots = adapter.listSnapshots(tableIdentifier);
+ adapter.refresh();
+ final List snapshots = adapter.listSnapshots();
final Snapshot snapshot = snapshots.stream()
.filter(s -> s.snapshotId() == snapshotId).findFirst()
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
index 3e9fa7d5fa5..9ec4eeb18b0 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/layout/IcebergTableLocationProviderBase.java
@@ -6,7 +6,7 @@
import io.deephaven.engine.table.impl.locations.*;
import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocationProvider;
import io.deephaven.engine.table.impl.locations.impl.TableLocationFactory;
-import io.deephaven.iceberg.util.IcebergCatalogAdapter;
+import io.deephaven.iceberg.util.IcebergTableAdapter;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.catalog.TableIdentifier;
import org.jetbrains.annotations.NotNull;
@@ -17,7 +17,7 @@ public abstract class IcebergTableLocationProviderBase locationFactory;
- final IcebergCatalogAdapter adapter;
+ final IcebergTableAdapter adapter;
final TableIdentifier tableIdentifier;
public IcebergTableLocationProviderBase(
@@ -25,7 +25,7 @@ public IcebergTableLocationProviderBase(
@NotNull final IcebergBaseLayout locationKeyFinder,
@NotNull final TableLocationFactory locationFactory,
final boolean isRefreshing,
- @Nullable final IcebergCatalogAdapter adapter,
+ @Nullable final IcebergTableAdapter adapter,
@NotNull final TableIdentifier tableIdentifier) {
super(tableKey, isRefreshing);
this.locationKeyFinder = locationKeyFinder;
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
index 6c0ecaee6cf..6a396a52bd8 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergCatalogAdapter.java
@@ -3,44 +3,20 @@
//
package io.deephaven.iceberg.util;
-import io.deephaven.api.util.NameValidator;
-import io.deephaven.engine.context.ExecutionContext;
import io.deephaven.engine.rowset.RowSetFactory;
import io.deephaven.engine.table.*;
import io.deephaven.engine.table.impl.QueryTable;
-import io.deephaven.engine.table.impl.locations.TableDataException;
-import io.deephaven.engine.table.impl.locations.TableKey;
-import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey;
-import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
import io.deephaven.engine.table.impl.sources.InMemoryColumnSource;
-import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl;
-import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
-import io.deephaven.iceberg.layout.*;
-import io.deephaven.engine.util.TableTools;
-import io.deephaven.iceberg.layout.IcebergFlatLayout;
-import io.deephaven.iceberg.layout.IcebergKeyValuePartitionedLayout;
-import io.deephaven.iceberg.location.IcebergTableLocationFactory;
-import io.deephaven.iceberg.location.IcebergTableLocationKey;
-import io.deephaven.time.DateTimeUtils;
import io.deephaven.util.annotations.VisibleForTesting;
-import org.apache.iceberg.PartitionField;
-import org.apache.iceberg.PartitionSpec;
-import org.apache.iceberg.Schema;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.SupportsNamespaces;
import org.apache.iceberg.catalog.TableIdentifier;
-import org.apache.iceberg.io.FileIO;
-import org.apache.iceberg.types.Type;
-import org.apache.iceberg.types.Types;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
-import java.time.Instant;
-import java.time.LocalDateTime;
import java.util.*;
-import java.util.stream.Collectors;
public class IcebergCatalogAdapter {
@@ -55,142 +31,15 @@ public class IcebergCatalogAdapter {
ColumnDefinition.ofString("TableName"),
ColumnDefinition.fromGenericType("TableIdentifierObject", TableIdentifier.class));
- @VisibleForTesting
- static final TableDefinition SNAPSHOT_DEFINITION = TableDefinition.of(
- ColumnDefinition.ofLong("Id"),
- ColumnDefinition.ofTime("Timestamp"),
- ColumnDefinition.ofString("Operation"),
- ColumnDefinition.fromGenericType("Summary", Map.class),
- ColumnDefinition.fromGenericType("SnapshotObject", Snapshot.class));
-
private final Catalog catalog;
- private final FileIO fileIO;
/**
* Construct an IcebergCatalogAdapter from a catalog and file IO.
*/
- IcebergCatalogAdapter(
- @NotNull final Catalog catalog,
- @NotNull final FileIO fileIO) {
+ IcebergCatalogAdapter(@NotNull final Catalog catalog) {
this.catalog = catalog;
- this.fileIO = fileIO;
- }
-
- /**
- * Create a single {@link TableDefinition} from a given Schema, PartitionSpec, and TableDefinition. Takes into
- * account {@link Map<> column rename instructions}
- *
- * @param schema The schema of the table.
- * @param partitionSpec The partition specification of the table.
- * @param userTableDef The table definition.
- * @param columnRename The map for renaming columns.
- * @return The generated TableDefinition.
- */
- private static TableDefinition fromSchema(
- @NotNull final Schema schema,
- @NotNull final PartitionSpec partitionSpec,
- @Nullable final TableDefinition userTableDef,
- @NotNull final Map columnRename) {
-
- final Set columnNames = userTableDef != null
- ? userTableDef.getColumnNameSet()
- : null;
-
- final Set partitionNames =
- partitionSpec.fields().stream()
- .map(PartitionField::name)
- .map(colName -> columnRename.getOrDefault(colName, colName))
- .collect(Collectors.toSet());
-
- final List> columns = new ArrayList<>();
-
- for (final Types.NestedField field : schema.columns()) {
- final String name = columnRename.getOrDefault(field.name(), field.name());
- // Skip columns that are not in the provided table definition.
- if (columnNames != null && !columnNames.contains(name)) {
- continue;
- }
- final Type type = field.type();
- final io.deephaven.qst.type.Type> qstType = convertPrimitiveType(type);
- final ColumnDefinition> column;
- if (partitionNames.contains(name)) {
- column = ColumnDefinition.of(name, qstType).withPartitioning();
- } else {
- column = ColumnDefinition.of(name, qstType);
- }
- columns.add(column);
- }
-
- final TableDefinition icebergTableDef = TableDefinition.of(columns);
- if (userTableDef == null) {
- return icebergTableDef;
- }
-
- // If the user supplied a table definition, make sure it's fully compatible.
- final TableDefinition tableDef = icebergTableDef.checkCompatibility(userTableDef);
-
- // Ensure that the user has not marked non-partitioned columns as partitioned.
- final Set userPartitionColumns = userTableDef.getPartitioningColumns().stream()
- .map(ColumnDefinition::getName)
- .collect(Collectors.toSet());
- final Set partitionColumns = tableDef.getPartitioningColumns().stream()
- .map(ColumnDefinition::getName)
- .collect(Collectors.toSet());
-
- // The working partitioning column set must be a super-set of the user-supplied set.
- if (!partitionColumns.containsAll(userPartitionColumns)) {
- final Set invalidColumns = new HashSet<>(userPartitionColumns);
- invalidColumns.removeAll(partitionColumns);
-
- throw new TableDataException("The following columns are not partitioned in the Iceberg table: " +
- invalidColumns);
- }
- return tableDef;
}
- /**
- * Convert an Iceberg data type to a Deephaven type.
- *
- * @param icebergType The Iceberg data type to be converted.
- * @return The converted Deephaven type.
- */
- static io.deephaven.qst.type.Type> convertPrimitiveType(@NotNull final Type icebergType) {
- final Type.TypeID typeId = icebergType.typeId();
- switch (typeId) {
- case BOOLEAN:
- return io.deephaven.qst.type.Type.booleanType().boxedType();
- case DOUBLE:
- return io.deephaven.qst.type.Type.doubleType();
- case FLOAT:
- return io.deephaven.qst.type.Type.floatType();
- case INTEGER:
- return io.deephaven.qst.type.Type.intType();
- case LONG:
- return io.deephaven.qst.type.Type.longType();
- case STRING:
- return io.deephaven.qst.type.Type.stringType();
- case TIMESTAMP:
- final Types.TimestampType timestampType = (Types.TimestampType) icebergType;
- return timestampType.shouldAdjustToUTC()
- ? io.deephaven.qst.type.Type.find(Instant.class)
- : io.deephaven.qst.type.Type.find(LocalDateTime.class);
- case DATE:
- return io.deephaven.qst.type.Type.find(java.time.LocalDate.class);
- case TIME:
- return io.deephaven.qst.type.Type.find(java.time.LocalTime.class);
- case DECIMAL:
- return io.deephaven.qst.type.Type.find(java.math.BigDecimal.class);
- case FIXED: // Fall through
- case BINARY:
- return io.deephaven.qst.type.Type.find(byte[].class);
- case UUID: // Fall through
- case STRUCT: // Fall through
- case LIST: // Fall through
- case MAP: // Fall through
- default:
- throw new TableDataException("Unsupported iceberg column type " + typeId.name());
- }
- }
/**
* List all {@link Namespace namespaces} in the catalog. This method is only supported if the catalog implements
@@ -225,8 +74,8 @@ public List listNamespaces(@NotNull final Namespace namespace) {
*
* @return A {@link Table table} of all namespaces.
*/
- public Table listNamespacesAsTable() {
- return listNamespacesAsTable(Namespace.empty());
+ public Table namespaces() {
+ return namespaces(Namespace.empty());
}
/**
@@ -235,7 +84,7 @@ public Table listNamespacesAsTable() {
*
* @return A {@link Table table} of all namespaces.
*/
- public Table listNamespacesAsTable(@NotNull final Namespace namespace) {
+ public Table namespaces(@NotNull final Namespace namespace) {
final List namespaces = listNamespaces(namespace);
final long size = namespaces.size();
@@ -269,8 +118,8 @@ public Table listNamespacesAsTable(@NotNull final Namespace namespace) {
* @return A {@link Table table} of all namespaces.
*/
@SuppressWarnings("unused")
- public Table listNamespacesAsTable(@NotNull final String... namespace) {
- return listNamespacesAsTable(Namespace.of(namespace));
+ public Table namespaces(@NotNull final String... namespace) {
+ return namespaces(Namespace.of(namespace));
}
/**
@@ -290,7 +139,7 @@ public List listTables(@NotNull final Namespace namespace) {
* @param namespace The namespace from which to gather the tables
* @return A list of all tables in the given namespace.
*/
- public Table listTablesAsTable(@NotNull final Namespace namespace) {
+ public Table tables(@NotNull final Namespace namespace) {
final List tableIdentifiers = listTables(namespace);
final long size = tableIdentifiers.size();
@@ -322,8 +171,35 @@ public Table listTablesAsTable(@NotNull final Namespace namespace) {
return new QueryTable(TABLES_DEFINITION, RowSetFactory.flat(size).toTracking(), columnSourceMap);
}
- public Table listTablesAsTable(@NotNull final String... namespace) {
- return listTablesAsTable(Namespace.of(namespace));
+ public Table tables(@NotNull final String... namespace) {
+ return tables(Namespace.of(namespace));
+ }
+
+ /**
+ * Load an Iceberg table from the catalog.
+ *
+ * @param tableIdentifier The identifier of the table to load.
+ * @return The {@link IcebergTableAdapter table adapter} for the Iceberg table.
+ */
+ public IcebergTableAdapter loadTable(@NotNull final TableIdentifier tableIdentifier) {
+ // Load the table from the catalog.
+ final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier);
+ if (table == null) {
+ throw new IllegalArgumentException("Table not found: " + tableIdentifier);
+ }
+ return new IcebergTableAdapter(tableIdentifier, table);
+ }
+
+ /**
+ * List all {@link Snapshot snapshots} of a given Iceberg table.
+ *
+ * @param tableIdentifier The identifier of the table from which to gather snapshots.
+ * @return A list of all snapshots of the given table.
+ */
+ @Deprecated(forRemoval = true)
+ public List listSnapshots(@NotNull final String tableIdentifier) {
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.listSnapshots();
}
/**
@@ -332,10 +208,10 @@ public Table listTablesAsTable(@NotNull final String... namespace) {
* @param tableIdentifier The identifier of the table from which to gather snapshots.
* @return A list of all snapshots of the given table.
*/
+ @Deprecated(forRemoval = true)
public List listSnapshots(@NotNull final TableIdentifier tableIdentifier) {
- final List snapshots = new ArrayList<>();
- catalog.loadTable(tableIdentifier).snapshots().forEach(snapshots::add);
- return snapshots;
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.listSnapshots();
}
/**
@@ -373,46 +249,10 @@ public List listSnapshots(@NotNull final TableIdentifier tableIdentifi
* @param tableIdentifier The identifier of the table from which to gather snapshots.
* @return A list of all tables in the given namespace.
*/
- public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier) {
- final List snapshots = listSnapshots(tableIdentifier);
- final long size = snapshots.size();
-
- // Create and return a table containing the namespaces as strings
- final Map> columnSourceMap = new LinkedHashMap<>();
-
- // Create the column source(s)
- final long[] idArr = new long[(int) size];
- columnSourceMap.put("Id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr, long.class, null));
-
- final long[] timestampArr = new long[(int) size];
- columnSourceMap.put("Timestamp",
- InMemoryColumnSource.getImmutableMemoryColumnSource(timestampArr, Instant.class, null));
-
- final String[] operatorArr = new String[(int) size];
- columnSourceMap.put("Operation",
- InMemoryColumnSource.getImmutableMemoryColumnSource(operatorArr, String.class, null));
-
- final Map[] summaryArr = new Map[(int) size];
- columnSourceMap.put("Summary",
- InMemoryColumnSource.getImmutableMemoryColumnSource(summaryArr, Map.class, null));
-
- final Snapshot[] snapshotArr = new Snapshot[(int) size];
- columnSourceMap.put("SnapshotObject",
- InMemoryColumnSource.getImmutableMemoryColumnSource(snapshotArr, Snapshot.class, null));
-
- // Populate the column source(s)
- for (int i = 0; i < size; i++) {
- final Snapshot snapshot = snapshots.get(i);
- idArr[i] = snapshot.snapshotId();
- // Provided as millis from epoch, convert to nanos
- timestampArr[i] = DateTimeUtils.millisToNanos(snapshot.timestampMillis());
- operatorArr[i] = snapshot.operation();
- summaryArr[i] = snapshot.summary();
- snapshotArr[i] = snapshot;
- }
-
- // Create and return the table
- return new QueryTable(SNAPSHOT_DEFINITION, RowSetFactory.flat(size).toTracking(), columnSourceMap);
+ @Deprecated(forRemoval = true)
+ public Table snapshots(@NotNull final TableIdentifier tableIdentifier) {
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.snapshots();
}
/**
@@ -422,77 +262,9 @@ public Table listSnapshotsAsTable(@NotNull final TableIdentifier tableIdentifier
* @param tableIdentifier The identifier of the table from which to gather snapshots.
* @return A list of all tables in the given namespace.
*/
- public Table listSnapshotsAsTable(@NotNull final String tableIdentifier) {
- return listSnapshotsAsTable(TableIdentifier.parse(tableIdentifier));
- }
-
- /**
- * Retrieve a specific {@link Snapshot snapshot} of an Iceberg table.
- *
- * @param tableIdentifier The identifier of the table from which to load the snapshot.
- * @param snapshotId The identifier of the snapshot to load.
- *
- * @return An Optional containing the requested snapshot if it exists.
- */
- private Optional getSnapshot(@NotNull final TableIdentifier tableIdentifier, final long snapshotId) {
- return listSnapshots(tableIdentifier).stream()
- .filter(snapshot -> snapshot.snapshotId() == snapshotId)
- .findFirst();
- }
-
- /**
- * Get the current {@link Snapshot snapshot} of a given Iceberg table.
- *
- * @param tableIdentifier The identifier of the table.
- * @return The current snapshot of the table.
- */
- public Snapshot getCurrentSnapshot(@NotNull final TableIdentifier tableIdentifier) {
- final List snapshots = listSnapshots(tableIdentifier);
- if (snapshots.isEmpty()) {
- throw new IllegalStateException("No snapshots found for table " + tableIdentifier);
- }
- return snapshots.get(snapshots.size() - 1);
- }
-
- /**
- * Get a legalized column rename map from a table schema and user instructions.
- */
- private Map getRenameColumnMap(
- @NotNull final org.apache.iceberg.Table table,
- @NotNull final Schema schema,
- @NotNull final IcebergInstructions instructions) {
-
- final Set takenNames = new HashSet<>();
-
- // Map all the column names in the schema to their legalized names.
- final Map legalizedColumnRenames = new HashMap<>();
-
- // Validate user-supplied names meet legalization instructions
- for (final Map.Entry entry : instructions.columnRenames().entrySet()) {
- final String destinationName = entry.getValue();
- if (!NameValidator.isValidColumnName(destinationName)) {
- throw new TableDataException(
- String.format("%s - invalid column name provided (%s)", table, destinationName));
- }
- // Add these renames to the legalized list.
- legalizedColumnRenames.put(entry.getKey(), destinationName);
- takenNames.add(destinationName);
- }
-
- for (final Types.NestedField field : schema.columns()) {
- final String name = field.name();
- // Do we already have a valid rename for this column from the user or a partitioned column?
- if (!legalizedColumnRenames.containsKey(name)) {
- final String legalizedName =
- NameValidator.legalizeColumnName(name, s -> s.replace(" ", "_"), takenNames);
- if (!legalizedName.equals(name)) {
- legalizedColumnRenames.put(name, legalizedName);
- takenNames.add(legalizedName);
- }
- }
- }
-
- return legalizedColumnRenames;
+ @Deprecated(forRemoval = true)
+ public Table snapshots(@NotNull final String tableIdentifier) {
+ return snapshots(TableIdentifier.parse(tableIdentifier));
}
/**
@@ -503,12 +275,12 @@ private Map getRenameColumnMap(
* @param instructions The instructions for customizations while reading
* @return The table definition
*/
+ @Deprecated(forRemoval = true)
public TableDefinition getTableDefinition(
@NotNull final String tableIdentifier,
@Nullable final IcebergInstructions instructions) {
- final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier);
- // Load the table from the catalog.
- return getTableDefinition(tableId, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.definition(instructions);
}
/**
@@ -519,11 +291,12 @@ public TableDefinition getTableDefinition(
* @param instructions The instructions for customizations while reading
* @return The table definition
*/
+ @Deprecated(forRemoval = true)
public TableDefinition getTableDefinition(
@NotNull final TableIdentifier tableIdentifier,
@Nullable final IcebergInstructions instructions) {
- // Load the table from the catalog.
- return getTableDefinitionInternal(tableIdentifier, null, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.definition(instructions);
}
/**
@@ -535,19 +308,13 @@ public TableDefinition getTableDefinition(
* @param instructions The instructions for customizations while reading
* @return The table definition
*/
+ @Deprecated(forRemoval = true)
public TableDefinition getTableDefinition(
@NotNull final String tableIdentifier,
final long snapshotId,
@Nullable final IcebergInstructions instructions) {
- final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier);
-
- // Find the snapshot with the given snapshot id
- final Snapshot tableSnapshot =
- getSnapshot(tableId, snapshotId).orElseThrow(() -> new IllegalArgumentException(
- "Snapshot with id " + snapshotId + " not found for table " + tableId));
-
- // Load the table from the catalog.
- return getTableDefinition(tableId, tableSnapshot, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.definition(snapshotId, instructions);
}
/**
@@ -559,12 +326,13 @@ public TableDefinition getTableDefinition(
* @param instructions The instructions for customizations while reading
* @return The table definition
*/
+ @Deprecated(forRemoval = true)
public TableDefinition getTableDefinition(
@NotNull final TableIdentifier tableIdentifier,
@Nullable final Snapshot tableSnapshot,
@Nullable final IcebergInstructions instructions) {
- // Load the table from the catalog.
- return getTableDefinitionInternal(tableIdentifier, tableSnapshot, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.definition(tableSnapshot, instructions);
}
/**
@@ -575,11 +343,12 @@ public TableDefinition getTableDefinition(
* @param instructions The instructions for customizations while reading
* @return The table definition as a Deephaven table
*/
+ @Deprecated(forRemoval = true)
public Table getTableDefinitionTable(
@NotNull final String tableIdentifier,
@Nullable final IcebergInstructions instructions) {
- final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier);
- return getTableDefinitionTable(tableId, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.definitionTable(instructions);
}
/**
@@ -590,11 +359,12 @@ public Table getTableDefinitionTable(
* @param instructions The instructions for customizations while reading
* @return The table definition as a Deephaven table
*/
+ @Deprecated(forRemoval = true)
public Table getTableDefinitionTable(
@NotNull final TableIdentifier tableIdentifier,
@Nullable final IcebergInstructions instructions) {
- final TableDefinition definition = getTableDefinition(tableIdentifier, instructions);
- return TableTools.metaTable(definition);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.definitionTable(instructions);
}
/**
@@ -606,18 +376,13 @@ public Table getTableDefinitionTable(
* @param instructions The instructions for customizations while reading
* @return The table definition as a Deephaven table
*/
+ @Deprecated(forRemoval = true)
public Table getTableDefinitionTable(
@NotNull final String tableIdentifier,
final long snapshotId,
@Nullable final IcebergInstructions instructions) {
- final TableIdentifier tableId = TableIdentifier.parse(tableIdentifier);
-
- // Find the snapshot with the given snapshot id
- final Snapshot tableSnapshot =
- getSnapshot(tableId, snapshotId).orElseThrow(() -> new IllegalArgumentException(
- "Snapshot with id " + snapshotId + " not found for table " + tableId));
-
- return getTableDefinitionTable(tableId, tableSnapshot, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.definitionTable(snapshotId, instructions);
}
/**
@@ -629,35 +394,13 @@ public Table getTableDefinitionTable(
* @param instructions The instructions for customizations while reading
* @return The table definition as a Deephaven table
*/
+ @Deprecated(forRemoval = true)
public Table getTableDefinitionTable(
@NotNull final TableIdentifier tableIdentifier,
@Nullable final Snapshot tableSnapshot,
@Nullable final IcebergInstructions instructions) {
- final TableDefinition definition = getTableDefinition(tableIdentifier, tableSnapshot, instructions);
- return TableTools.metaTable(definition);
- }
-
- /**
- * Internal method to create a {@link TableDefinition} from the table schema, snapshot and user instructions.
- */
- private TableDefinition getTableDefinitionInternal(
- @NotNull final TableIdentifier tableIdentifier,
- @Nullable final Snapshot tableSnapshot,
- @Nullable final IcebergInstructions instructions) {
- final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier);
- if (table == null) {
- throw new IllegalArgumentException("Table not found: " + tableIdentifier);
- }
-
- final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot();
- final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema();
-
- final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions;
-
- return fromSchema(schema,
- table.spec(),
- userInstructions.tableDefinition().orElse(null),
- getRenameColumnMap(table, schema, userInstructions));
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.definitionTable(tableSnapshot, instructions);
}
/**
@@ -668,10 +411,12 @@ private TableDefinition getTableDefinitionInternal(
* @return The loaded table
*/
@SuppressWarnings("unused")
+ @Deprecated(forRemoval = true)
public IcebergTable readTable(
@NotNull final TableIdentifier tableIdentifier,
@Nullable final IcebergInstructions instructions) {
- return readTableInternal(tableIdentifier, null, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.table(instructions);
}
/**
@@ -682,10 +427,12 @@ public IcebergTable readTable(
* @return The loaded table
*/
@SuppressWarnings("unused")
+ @Deprecated(forRemoval = true)
public IcebergTable readTable(
@NotNull final String tableIdentifier,
@Nullable final IcebergInstructions instructions) {
- return readTable(TableIdentifier.parse(tableIdentifier), instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.table(instructions);
}
/**
@@ -697,16 +444,13 @@ public IcebergTable readTable(
* @return The loaded table
*/
@SuppressWarnings("unused")
+ @Deprecated(forRemoval = true)
public IcebergTable readTable(
@NotNull final TableIdentifier tableIdentifier,
final long tableSnapshotId,
@Nullable final IcebergInstructions instructions) {
- // Find the snapshot with the given snapshot id
- final Snapshot tableSnapshot =
- getSnapshot(tableIdentifier, tableSnapshotId).orElseThrow(() -> new IllegalArgumentException(
- "Snapshot with id " + tableSnapshotId + " not found for table " + tableIdentifier));
-
- return readTableInternal(tableIdentifier, tableSnapshot, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.table(tableSnapshotId, instructions);
}
/**
@@ -718,11 +462,13 @@ public IcebergTable readTable(
* @return The loaded table
*/
@SuppressWarnings("unused")
+ @Deprecated(forRemoval = true)
public IcebergTable readTable(
@NotNull final String tableIdentifier,
final long tableSnapshotId,
@Nullable final IcebergInstructions instructions) {
- return readTable(TableIdentifier.parse(tableIdentifier), tableSnapshotId, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.table(tableSnapshotId, instructions);
}
/**
@@ -734,98 +480,24 @@ public IcebergTable readTable(
* @return The loaded table
*/
@SuppressWarnings("unused")
+ @Deprecated(forRemoval = true)
public IcebergTable readTable(
@NotNull final TableIdentifier tableIdentifier,
@NotNull final Snapshot tableSnapshot,
@Nullable final IcebergInstructions instructions) {
- return readTableInternal(tableIdentifier, tableSnapshot, instructions);
+ final IcebergTableAdapter tableAdapter = loadTable(tableIdentifier);
+ return tableAdapter.table(tableSnapshot, instructions);
}
- private IcebergTable readTableInternal(
- @NotNull final TableIdentifier tableIdentifier,
- @Nullable final Snapshot tableSnapshot,
- @Nullable final IcebergInstructions instructions) {
- // Load the table from the catalog.
- final org.apache.iceberg.Table table = catalog.loadTable(tableIdentifier);
- if (table == null) {
- throw new IllegalArgumentException("Table not found: " + tableIdentifier);
- }
-
- // Do we want the latest or a specific snapshot?
- final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot();
- final Schema schema = table.schemas().get(snapshot.schemaId());
-
- // Load the partitioning schema.
- final org.apache.iceberg.PartitionSpec partitionSpec = table.spec();
-
- // Get default instructions if none are provided
- final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions;
-
- // Get the user supplied table definition.
- final TableDefinition userTableDef = userInstructions.tableDefinition().orElse(null);
-
- // Map all the column names in the schema to their legalized names.
- final Map legalizedColumnRenames = getRenameColumnMap(table, schema, userInstructions);
-
- // Get the table definition from the schema (potentially limited by the user supplied table definition and
- // applying column renames).
- final TableDefinition tableDef = fromSchema(schema, partitionSpec, userTableDef, legalizedColumnRenames);
-
- final IcebergBaseLayout keyFinder;
-
- if (partitionSpec.isUnpartitioned()) {
- // Create the flat layout location key finder
- keyFinder = new IcebergFlatLayout(tableDef, table, snapshot, fileIO, userInstructions);
- } else {
- // Create the partitioning column location key finder
- keyFinder = new IcebergKeyValuePartitionedLayout(tableDef, table, snapshot, fileIO, partitionSpec,
- userInstructions);
- }
-
- if (instructions == null
- || instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) {
- final IcebergTableLocationProviderBase locationProvider =
- new IcebergStaticTableLocationProvider<>(
- StandaloneTableKey.getInstance(),
- keyFinder,
- new IcebergTableLocationFactory(),
- tableIdentifier);
-
- return new IcebergTableImpl(
- tableDef,
- tableIdentifier.toString(),
- RegionedTableComponentFactoryImpl.INSTANCE,
- locationProvider,
- null);
- }
-
- final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
- final IcebergTableLocationProviderBase locationProvider;
-
- if (instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) {
- locationProvider = new IcebergManualRefreshTableLocationProvider<>(
- StandaloneTableKey.getInstance(),
- keyFinder,
- new IcebergTableLocationFactory(),
- this,
- tableIdentifier);
- } else {
- locationProvider = new IcebergAutoRefreshTableLocationProvider<>(
- StandaloneTableKey.getInstance(),
- keyFinder,
- new IcebergTableLocationFactory(),
- TableDataRefreshService.getSharedRefreshService(),
- instructions.updateMode().autoRefreshMs(),
- this,
- tableIdentifier);
- }
-
- return new IcebergTableImpl(
- tableDef,
- tableIdentifier.toString(),
- RegionedTableComponentFactoryImpl.INSTANCE,
- locationProvider,
- updateSourceRegistrar);
+ /**
+ * Load an Iceberg table from the catalog.
+ *
+ * @param tableIdentifier The identifier of the table to load.
+ * @return The {@link IcebergTableAdapter table adapter} for the Iceberg table.
+ */
+ @Deprecated(forRemoval = true)
+ public IcebergTableAdapter loadTable(final String tableIdentifier) {
+ return loadTable(TableIdentifier.parse(tableIdentifier));
}
/**
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
index a0ae36a6eaa..c9d4b031354 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTable.java
@@ -11,7 +11,6 @@ public interface IcebergTable extends Table {
/**
* Update the table with the latest snapshot from the catalog.
*/
- @SuppressWarnings("unused")
void update();
/**
@@ -21,7 +20,6 @@ public interface IcebergTable extends Table {
*
* @param snapshotId The identifier of the snapshot to use when updating the table.
*/
- @SuppressWarnings("unused")
void update(final long snapshotId);
/**
@@ -30,6 +28,5 @@ public interface IcebergTable extends Table {
*
* @param snapshot The snapshot to use when updating the table.
*/
- @SuppressWarnings("unused")
void update(final @NotNull Snapshot snapshot);
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java
new file mode 100644
index 00000000000..4f1971d0dea
--- /dev/null
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableAdapter.java
@@ -0,0 +1,587 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.iceberg.util;
+
+import io.deephaven.api.util.NameValidator;
+import io.deephaven.engine.context.ExecutionContext;
+import io.deephaven.engine.rowset.RowSetFactory;
+import io.deephaven.engine.table.ColumnDefinition;
+import io.deephaven.engine.table.ColumnSource;
+import io.deephaven.engine.table.Table;
+import io.deephaven.engine.table.TableDefinition;
+import io.deephaven.engine.table.impl.QueryTable;
+import io.deephaven.engine.table.impl.locations.TableDataException;
+import io.deephaven.engine.table.impl.locations.TableKey;
+import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey;
+import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService;
+import io.deephaven.engine.table.impl.sources.InMemoryColumnSource;
+import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl;
+import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
+import io.deephaven.engine.util.TableTools;
+import io.deephaven.iceberg.layout.*;
+import io.deephaven.iceberg.location.IcebergTableLocationFactory;
+import io.deephaven.iceberg.location.IcebergTableLocationKey;
+import io.deephaven.time.DateTimeUtils;
+import io.deephaven.util.annotations.VisibleForTesting;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.util.*;
+import java.util.stream.Collectors;
+
+/**
+ * This class manages an Iceberg {@link org.apache.iceberg.Table table} and provides methods to interact with it.
+ */
+public class IcebergTableAdapter {
+ @VisibleForTesting
+ static final TableDefinition SNAPSHOT_DEFINITION = TableDefinition.of(
+ ColumnDefinition.ofLong("Id"),
+ ColumnDefinition.ofTime("Timestamp"),
+ ColumnDefinition.ofString("Operation"),
+ ColumnDefinition.fromGenericType("Summary", Map.class),
+ ColumnDefinition.fromGenericType("SnapshotObject", Snapshot.class));
+
+ private final org.apache.iceberg.Table table;
+ private final TableIdentifier tableIdentifier;
+
+ public IcebergTableAdapter(
+ final TableIdentifier tableIdentifier,
+ final org.apache.iceberg.Table table) {
+ this.table = table;
+ this.tableIdentifier = tableIdentifier;
+ }
+
+ /**
+ * Get the current {@link Snapshot snapshot} of a given Iceberg table.
+ *
+ * @return The current snapshot of the table.
+ */
+ public Snapshot currentSnapshot() {
+ final List snapshots = listSnapshots();
+ if (snapshots.isEmpty()) {
+ throw new IllegalStateException("No snapshots found for table " + tableIdentifier);
+ }
+ return snapshots.get(snapshots.size() - 1);
+ }
+
+ /**
+ * List all {@link Snapshot snapshots} of the Iceberg table.
+ *
+ * @return A list of all snapshots of the given table.
+ */
+ public List listSnapshots() {
+ final List snapshots = new ArrayList<>();
+ table.snapshots().forEach(snapshots::add);
+ return snapshots;
+ }
+
+ /**
+ * List all {@link Snapshot snapshots} of a given Iceberg table as a Deephaven {@link Table table}. The resulting
+ * table will be static and contain the following columns:
+ *
+ *
+ *
Column Name
+ *
Description
+ *
+ *
+ *
Id
+ *
The snapshot identifier (can be used for updating the table or loading a specific snapshot)
+ *
+ *
+ *
TimestampMs
+ *
The timestamp of the snapshot
+ *
+ *
+ *
Operation
+ *
The data operation that created this snapshot
+ *
+ *
+ *
Summary
+ *
Additional information about the snapshot from the Iceberg metadata
+ *
+ *
+ *
SnapshotObject
+ *
A Java object containing the Iceberg API snapshot
+ *
+ *
+ *
+ *
+ *
+ * @return A Table containing a list of all tables in the given namespace.
+ */
+ public Table snapshots() {
+ final List snapshots = listSnapshots();
+ final long size = snapshots.size();
+
+ // Create and return a table containing the namespaces as strings
+ final Map> columnSourceMap = new LinkedHashMap<>();
+
+ // Create the column source(s)
+ final long[] idArr = new long[(int) size];
+ columnSourceMap.put("Id", InMemoryColumnSource.getImmutableMemoryColumnSource(idArr, long.class, null));
+
+ final long[] timestampArr = new long[(int) size];
+ columnSourceMap.put("Timestamp",
+ InMemoryColumnSource.getImmutableMemoryColumnSource(timestampArr, Instant.class, null));
+
+ final String[] operatorArr = new String[(int) size];
+ columnSourceMap.put("Operation",
+ InMemoryColumnSource.getImmutableMemoryColumnSource(operatorArr, String.class, null));
+
+ final Map[] summaryArr = new Map[(int) size];
+ columnSourceMap.put("Summary",
+ InMemoryColumnSource.getImmutableMemoryColumnSource(summaryArr, Map.class, null));
+
+ final Snapshot[] snapshotArr = new Snapshot[(int) size];
+ columnSourceMap.put("SnapshotObject",
+ InMemoryColumnSource.getImmutableMemoryColumnSource(snapshotArr, Snapshot.class, null));
+
+ // Populate the column source(s)
+ for (int i = 0; i < size; i++) {
+ final Snapshot snapshot = snapshots.get(i);
+ idArr[i] = snapshot.snapshotId();
+ // Provided as millis from epoch, convert to nanos
+ timestampArr[i] = DateTimeUtils.millisToNanos(snapshot.timestampMillis());
+ operatorArr[i] = snapshot.operation();
+ summaryArr[i] = snapshot.summary();
+ snapshotArr[i] = snapshot;
+ }
+
+ // Create and return the table
+ return new QueryTable(SNAPSHOT_DEFINITION, RowSetFactory.flat(size).toTracking(), columnSourceMap);
+ }
+
+ /**
+ * Retrieve a specific {@link Snapshot snapshot} of an Iceberg table.
+ *
+ * @param snapshotId The identifier of the snapshot to load.
+ *
+ * @return An Optional containing the requested snapshot if it exists.
+ */
+ private Optional snapshot(final long snapshotId) {
+ return listSnapshots().stream()
+ .filter(snapshot -> snapshot.snapshotId() == snapshotId)
+ .findFirst();
+ }
+
+ /**
+ * Return {@link TableDefinition table definition}.
+ *
+ * @return The table definition
+ */
+ public TableDefinition definition() {
+ // Load the table from the catalog.
+ return definition(null, null);
+ }
+
+ /**
+ * Return {@link TableDefinition table definition} with optional instructions for customizations while reading.
+ *
+ * @param instructions The instructions for customizations while reading (or null for default instructions)
+ * @return The table definition
+ */
+ public TableDefinition definition(@Nullable final IcebergInstructions instructions) {
+ // Load the table from the catalog.
+ return definition(null, instructions);
+ }
+
+ /**
+ * Return {@link TableDefinition table definition} for the Iceberg table and snapshot id, with optional instructions
+ * for customizations while reading.
+ *
+ * @param snapshotId The identifier of the snapshot to load
+ * @param instructions The instructions for customizations while reading (or null for default instructions)
+ * @return The table definition
+ */
+ public TableDefinition definition(
+ final long snapshotId,
+ @Nullable final IcebergInstructions instructions) {
+
+ // Find the snapshot with the given snapshot id
+ final Snapshot tableSnapshot =
+ snapshot(snapshotId).orElseThrow(() -> new IllegalArgumentException(
+ "Snapshot with id " + snapshotId + " not found for table " + tableIdentifier));
+
+ // Load the table from the catalog.
+ return definition(tableSnapshot, instructions);
+ }
+
+ /**
+ * Return {@link TableDefinition table definition} for the Iceberg table and snapshot, with optional instructions
+ * for customizations while reading.
+ *
+ * @param tableSnapshot The snapshot to load
+ * @param instructions The instructions for customizations while reading (or null for default instructions)
+ * @return The table definition
+ */
+ public TableDefinition definition(
+ @Nullable final Snapshot tableSnapshot,
+ @Nullable final IcebergInstructions instructions) {
+ final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot();
+ final Schema schema = snapshot != null ? table.schemas().get(snapshot.schemaId()) : table.schema();
+
+ final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions;
+
+ return fromSchema(schema,
+ table.spec(),
+ userInstructions.tableDefinition().orElse(null),
+ getRenameColumnMap(table, schema, userInstructions));
+ }
+
+ /**
+ * Return {@link Table table} containing the {@link TableDefinition definition} of the Iceberg table.
+ *
+ * @return The table definition as a Deephaven table
+ */
+ public Table definitionTable() {
+ return definitionTable(null, null);
+ }
+
+ /**
+ * Return {@link Table table} containing the {@link TableDefinition definition} of the Iceberg table, with optional
+ * instructions for customizations while reading.
+ *
+ * @param instructions The instructions for customizations while reading
+ * @return The table definition as a Deephaven table
+ */
+ public Table definitionTable(@Nullable final IcebergInstructions instructions) {
+ return definitionTable(null, instructions);
+ }
+
+ /**
+ * Return {@link Table table} containing the {@link TableDefinition definition} of a given Iceberg table and
+ * snapshot id, with optional instructions for customizations while reading.
+ *
+ * @param snapshotId The identifier of the snapshot to load
+ * @param instructions The instructions for customizations while reading (or null for default instructions)
+ * @return The table definition as a Deephaven table
+ */
+ public Table definitionTable(
+ final long snapshotId,
+ @Nullable final IcebergInstructions instructions) {
+
+ // Find the snapshot with the given snapshot id
+ final Snapshot tableSnapshot =
+ snapshot(snapshotId).orElseThrow(() -> new IllegalArgumentException(
+ "Snapshot with id " + snapshotId + " not found for table " + tableIdentifier));
+
+ return definitionTable(tableSnapshot, instructions);
+ }
+
+ /**
+ * Return {@link Table table} containing the {@link TableDefinition definition} of a given Iceberg table and
+ * snapshot id, with optional instructions for customizations while reading.
+ *
+ * @param tableSnapshot The snapshot to load
+ * @param instructions The instructions for customizations while reading (or null for default instructions)
+ * @return The table definition as a Deephaven table
+ */
+ public Table definitionTable(
+ @Nullable final Snapshot tableSnapshot,
+ @Nullable final IcebergInstructions instructions) {
+ final TableDefinition definition = definition(tableSnapshot, instructions);
+ return TableTools.metaTable(definition);
+ }
+
+ /**
+ * Read the latest snapshot of an Iceberg table from the Iceberg catalog as a Deephaven {@link Table table}.
+ *
+ * @param instructions The instructions for customizations while reading (or null for default instructions)
+ * @return The loaded table
+ */
+ @SuppressWarnings("unused")
+ public IcebergTable table(@Nullable final IcebergInstructions instructions) {
+ return table(null, instructions);
+ }
+
+ /**
+ * Read a snapshot of an Iceberg table from the Iceberg catalog.
+ *
+ * @param tableSnapshotId The snapshot id to load
+ * @param instructions The instructions for customizations while reading (or null for default instructions)
+ * @return The loaded table
+ */
+ @SuppressWarnings("unused")
+ public IcebergTable table(final long tableSnapshotId, @Nullable final IcebergInstructions instructions) {
+ // Find the snapshot with the given snapshot id
+ final Snapshot tableSnapshot =
+ snapshot(tableSnapshotId).orElseThrow(() -> new IllegalArgumentException(
+ "Snapshot with id " + tableSnapshotId + " not found for table " + tableIdentifier));
+
+ return table(tableSnapshot, instructions);
+ }
+
+ /**
+ * Read a snapshot of an Iceberg table from the Iceberg catalog.
+ *
+ * @param tableSnapshot The snapshot id to load
+ * @param instructions The instructions for customizations while reading
+ * @return The loaded table
+ */
+ public IcebergTable table(
+ @Nullable final Snapshot tableSnapshot,
+ @Nullable final IcebergInstructions instructions) {
+
+ // Do we want the latest or a specific snapshot?
+ final Snapshot snapshot = tableSnapshot != null ? tableSnapshot : table.currentSnapshot();
+ final Schema schema = table.schemas().get(snapshot.schemaId());
+
+ // Load the partitioning schema.
+ final org.apache.iceberg.PartitionSpec partitionSpec = table.spec();
+
+ // Get default instructions if none are provided
+ final IcebergInstructions userInstructions = instructions == null ? IcebergInstructions.DEFAULT : instructions;
+
+ // Get the user supplied table definition.
+ final TableDefinition userTableDef = userInstructions.tableDefinition().orElse(null);
+
+ // Map all the column names in the schema to their legalized names.
+ final Map legalizedColumnRenames = getRenameColumnMap(table, schema, userInstructions);
+
+ // Get the table definition from the schema (potentially limited by the user supplied table definition and
+ // applying column renames).
+ final TableDefinition tableDef = fromSchema(schema, partitionSpec, userTableDef, legalizedColumnRenames);
+
+ final IcebergBaseLayout keyFinder;
+
+ if (partitionSpec.isUnpartitioned()) {
+ // Create the flat layout location key finder
+ keyFinder = new IcebergFlatLayout(this, snapshot, userInstructions);
+ } else {
+ // Create the partitioning column location key finder
+ keyFinder = new IcebergKeyValuePartitionedLayout(this, snapshot, partitionSpec,
+ userInstructions);
+ }
+
+ if (instructions == null
+ || instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) {
+ final IcebergTableLocationProviderBase locationProvider =
+ new IcebergStaticTableLocationProvider<>(
+ StandaloneTableKey.getInstance(),
+ keyFinder,
+ new IcebergTableLocationFactory(),
+ tableIdentifier);
+
+ return new IcebergTableImpl(
+ tableDef,
+ tableIdentifier.toString(),
+ RegionedTableComponentFactoryImpl.INSTANCE,
+ locationProvider,
+ null);
+ }
+
+ final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
+ final IcebergTableLocationProviderBase locationProvider;
+
+ if (instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) {
+ locationProvider = new IcebergManualRefreshTableLocationProvider<>(
+ StandaloneTableKey.getInstance(),
+ keyFinder,
+ new IcebergTableLocationFactory(),
+ this,
+ tableIdentifier);
+ } else {
+ locationProvider = new IcebergAutoRefreshTableLocationProvider<>(
+ StandaloneTableKey.getInstance(),
+ keyFinder,
+ new IcebergTableLocationFactory(),
+ TableDataRefreshService.getSharedRefreshService(),
+ instructions.updateMode().autoRefreshMs(),
+ this,
+ tableIdentifier);
+ }
+
+ return new IcebergTableImpl(
+ tableDef,
+ tableIdentifier.toString(),
+ RegionedTableComponentFactoryImpl.INSTANCE,
+ locationProvider,
+ updateSourceRegistrar);
+ }
+
+ /**
+ * Refresh the table with the latest information from the Iceberg catalog, including new snapshots and schema.
+ */
+ public void refresh() {
+ table.refresh();
+ }
+
+ /**
+ * Return the underlying Iceberg table.
+ */
+ public org.apache.iceberg.Table icebergTable() {
+ return table;
+ }
+
+ @Override
+ public String toString() {
+ return table.toString();
+ }
+
+ /**
+ * Get a legalized column rename map from a table schema and user instructions.
+ */
+ private Map getRenameColumnMap(
+ @NotNull final org.apache.iceberg.Table table,
+ @NotNull final Schema schema,
+ @NotNull final IcebergInstructions instructions) {
+
+ final Set takenNames = new HashSet<>();
+
+ // Map all the column names in the schema to their legalized names.
+ final Map legalizedColumnRenames = new HashMap<>();
+
+ // Validate user-supplied names meet legalization instructions
+ for (final Map.Entry entry : instructions.columnRenames().entrySet()) {
+ final String destinationName = entry.getValue();
+ if (!NameValidator.isValidColumnName(destinationName)) {
+ throw new TableDataException(
+ String.format("%s - invalid column name provided (%s)", table, destinationName));
+ }
+ // Add these renames to the legalized list.
+ legalizedColumnRenames.put(entry.getKey(), destinationName);
+ takenNames.add(destinationName);
+ }
+
+ for (final Types.NestedField field : schema.columns()) {
+ final String name = field.name();
+ // Do we already have a valid rename for this column from the user or a partitioned column?
+ if (!legalizedColumnRenames.containsKey(name)) {
+ final String legalizedName =
+ NameValidator.legalizeColumnName(name, s -> s.replace(" ", "_"), takenNames);
+ if (!legalizedName.equals(name)) {
+ legalizedColumnRenames.put(name, legalizedName);
+ takenNames.add(legalizedName);
+ }
+ }
+ }
+
+ return legalizedColumnRenames;
+ }
+
+ /**
+ * Create a single {@link TableDefinition} from a given Schema, PartitionSpec, and TableDefinition. Takes into
+ * account {@link Map<> column rename instructions}
+ *
+ * @param schema The schema of the table.
+ * @param partitionSpec The partition specification of the table.
+ * @param userTableDef The table definition.
+ * @param columnRename The map for renaming columns.
+ * @return The generated TableDefinition.
+ */
+ private static TableDefinition fromSchema(
+ @NotNull final Schema schema,
+ @NotNull final PartitionSpec partitionSpec,
+ @Nullable final TableDefinition userTableDef,
+ @NotNull final Map columnRename) {
+
+ final Set columnNames = userTableDef != null
+ ? userTableDef.getColumnNameSet()
+ : null;
+
+ final Set partitionNames =
+ partitionSpec.fields().stream()
+ .map(PartitionField::name)
+ .map(colName -> columnRename.getOrDefault(colName, colName))
+ .collect(Collectors.toSet());
+
+ final List> columns = new ArrayList<>();
+
+ for (final Types.NestedField field : schema.columns()) {
+ final String name = columnRename.getOrDefault(field.name(), field.name());
+ // Skip columns that are not in the provided table definition.
+ if (columnNames != null && !columnNames.contains(name)) {
+ continue;
+ }
+ final Type type = field.type();
+ final io.deephaven.qst.type.Type> qstType = convertPrimitiveType(type);
+ final ColumnDefinition> column;
+ if (partitionNames.contains(name)) {
+ column = ColumnDefinition.of(name, qstType).withPartitioning();
+ } else {
+ column = ColumnDefinition.of(name, qstType);
+ }
+ columns.add(column);
+ }
+
+ final TableDefinition icebergTableDef = TableDefinition.of(columns);
+ if (userTableDef == null) {
+ return icebergTableDef;
+ }
+
+ // If the user supplied a table definition, make sure it's fully compatible.
+ final TableDefinition tableDef = icebergTableDef.checkCompatibility(userTableDef);
+
+ // Ensure that the user has not marked non-partitioned columns as partitioned.
+ final Set userPartitionColumns = userTableDef.getPartitioningColumns().stream()
+ .map(ColumnDefinition::getName)
+ .collect(Collectors.toSet());
+ final Set partitionColumns = tableDef.getPartitioningColumns().stream()
+ .map(ColumnDefinition::getName)
+ .collect(Collectors.toSet());
+
+ // The working partitioning column set must be a super-set of the user-supplied set.
+ if (!partitionColumns.containsAll(userPartitionColumns)) {
+ final Set invalidColumns = new HashSet<>(userPartitionColumns);
+ invalidColumns.removeAll(partitionColumns);
+
+ throw new TableDataException("The following columns are not partitioned in the Iceberg table: " +
+ invalidColumns);
+ }
+ return tableDef;
+ }
+
+ /**
+ * Convert an Iceberg data type to a Deephaven type.
+ *
+ * @param icebergType The Iceberg data type to be converted.
+ * @return The converted Deephaven type.
+ */
+ static io.deephaven.qst.type.Type> convertPrimitiveType(@NotNull final Type icebergType) {
+ final Type.TypeID typeId = icebergType.typeId();
+ switch (typeId) {
+ case BOOLEAN:
+ return io.deephaven.qst.type.Type.booleanType().boxedType();
+ case DOUBLE:
+ return io.deephaven.qst.type.Type.doubleType();
+ case FLOAT:
+ return io.deephaven.qst.type.Type.floatType();
+ case INTEGER:
+ return io.deephaven.qst.type.Type.intType();
+ case LONG:
+ return io.deephaven.qst.type.Type.longType();
+ case STRING:
+ return io.deephaven.qst.type.Type.stringType();
+ case TIMESTAMP:
+ final Types.TimestampType timestampType = (Types.TimestampType) icebergType;
+ return timestampType.shouldAdjustToUTC()
+ ? io.deephaven.qst.type.Type.find(Instant.class)
+ : io.deephaven.qst.type.Type.find(LocalDateTime.class);
+ case DATE:
+ return io.deephaven.qst.type.Type.find(java.time.LocalDate.class);
+ case TIME:
+ return io.deephaven.qst.type.Type.find(java.time.LocalTime.class);
+ case DECIMAL:
+ return io.deephaven.qst.type.Type.find(java.math.BigDecimal.class);
+ case FIXED: // Fall through
+ case BINARY:
+ return io.deephaven.qst.type.Type.find(byte[].class);
+ case UUID: // Fall through
+ case STRUCT: // Fall through
+ case LIST: // Fall through
+ case MAP: // Fall through
+ default:
+ throw new TableDataException("Unsupported iceberg column type " + typeId.name());
+ }
+ }
+}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java
index 02e3db6e115..a73a55d40be 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTableImpl.java
@@ -57,15 +57,4 @@ public void update(final long snapshotId) {
public void update(final @NotNull Snapshot snapshot) {
locationProvider.update(snapshot);
}
-
- @Override
- public int currentSnapshotId() {
- // TODO: implement the TableAdapter
- return 0;
- }
-
- @Override
- public Snapshot currentSnapshot() {
- return null;
- }
}
diff --git a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java
index bcdda326dca..33fb26a3647 100644
--- a/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java
+++ b/extensions/iceberg/src/main/java/io/deephaven/iceberg/util/IcebergTools.java
@@ -3,17 +3,16 @@
//
package io.deephaven.iceberg.util;
+import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.catalog.Catalog;
-import org.apache.iceberg.io.FileIO;
/**
* Tools for accessing tables in the Iceberg table format.
*/
public abstract class IcebergTools {
@SuppressWarnings("unused")
- public static IcebergCatalogAdapter createAdapter(
- final Catalog catalog,
- final FileIO fileIO) {
- return new IcebergCatalogAdapter(catalog, fileIO);
+ public static IcebergCatalogAdapter createAdapter(final Catalog catalog) {
+ Configuration conf = new Configuration();
+ return new IcebergCatalogAdapter(catalog);
}
}
diff --git a/py/server/deephaven/experimental/iceberg.py b/py/server/deephaven/experimental/iceberg.py
index 623cccc6c16..79e0eb15796 100644
--- a/py/server/deephaven/experimental/iceberg.py
+++ b/py/server/deephaven/experimental/iceberg.py
@@ -15,6 +15,7 @@
_JIcebergInstructions = jpy.get_type("io.deephaven.iceberg.util.IcebergInstructions")
_JIcebergUpdateMode = jpy.get_type("io.deephaven.iceberg.util.IcebergUpdateMode")
_JIcebergCatalogAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergCatalogAdapter")
+_JIcebergTableAdapter = jpy.get_type("io.deephaven.iceberg.util.IcebergTableAdapter")
_JIcebergTable = jpy.get_type("io.deephaven.iceberg.util.IcebergTable")
# IcebergToolsS3 is an optional library
@@ -166,6 +167,79 @@ def j_object(self) -> jpy.JType:
return self.j_table
+class IcebergTableAdapter(JObjectWrapper):
+ """
+ This class provides an interface for interacting with Iceberg tables. It allows listing snapshots, table
+ definitions, as well as reading Iceberg tables into Deephaven tables.
+ """
+ j_object_type = _JIcebergTableAdapter or type(None)
+
+ def __init__(self, j_object: _JIcebergTableAdapter):
+ self.j_table_adapter = j_object
+
+ def snapshots(self) -> Table:
+ """
+ Returns information on the snapshots of this table as a Deephaven table. The table contains the
+ following columns:
+ - `Id`: the snapshot identifier (can be used for updating the table or loading a specific snapshot).
+ - `TimestampMs`: the timestamp of the snapshot.
+ - `Operation`: the data operation that created this snapshot.
+ - `Summary`: additional information about this snapshot from the Iceberg metadata.
+ - `SnapshotObject`: a Java object containing the Iceberg API snapshot.
+
+ Args:
+ table_identifier (str): the table from which to list snapshots.
+
+ Returns:
+ a table containing the snapshot information.
+ """
+ return Table(self.j_object.snapshots())
+
+ def definition(self, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> Table:
+ """
+ Returns information on the tables in the specified namespace as a Deephaven table.
+
+ Args:
+ namespace (str): the namespace from which to list tables.
+
+ Returns:
+ a table containing the tables in the provided namespace.
+ """
+
+ if snapshot_id is not None:
+ return Table(self.j_object.definitionTable(snapshot_id, instructions))
+ return Table(self.j_object.definitionTable(instructions))
+
+ def table(self, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> IcebergTable:
+ """
+ Reads the table using the provided instructions. Optionally, a snapshot id can be provided to
+ read a specific snapshot of the table.
+
+ Args:
+ table_identifier (str): the table to read.
+ instructions (Optional[IcebergInstructions]): the instructions for reading the table. These instructions
+ can include column renames, table definition, and specific data instructions for reading the data files
+ from the provider. If omitted, the table will be read with default instructions.
+ snapshot_id (Optional[int]): the snapshot id to read; if omitted the most recent snapshot will be selected.
+
+ Returns:
+ Table: the table read from the catalog.
+ """
+
+ if instructions is not None:
+ instructions_object = instructions.j_object
+ else:
+ instructions_object = _JIcebergInstructions.DEFAULT
+
+ if snapshot_id is not None:
+ return IcebergTable(self.j_object.table(snapshot_id, instructions_object))
+ return IcebergTable(self.j_object.readTable(instructions_object))
+
+ @property
+ def j_object(self) -> jpy.JType:
+ return self.j_table_adapter
+
+
class IcebergCatalogAdapter(JObjectWrapper):
"""
This class provides an interface for interacting with Iceberg catalogs. It allows listing namespaces, tables and
@@ -190,10 +264,10 @@ def namespaces(self, namespace: Optional[str] = None) -> Table:
"""
if namespace is not None:
- return Table(self.j_object.listNamespaces(namespace))
- return Table(self.j_object.listNamespacesAsTable())
+ return Table(self.j_object.namespaces(namespace))
+ return Table(self.j_object.namespaces())
- def tables(self, namespace: str) -> Table:
+ def tables(self, namespace: str) -> IcebergTableAdapter:
"""
Returns information on the tables in the specified namespace as a Deephaven table.
@@ -204,12 +278,25 @@ def tables(self, namespace: str) -> Table:
a table containing the tables in the provided namespace.
"""
- if namespace is not None:
- return Table(self.j_object.listTablesAsTable(namespace))
- return Table(self.j_object.listTablesAsTable())
+ return Table(self.j_object.tables(namespace))
+
+ def load_table(self, table_identifier: str) -> IcebergTable:
+ """
+ Load the table from the catalog using the provided instructions.
+
+ Args:
+ table_identifier (str): the table to read.
+
+ Returns:
+ Table: the table read from the catalog.
+ """
+
+ return IcebergTableAdapter(self.j_object.loadTable(table_identifier))
def snapshots(self, table_identifier: str) -> Table:
"""
+ NOTE: deprecated, use `load_table(table_identifier).snapshots()` instead.
+
Returns information on the snapshots of the specified table as a Deephaven table. The table contains the
following columns:
- `Id`: the snapshot identifier (can be used for updating the table or loading a specific snapshot).
@@ -225,10 +312,12 @@ def snapshots(self, table_identifier: str) -> Table:
a table containing the snapshot information.
"""
- return self.j_object.listSnapshotsAsTable(table_identifier)
+ return self.j_object.snapshots(table_identifier)
def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> IcebergTable:
"""
+ NOTE: deprecated, use `load_table(table_identifier).table()` instead.
+
Reads the table from the catalog using the provided instructions. Optionally, a snapshot id can be provided to
read a specific snapshot of the table.
From 68e45460b4690d9dd0544f49cd019856829fa043 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Fri, 30 Aug 2024 16:12:47 -0700
Subject: [PATCH 22/56] Incorporated external PR to update
PartitioningColumnDataIndex for refreshing tables.
---
.../regioned/PartitioningColumnDataIndex.java | 151 ++++++++++++------
.../regioned/RegionedColumnSourceManager.java | 23 +--
2 files changed, 114 insertions(+), 60 deletions(-)
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java
index ae95d86b8ee..1fc5e540b88 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java
@@ -6,12 +6,7 @@
import gnu.trove.map.hash.TObjectIntHashMap;
import io.deephaven.base.verify.Assert;
import io.deephaven.base.verify.Require;
-import io.deephaven.engine.rowset.RowSequence;
-import io.deephaven.engine.rowset.RowSet;
-import io.deephaven.engine.rowset.RowSetBuilderRandom;
-import io.deephaven.engine.rowset.RowSetFactory;
-import io.deephaven.engine.rowset.RowSetShiftData;
-import io.deephaven.engine.rowset.WritableRowSet;
+import io.deephaven.engine.rowset.*;
import io.deephaven.engine.table.ColumnSource;
import io.deephaven.engine.table.ModifiedColumnSet;
import io.deephaven.engine.table.Table;
@@ -56,7 +51,7 @@ class PartitioningColumnDataIndex extends AbstractDataIndex {
/** Provides fast lookup from keys to positions in the index table **/
private final TObjectIntHashMap keyPositionMap;
- private final ModifiedColumnSet upstreamLocationModified;
+ private final ModifiedColumnSet upstreamKeyModified;
private final ModifiedColumnSet upstreamRowSetModified;
private final ModifiedColumnSet downstreamRowSetModified;
@@ -65,7 +60,7 @@ class PartitioningColumnDataIndex extends AbstractDataIndex {
* {@link RegionedColumnSourceManager} at a time when there cannot be any concurrent "refresh" behavior, and so we
* can safely use the {@link RegionedColumnSourceManager#locationTable() location table} without snapshotting or
* considering previous values.
- *
+ *
* @param keyColumnName The key column name
* @param keySource The key source in the indexed table
* @param columnSourceManager The column source manager that provides locations and region indexes
@@ -113,7 +108,7 @@ class PartitioningColumnDataIndex extends AbstractDataIndex {
if (locationTable.isRefreshing()) {
// No need to track previous values; we mutate the index table's RowSets in-place, and we never move a key.
indexTable.getRowSet().writableCast().initializePreviousValue();
- upstreamLocationModified = locationTable.newModifiedColumnSet(columnSourceManager.locationColumnName());
+ upstreamKeyModified = locationTable.newModifiedColumnSet(keyColumnName);
upstreamRowSetModified = locationTable.newModifiedColumnSet(columnSourceManager.rowSetColumnName());
downstreamRowSetModified = indexTable.newModifiedColumnSet(rowSetColumnName());
final TableUpdateListener tableListener = new BaseTable.ListenerImpl(String.format(
@@ -126,7 +121,7 @@ public void onUpdate(@NotNull final TableUpdate upstream) {
locationTable.addUpdateListener(tableListener);
manage(indexTable);
} else {
- upstreamLocationModified = null;
+ upstreamKeyModified = null;
upstreamRowSetModified = null;
downstreamRowSetModified = null;
}
@@ -138,29 +133,30 @@ private synchronized void processUpdate(
if (upstream.empty()) {
return;
}
- if (upstream.removed().isNonempty()) {
- throw new UnsupportedOperationException("Removed locations are not currently supported");
- }
if (upstream.shifted().nonempty()) {
throw new UnsupportedOperationException("Shifted locations are not currently supported");
}
- if (upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(upstreamLocationModified)) {
- throw new UnsupportedOperationException("Modified locations are not currently supported");
+ if (upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(upstreamKeyModified)) {
+ throw new UnsupportedOperationException("Modified location keys are not currently supported");
}
Assert.assertion(initializing || isRefreshing(), "initializing || isRefreshing()");
final int previousSize = keyPositionMap.size();
- final RowSetBuilderRandom modifiedBuilder = initializing ? null : RowSetFactory.builderRandom();
+ final RowSetBuilderRandom modifiedPositionBuilder = initializing ? null : RowSetFactory.builderRandom();
- if (upstream.added().isNonempty()) {
- upstream.added().forAllRowKeys((final long locationRowKey) -> handleKey(
- locationRowKey, false, previousSize, modifiedBuilder));
+ if (upstream.removed().isNonempty()) {
+ Assert.eqFalse(initializing, "initializing");
+ upstream.removed().forAllRowKeys((final long locationRowKey) -> handleLocation(
+ locationRowKey, ChangeType.REMOVE, modifiedPositionBuilder));
}
-
if (upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(upstreamRowSetModified)) {
Assert.eqFalse(initializing, "initializing");
- upstream.modified().forAllRowKeys((final long locationRowKey) -> handleKey(
- locationRowKey, true, previousSize, modifiedBuilder));
+ upstream.modified().forAllRowKeys((final long locationRowKey) -> handleLocation(
+ locationRowKey, ChangeType.MODIFY, modifiedPositionBuilder));
+ }
+ if (upstream.added().isNonempty()) {
+ upstream.added().forAllRowKeys((final long locationRowKey) -> handleLocation(
+ locationRowKey, ChangeType.ADD, modifiedPositionBuilder));
}
final int newSize = keyPositionMap.size();
@@ -172,40 +168,89 @@ private synchronized void processUpdate(
return;
}
- // Send the downstream updates to any listeners of the index table
- final WritableRowSet modified = modifiedBuilder.build();
+ final WritableRowSet modified = modifiedPositionBuilder.build();
if (previousSize == newSize && modified.isEmpty()) {
modified.close();
return;
}
+ final RowSetBuilderSequential removedPositionsBuilder = RowSetFactory.builderSequential();
+ final RowSetBuilderSequential resurrectedPositionsBuilder = RowSetFactory.builderSequential();
+ modified.forAllRowKeys((final long pos) -> {
+ final RowSet indexRowSet = indexRowSetSource.get(pos);
+ // noinspection DataFlowIssue
+ if (indexRowSet.isEmpty()) {
+ removedPositionsBuilder.appendKey(pos);
+ } else if (indexRowSet.trackingCast().prev().isEmpty()) {
+ resurrectedPositionsBuilder.appendKey(pos);
+ }
+ });
+
+ final WritableRowSet added = RowSetFactory.fromRange(previousSize, newSize - 1);
+ final RowSet removed = removedPositionsBuilder.build();
+ modified.remove(removed);
+ try (final RowSet resurrected = resurrectedPositionsBuilder.build()) {
+ added.insert(resurrected);
+ modified.remove(resurrected);
+ }
+
+ // Send the downstream updates to any listeners of the index table
final TableUpdate downstream = new TableUpdateImpl(
- RowSetFactory.fromRange(previousSize, newSize - 1),
- RowSetFactory.empty(),
+ added,
+ removed,
modified,
RowSetShiftData.EMPTY,
modified.isNonempty() ? downstreamRowSetModified : ModifiedColumnSet.EMPTY);
indexTable.notifyListeners(downstream);
}
- private void handleKey(
+ private enum ChangeType {
+ // @formatter:off
+ ADD("Added"),
+ REMOVE("Removed"),
+ MODIFY("Modified");
+ // @formatter:on
+
+ private final String actionLabel;
+
+ ChangeType(@NotNull final String actionLabel) {
+ this.actionLabel = actionLabel;
+ }
+ }
+
+ private void handleLocation(
final long locationRowKey,
- final boolean isModify,
- final int previousSize,
- @Nullable final RowSetBuilderRandom modifiedBuilder) {
+ @NotNull final ChangeType changeType,
+ @Nullable final RowSetBuilderRandom modifiedPositionBuilder) {
final KEY_TYPE locationKey = locationTableKeySource.get(locationRowKey);
final Object locationKeyReinterpreted = locationTableKeySourceReinterpreted.get(locationRowKey);
- final RowSet regionRowSet = locationTableRowSetSource.get(locationRowKey);
- if (regionRowSet == null) {
- throw new IllegalStateException(String.format("Null row set found at location index %d", locationRowKey));
+
+ final RowSet currentRegionRowSet = changeType == ChangeType.REMOVE
+ ? null
+ : locationTableRowSetSource.get(locationRowKey);
+ final RowSet previousRegionRowSet = changeType == ChangeType.ADD
+ ? null
+ : locationTableRowSetSource.getPrev(locationRowKey);
+
+ if (changeType != ChangeType.REMOVE && (currentRegionRowSet == null || currentRegionRowSet.isEmpty())) {
+ throw new IllegalStateException(String.format(
+ "%s partition (index=%d, key=%s): Unexpected null or empty current row set",
+ changeType.actionLabel, locationRowKey, locationKey));
+ }
+ if (changeType != ChangeType.ADD && (previousRegionRowSet == null || previousRegionRowSet.isEmpty())) {
+ throw new IllegalStateException(String.format(
+ "%s partition (index=%d, key=%s): Unexpected null or empty previous row set",
+ changeType.actionLabel, locationRowKey, locationKey));
}
- final long regionFirstRowKey = RegionedColumnSource.getFirstRowKey(Math.toIntExact(locationRowKey));
// Test using the (maybe) reinterpreted key
final int pos = keyPositionMap.get(locationKeyReinterpreted);
+
+ // Inserting a new bucket
if (pos == KEY_NOT_FOUND) {
- if (isModify) {
- throw new IllegalStateException(String.format("Modified partition key %s not found", locationKey));
+ if (changeType == ChangeType.REMOVE || changeType == ChangeType.MODIFY) {
+ throw new IllegalStateException(String.format("%s partition (index=%d, key=%s): Key not found",
+ changeType.actionLabel, locationRowKey, locationKey));
}
final int addedKeyPos = keyPositionMap.size();
// Store the (maybe) reinterpreted key in the lookup hashmap.
@@ -216,21 +261,27 @@ private void handleKey(
indexKeySource.set(addedKeyPos, locationKey);
indexRowSetSource.ensureCapacity(addedKeyPos + 1);
- indexRowSetSource.set(addedKeyPos, regionRowSet.shift(regionFirstRowKey).toTracking());
- } else {
- // noinspection DataFlowIssue
- final WritableRowSet existingRowSet = indexRowSetSource.get(pos).writableCast();
- try (final WritableRowSet shiftedRowSet = regionRowSet.shift(regionFirstRowKey)) {
- // We could assert that:
- // 1. an added location is non-overlapping with the key's existing row set
- // 2. a modified location's current row set is a superset of its previous row set
- // 3. a modified location's previous row set is a subset of the key's existing row set
- existingRowSet.insert(shiftedRowSet);
- }
+ indexRowSetSource.set(addedKeyPos, currentRegionRowSet.copy().toTracking());
+ return;
+ }
- if (modifiedBuilder != null && pos < previousSize) {
- modifiedBuilder.addKey(pos);
- }
+ // Updating an existing bucket
+ // noinspection DataFlowIssue
+ final WritableRowSet existingRowSet = indexRowSetSource.get(pos).writableCast();
+ // We _could_ assert that:
+ // 1. An added location is non-overlapping with the key's existing row set
+ // 2. A modified location's current row set is a superset of its previous row set (with existing RCSM)
+ // 3. A removed or modified location's previous row set is a subset of the key's existing row set
+ if (previousRegionRowSet != null) {
+ existingRowSet.remove(previousRegionRowSet);
+ }
+ if (currentRegionRowSet != null) {
+ existingRowSet.insert(currentRegionRowSet);
+ }
+ if (modifiedPositionBuilder != null) {
+ // Note that once done processing everything, we're going to adjust this to pull out transitions _from_
+ // empty as adds and _to_ empty as removes.
+ modifiedPositionBuilder.addKey(pos);
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
index 274f603835a..cb71d45053f 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java
@@ -35,6 +35,9 @@
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
+import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK;
+import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.getFirstRowKey;
+
/**
* Manage column sources made up of regions in their own row key address space.
*/
@@ -369,10 +372,10 @@ private TableUpdateImpl update(final boolean initializing) {
if (entry.pollUpdates(addedRowSetBuilder)) {
// Changes were detected, update the row set in the table and mark the row/column as modified.
/*
- * Since TableLocationState.getRowSet() returns a copy(), we own entry.rowSetAtLastUpdate and can
- * propagate it without making another copy().
+ * We should consider adding an UpdateCommitter to close() the previous row sets for modified locations.
+ * This is not important for current implementations, since they always allocate new, flat RowSets.
*/
- rowSetSource.set(entry.regionIndex, entry.rowSetAtLastUpdate);
+ rowSetSource.set(entry.regionIndex, entry.rowSetAtLastUpdate.shift(getFirstRowKey(entry.regionIndex)));
if (modifiedRegionBuilder != null) {
modifiedRegionBuilder.appendKey(entry.regionIndex);
}
@@ -426,7 +429,7 @@ private TableUpdateImpl update(final boolean initializing) {
wcs.set(entry.regionIndex, entry.location.getKey().getPartitionValue(key)));
// @formatter:on
locationSource.set(entry.regionIndex, entry.location);
- rowSetSource.set(entry.regionIndex, entry.rowSetAtLastUpdate);
+ rowSetSource.set(entry.regionIndex, entry.rowSetAtLastUpdate.shift(getFirstRowKey(entry.regionIndex)));
addedRegionBuilder.appendKey(entry.regionIndex);
});
}
@@ -584,14 +587,14 @@ private void processInitial(final RowSetBuilderSequential addedRowSetBuilder, fi
Assert.neqNull(initialRowSet, "initialRowSet");
Assert.eqTrue(initialRowSet.isNonempty(), "initialRowSet.isNonempty()");
Assert.eqNull(rowSetAtLastUpdate, "rowSetAtLastUpdate");
- if (initialRowSet.lastRowKey() > RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK) {
+ if (initialRowSet.lastRowKey() > ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK) {
throw new TableDataException(String.format(
"Location %s has initial last key %#016X, larger than maximum supported key %#016X",
location, initialRowSet.lastRowKey(),
- RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK));
+ ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK));
}
- final long regionFirstKey = RegionedColumnSource.getFirstRowKey(regionIndex);
+ final long regionFirstKey = getFirstRowKey(regionIndex);
initialRowSet.forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> addedRowSetBuilder
.appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey));
@@ -644,11 +647,11 @@ private boolean pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) {
// Nothing to do
return false;
}
- if (updateRowSet.lastRowKey() > RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK) {
+ if (updateRowSet.lastRowKey() > ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK) {
throw new TableDataException(String.format(
"Location %s has updated last key %#016X, larger than maximum supported key %#016X",
location, updateRowSet.lastRowKey(),
- RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK));
+ ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK));
}
if (log.isDebugEnabled()) {
@@ -657,7 +660,7 @@ private boolean pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) {
.append(",TO:").append(updateRowSet.size()).endl();
}
try (final RowSet addedRowSet = updateRowSet.minus(rowSetAtLastUpdate)) {
- final long regionFirstKey = RegionedColumnSource.getFirstRowKey(regionIndex);
+ final long regionFirstKey = getFirstRowKey(regionIndex);
addedRowSet.forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> addedRowSetBuilder
.appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey));
}
From 273f5c137829ccc5c94bb147fb832d11dd2f3e93 Mon Sep 17 00:00:00 2001
From: Larry Booker
Date: Tue, 3 Sep 2024 15:28:43 -0700
Subject: [PATCH 23/56] Added additional snapshots with removes to
IcebergToolsTest resources.
---
.../iceberg/util/IcebergToolsTest.java | 110 ++++++++--
...efb-4a24-a6a6-46b2b34b986c-0-00001.parquet | 3 +
...e7d-4830-aedb-14dab6fabdd6-0-00001.parquet | 3 -
...fc0-40f8-9349-fcd5763dd691-0-00001.parquet | 3 +
...f10-4e08-89ae-f1b1b578ce63-0-00001.parquet | 3 -
...bcf-44c9-8482-90d09621c2ee-0-00001.parquet | 3 +
...256-4a04-9248-d7c69bec4881-0-00001.parquet | 3 -
...5de-4aac-bafc-cf750f9f7f70-0-00001.parquet | 3 +
...b08-46a6-bd9d-c8008dc66519-0-00001.parquet | 3 +
...3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet | 3 -
...9c89-46e8-b0f7-b4ec128b0f5e.metadata.json} | 30 +--
...5f28-4965-b390-ab426779517b.metadata.json} | 56 ++---
...fa56-47ec-9639-079621849f5a.metadata.json} | 82 +++----
...a6bf-4d78-b6f0-d927d58feb4c.metadata.json} | 106 ++++-----
...-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json | 186 ++++++++++++++++
...-1647-44c6-930e-df3fe7867d8f.metadata.json | 206 ++++++++++++++++++
...480b9-ac20-4445-b67e-7cd3f7ef27d0-m0.avro} | Bin 6976 -> 6977 bytes
...446ba-c04a-440a-afad-f090aa27c81a-m0.avro} | Bin 6978 -> 6978 bytes
...0d7049-0780-4636-972a-7f46bb6c395a-m0.avro | Bin 0 -> 6981 bytes
...2a84e-df73-4cb9-a92f-18d737a54489-m0.avro} | Bin 6978 -> 6978 bytes
...d1067-e40a-4222-bab9-959f5af4dd4d-m0.avro} | Bin 6978 -> 6978 bytes
...bb0d7049-0780-4636-972a-7f46bb6c395a.avro} | Bin 4302 -> 4226 bytes
...226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro} | Bin 4399 -> 4399 bytes
...bdc6d711-4699-4d21-a7f8-907c7740c70c.avro} | Bin 4224 -> 4095 bytes
...e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro} | Bin 4353 -> 4352 bytes
...-4b9446ba-c04a-440a-afad-f090aa27c81a.avro | Bin 0 -> 4301 bytes
...-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro | Bin 0 -> 4225 bytes
.../iceberg/util/IcebergCatalogAdapter.java | 1 -
28 files changed, 633 insertions(+), 171 deletions(-)
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-2-6f8d12ab-3efb-4a24-a6a6-46b2b34b986c-0-00001.parquet
delete mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-75fe700e-5fc0-40f8-9349-fcd5763dd691-0-00001.parquet
delete mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-4be14704-ebcf-44c9-8482-90d09621c2ee-0-00001.parquet
delete mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-efdf14cf-15de-4aac-bafc-cf750f9f7f70-0-00001.parquet
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-01e5d979-ab08-46a6-bd9d-c8008dc66519-0-00001.parquet
delete mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json => 00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json} (72%)
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json => 00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json} (64%)
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json => 00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json} (60%)
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json => 00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json} (58%)
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00005-3476adcd-1647-44c6-930e-df3fe7867d8f.metadata.json
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro => 226480b9-ac20-4445-b67e-7cd3f7ef27d0-m0.avro} (95%)
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro => 4b9446ba-c04a-440a-afad-f090aa27c81a-m0.avro} (95%)
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/bb0d7049-0780-4636-972a-7f46bb6c395a-m0.avro
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{77bfad1c-123c-452c-814d-298a1483a99f-m0.avro => ddb2a84e-df73-4cb9-a92f-18d737a54489-m0.avro} (95%)
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro => e1bd1067-e40a-4222-bab9-959f5af4dd4d-m0.avro} (95%)
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro => snap-1277776933184906785-1-bb0d7049-0780-4636-972a-7f46bb6c395a.avro} (90%)
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro => snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro} (88%)
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro => snap-3825168261540020388-1-bdc6d711-4699-4d21-a7f8-907c7740c70c.avro} (92%)
rename extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/{snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro => snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro} (89%)
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro
create mode 100644 extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro
diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java
index 07b6b127cdb..75fddadbc35 100644
--- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java
+++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java
@@ -235,20 +235,22 @@ public void testListSnapshots() {
adapter.listSnapshots(tableIdentifier)
.forEach(snapshot -> snapshotIds.add(snapshot.snapshotId()));
- Assert.eq(snapshotIds.size(), "snapshots.size()", 4, "4 snapshots for sales/sales_multi");
+ Assert.eq(snapshotIds.size(), "snapshots.size()", 6, "6 snapshots for sales/sales_multi");
- Assert.eqTrue(snapshotIds.contains(2001582482032951248L), "snapshots.contains(2001582482032951248)");
- Assert.eqTrue(snapshotIds.contains(8325605756612719366L), "snapshots.contains(8325605756612719366L)");
- Assert.eqTrue(snapshotIds.contains(3247344357341484163L), "snapshots.contains(3247344357341484163L)");
- Assert.eqTrue(snapshotIds.contains(1792185872197984875L), "snapshots.contains(1792185872197984875L)");
+ Assert.eqTrue(snapshotIds.contains(8121674823173822790L), "snapshots.contains(8121674823173822790L)");
+ Assert.eqTrue(snapshotIds.contains(6040260770188877244L), "snapshots.contains(6040260770188877244L)");
+ Assert.eqTrue(snapshotIds.contains(5693547373255393922L), "snapshots.contains(5693547373255393922L)");
+ Assert.eqTrue(snapshotIds.contains(3445166522731196553L), "snapshots.contains(3445166522731196553L)");
+ Assert.eqTrue(snapshotIds.contains(1277776933184906785L), "snapshots.contains(1277776933184906785L)");
+ Assert.eqTrue(snapshotIds.contains(3825168261540020388L), "snapshots.contains(3825168261540020388L)");
Table table = adapter.snapshots(tableIdentifier);
- Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi");
+ Assert.eq(table.size(), "table.size()", 6, "6 snapshots for sales/sales_multi");
Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION);
// Test the string versions of the methods
table = adapter.snapshots("sales.sales_multi");
- Assert.eq(table.size(), "table.size()", 4, "4 snapshots for sales/sales_multi");
+ Assert.eq(table.size(), "table.size()", 6, "6 snapshots for sales/sales_multi");
Assert.equals(table.getDefinition(), "table.getDefinition()", SNAPSHOT_DEFINITION);
}
@@ -286,14 +288,15 @@ public void testOpenTableB() throws ExecutionException, InterruptedException, Ti
final TableIdentifier tableId = TableIdentifier.of(ns, "sales_multi");
io.deephaven.engine.table.Table table = adapter.readTable(tableId, instructions);
- Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table");
+ // This table ends up with zero records
+ Assert.eq(table.size(), "table.size()", 0, "0 rows in the table");
Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION);
// Test the string versions of the methods
table = adapter.readTable("sales.sales_multi", instructions);
// Verify we retrieved all the rows.
- Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table");
+ Assert.eq(table.size(), "table.size()", 0, "0 rows in the table");
Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION);
}
@@ -729,23 +732,33 @@ public void testOpenTableSnapshot() throws ExecutionException, InterruptedExcept
// Verify we retrieved all the rows.
final io.deephaven.engine.table.Table table0 =
adapter.readTable(tableId, snapshots.get(0).snapshotId(), instructions);
- Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table");
+ Assert.eq(table0.size(), "table0.size()", 18073, "18266 rows in the table");
Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION);
final io.deephaven.engine.table.Table table1 =
adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions);
- Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table");
+ Assert.eq(table1.size(), "table1.size()", 54433, "54373 rows in the table");
Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION);
final io.deephaven.engine.table.Table table2 =
adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions);
- Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table");
+ Assert.eq(table2.size(), "table2.size()", 72551, "72603 rows in the table");
Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION);
final io.deephaven.engine.table.Table table3 =
adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions);
Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table");
Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION);
+
+ final io.deephaven.engine.table.Table table4 =
+ adapter.readTable(tableId, snapshots.get(4).snapshotId(), instructions);
+ Assert.eq(table4.size(), "table4.size()", 100_000, "100_000 rows in the table");
+ Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION);
+
+ final io.deephaven.engine.table.Table table5 =
+ adapter.readTable(tableId, snapshots.get(5).snapshotId(), instructions);
+ Assert.eq(table5.size(), "table5.size()", 0, "0 rows in the table");
+ Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION);
}
@Test
@@ -760,39 +773,55 @@ public void testOpenTableSnapshotByID() throws ExecutionException, InterruptedEx
// Verify we retrieved all the rows.
io.deephaven.engine.table.Table table0 = adapter.readTable(tableId, snapshots.get(0), instructions);
- Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table");
+ Assert.eq(table0.size(), "table0.size()", 18073, "18073 rows in the table");
Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION);
io.deephaven.engine.table.Table table1 = adapter.readTable(tableId, snapshots.get(1), instructions);
- Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table");
+ Assert.eq(table1.size(), "table1.size()", 54433, "54433 rows in the table");
Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION);
io.deephaven.engine.table.Table table2 = adapter.readTable(tableId, snapshots.get(2), instructions);
- Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table");
+ Assert.eq(table2.size(), "table2.size()", 72551, "72551 rows in the table");
Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION);
io.deephaven.engine.table.Table table3 = adapter.readTable(tableId, snapshots.get(3), instructions);
Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table");
Assert.equals(table3.getDefinition(), "table3.getDefinition()", SALES_MULTI_DEFINITION);
+ io.deephaven.engine.table.Table table4 = adapter.readTable(tableId, snapshots.get(4), instructions);
+ Assert.eq(table4.size(), "table4.size()", 100_000, "100_000 rows in the table");
+ Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION);
+
+ io.deephaven.engine.table.Table table5 = adapter.readTable(tableId, snapshots.get(5), instructions);
+ Assert.eq(table5.size(), "table5.size()", 0, "0 rows in the table");
+ Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION);
+
// Test the string versions of the methods
// Verify we retrieved all the rows.
table0 = adapter.readTable("sales.sales_multi", snapshots.get(0).snapshotId(), instructions);
- Assert.eq(table0.size(), "table0.size()", 18266, "18266 rows in the table");
+ Assert.eq(table0.size(), "table0.size()", 18073, "18073 rows in the table");
Assert.equals(table0.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION);
- table1 = adapter.readTable(tableId, snapshots.get(1).snapshotId(), instructions);
- Assert.eq(table1.size(), "table1.size()", 54373, "54373 rows in the table");
+ table1 = adapter.readTable("sales.sales_multi", snapshots.get(1).snapshotId(), instructions);
+ Assert.eq(table1.size(), "table1.size()", 54433, "54433 rows in the table");
Assert.equals(table1.getDefinition(), "table1.getDefinition()", SALES_MULTI_DEFINITION);
- table2 = adapter.readTable(tableId, snapshots.get(2).snapshotId(), instructions);
- Assert.eq(table2.size(), "table2.size()", 72603, "72603 rows in the table");
+ table2 = adapter.readTable("sales.sales_multi", snapshots.get(2).snapshotId(), instructions);
+ Assert.eq(table2.size(), "table2.size()", 72551, "72551 rows in the table");
Assert.equals(table2.getDefinition(), "table2.getDefinition()", SALES_MULTI_DEFINITION);
- table3 = adapter.readTable(tableId, snapshots.get(3).snapshotId(), instructions);
+ table3 = adapter.readTable("sales.sales_multi", snapshots.get(3).snapshotId(), instructions);
Assert.eq(table3.size(), "table3.size()", 100_000, "100_000 rows in the table");
Assert.equals(table3.getDefinition(), "table0.getDefinition()", SALES_MULTI_DEFINITION);
+
+ table4 = adapter.readTable("sales.sales_multi", snapshots.get(4).snapshotId(), instructions);
+ Assert.eq(table4.size(), "table4.size()", 100_000, "100_000 rows in the table");
+ Assert.equals(table4.getDefinition(), "table4.getDefinition()", SALES_MULTI_DEFINITION);
+
+ table5 = adapter.readTable("sales.sales_multi", snapshots.get(5).snapshotId(), instructions);
+ Assert.eq(table5.size(), "table5.size()", 0, "0 rows in the table");
+ Assert.equals(table5.getDefinition(), "table5.getDefinition()", SALES_MULTI_DEFINITION);
}
@Test
@@ -908,4 +937,43 @@ public void testTableDefinitionWithInstructions() {
tableDef = adapter.getTableDefinition("sales.sales_multi", localInstructions);
Assert.equals(tableDef, "tableDef", userTableDef);
}
+
+ @Test
+ public void testManualRefreshingTable() {
+ final IcebergCatalogAdapter adapter = IcebergTools.createAdapter(resourceCatalog);
+
+ final IcebergInstructions localInstructions = IcebergInstructions.builder()
+ .dataInstructions(instructions.dataInstructions().get())
+ .updateMode(IcebergUpdateMode.manualRefreshingMode())
+ .build();
+
+ final IcebergTableAdapter tableAdapter = adapter.loadTable("sales.sales_multi");
+
+ final List snapshots = tableAdapter.listSnapshots();
+
+ // Use string and current snapshot
+ final IcebergTable table = tableAdapter.table(snapshots.get(0).snapshotId(), localInstructions);
+ Assert.eq(table.size(), "table.size()", 18073, "18073 rows in the table");
+ Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION);
+
+ table.update(snapshots.get(1).snapshotId());
+ Assert.eq(table.size(), "table.size()", 54433, "54433 rows in the table");
+ Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION);
+
+ table.update(snapshots.get(2).snapshotId());
+ Assert.eq(table.size(), "table.size()", 72551, "72551 rows in the table");
+ Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION);
+
+ table.update(snapshots.get(3).snapshotId());
+ Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table");
+ Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION);
+
+ table.update(snapshots.get(4).snapshotId());
+ Assert.eq(table.size(), "table.size()", 100_000, "100_000 rows in the table");
+ Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION);
+
+ table.update(snapshots.get(5).snapshotId());
+ Assert.eq(table.size(), "table.size()", 0, "0 rows in the table");
+ Assert.equals(table.getDefinition(), "table.getDefinition()", SALES_MULTI_DEFINITION);
+ }
}
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-2-6f8d12ab-3efb-4a24-a6a6-46b2b34b986c-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-2-6f8d12ab-3efb-4a24-a6a6-46b2b34b986c-0-00001.parquet
new file mode 100644
index 00000000000..4ec41a05ae1
--- /dev/null
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-2-6f8d12ab-3efb-4a24-a6a6-46b2b34b986c-0-00001.parquet
@@ -0,0 +1,3 @@
+version https://git-lfs.github.com/spec/v1
+oid sha256:463d2a2753ebcf86deacc9cea3042918c2ad8adc2f124dd5ca2a5eb6cf86ae54
+size 175359
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet
deleted file mode 100644
index 04d259decea..00000000000
--- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-57e22ac8-7e7d-4830-aedb-14dab6fabdd6-0-00001.parquet
+++ /dev/null
@@ -1,3 +0,0 @@
-version https://git-lfs.github.com/spec/v1
-oid sha256:9749ae2922aa9d21b7e779142d6c2476d0444c2c24f7e93397e6750147180724
-size 176970
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-75fe700e-5fc0-40f8-9349-fcd5763dd691-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-75fe700e-5fc0-40f8-9349-fcd5763dd691-0-00001.parquet
new file mode 100644
index 00000000000..634d73c5d51
--- /dev/null
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-3-75fe700e-5fc0-40f8-9349-fcd5763dd691-0-00001.parquet
@@ -0,0 +1,3 @@
+version https://git-lfs.github.com/spec/v1
+oid sha256:432d8cdd6e09203748f800b914728a63b2541b99fceaa215d764910adaa000e3
+size 333827
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet
deleted file mode 100644
index 206e1f84a6c..00000000000
--- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-3aacc74b-7f10-4e08-89ae-f1b1b578ce63-0-00001.parquet
+++ /dev/null
@@ -1,3 +0,0 @@
-version https://git-lfs.github.com/spec/v1
-oid sha256:d5007c70d0e8be33ef012c48d7f067b47812b22747c145cfa4bab4bef944fb0f
-size 331675
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-4be14704-ebcf-44c9-8482-90d09621c2ee-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-4be14704-ebcf-44c9-8482-90d09621c2ee-0-00001.parquet
new file mode 100644
index 00000000000..1cb0281c073
--- /dev/null
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-4-4be14704-ebcf-44c9-8482-90d09621c2ee-0-00001.parquet
@@ -0,0 +1,3 @@
+version https://git-lfs.github.com/spec/v1
+oid sha256:3f5d595ce8cd13180368f05c2dfe57f4f2c8fba7c7437066bf998fc805662dc0
+size 175594
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet
deleted file mode 100644
index 6d552bf0e9a..00000000000
--- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-47debf3c-b256-4a04-9248-d7c69bec4881-0-00001.parquet
+++ /dev/null
@@ -1,3 +0,0 @@
-version https://git-lfs.github.com/spec/v1
-oid sha256:c02909448106787f93e8a2550e9db360aedd9ab25c925e8a536b18096aa8ed91
-size 176482
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-efdf14cf-15de-4aac-bafc-cf750f9f7f70-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-efdf14cf-15de-4aac-bafc-cf750f9f7f70-0-00001.parquet
new file mode 100644
index 00000000000..3455ea39555
--- /dev/null
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-5-efdf14cf-15de-4aac-bafc-cf750f9f7f70-0-00001.parquet
@@ -0,0 +1,3 @@
+version https://git-lfs.github.com/spec/v1
+oid sha256:c127833dec01dac8258db8dbd0acd9a3bc7e23957e4dce46c0a07c1174498875
+size 257345
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-01e5d979-ab08-46a6-bd9d-c8008dc66519-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-01e5d979-ab08-46a6-bd9d-c8008dc66519-0-00001.parquet
new file mode 100644
index 00000000000..1a06c24b196
--- /dev/null
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-01e5d979-ab08-46a6-bd9d-c8008dc66519-0-00001.parquet
@@ -0,0 +1,3 @@
+version https://git-lfs.github.com/spec/v1
+oid sha256:d5dbc201439bcf1bf4afb778a5cfc4e40a6b5d9d1a9a9b77cacd25bea21c2486
+size 729342
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet
deleted file mode 100644
index 540e5bc3392..00000000000
--- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/data/00000-6-7860a7c9-b3cb-4b2a-a69b-0f1e5c068512-0-00001.parquet
+++ /dev/null
@@ -1,3 +0,0 @@
-version https://git-lfs.github.com/spec/v1
-oid sha256:c872a3defa86826496fbc76fa4235c858d45edb5441e196c2c812f4c8a1166e7
-size 256855
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json
similarity index 72%
rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json
rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json
index 53c1afcba7b..90bd5751ba7 100644
--- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json
@@ -1,9 +1,9 @@
{
"format-version" : 2,
- "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512",
+ "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79",
"location" : "s3://warehouse/sales/sales_multi",
"last-sequence-number" : 1,
- "last-updated-ms" : 1716234545155,
+ "last-updated-ms" : 1725396294486,
"last-column-id" : 5,
"current-schema-id" : 0,
"schemas" : [ {
@@ -49,43 +49,43 @@
} ],
"properties" : {
"owner" : "root",
- "created-at" : "2024-05-20T19:49:04.669537174Z",
+ "created-at" : "2024-09-03T20:44:52.796187633Z",
"write.format.default" : "parquet",
"write.parquet.compression-codec" : "zstd"
},
- "current-snapshot-id" : 2001582482032951248,
+ "current-snapshot-id" : 8121674823173822790,
"refs" : {
"main" : {
- "snapshot-id" : 2001582482032951248,
+ "snapshot-id" : 8121674823173822790,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
- "snapshot-id" : 2001582482032951248,
- "timestamp-ms" : 1716234545155,
+ "snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396294486,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "18266",
- "added-files-size" : "176970",
+ "added-records" : "18073",
+ "added-files-size" : "175359",
"changed-partition-count" : "1",
- "total-records" : "18266",
- "total-files-size" : "176970",
+ "total-records" : "18073",
+ "total-files-size" : "175359",
"total-data-files" : "1",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"partition-statistics" : [ ],
"snapshot-log" : [ {
- "timestamp-ms" : 1716234545155,
- "snapshot-id" : 2001582482032951248
+ "timestamp-ms" : 1725396294486,
+ "snapshot-id" : 8121674823173822790
} ],
"metadata-log" : [ ]
}
\ No newline at end of file
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json
similarity index 64%
rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json
rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json
index 5aa1996ce92..9ce7f8b1bd1 100644
--- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json
@@ -1,9 +1,9 @@
{
"format-version" : 2,
- "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512",
+ "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79",
"location" : "s3://warehouse/sales/sales_multi",
"last-sequence-number" : 2,
- "last-updated-ms" : 1716234545529,
+ "last-updated-ms" : 1725396298768,
"last-column-id" : 5,
"current-schema-id" : 0,
"schemas" : [ {
@@ -49,70 +49,70 @@
} ],
"properties" : {
"owner" : "root",
- "created-at" : "2024-05-20T19:49:04.669537174Z",
+ "created-at" : "2024-09-03T20:44:52.796187633Z",
"write.format.default" : "parquet",
"write.parquet.compression-codec" : "zstd"
},
- "current-snapshot-id" : 8325605756612719366,
+ "current-snapshot-id" : 6040260770188877244,
"refs" : {
"main" : {
- "snapshot-id" : 8325605756612719366,
+ "snapshot-id" : 6040260770188877244,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
- "snapshot-id" : 2001582482032951248,
- "timestamp-ms" : 1716234545155,
+ "snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396294486,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "18266",
- "added-files-size" : "176970",
+ "added-records" : "18073",
+ "added-files-size" : "175359",
"changed-partition-count" : "1",
- "total-records" : "18266",
- "total-files-size" : "176970",
+ "total-records" : "18073",
+ "total-files-size" : "175359",
"total-data-files" : "1",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
- "snapshot-id" : 8325605756612719366,
- "parent-snapshot-id" : 2001582482032951248,
- "timestamp-ms" : 1716234545529,
+ "snapshot-id" : 6040260770188877244,
+ "parent-snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396298768,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "36107",
- "added-files-size" : "331675",
+ "added-records" : "36360",
+ "added-files-size" : "333827",
"changed-partition-count" : "1",
- "total-records" : "54373",
- "total-files-size" : "508645",
+ "total-records" : "54433",
+ "total-files-size" : "509186",
"total-data-files" : "2",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"partition-statistics" : [ ],
"snapshot-log" : [ {
- "timestamp-ms" : 1716234545155,
- "snapshot-id" : 2001582482032951248
+ "timestamp-ms" : 1725396294486,
+ "snapshot-id" : 8121674823173822790
}, {
- "timestamp-ms" : 1716234545529,
- "snapshot-id" : 8325605756612719366
+ "timestamp-ms" : 1725396298768,
+ "snapshot-id" : 6040260770188877244
} ],
"metadata-log" : [ {
- "timestamp-ms" : 1716234545155,
- "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json"
+ "timestamp-ms" : 1725396294486,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json"
} ]
}
\ No newline at end of file
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json
similarity index 60%
rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json
rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json
index 1be00194c4e..f233d8d55b1 100644
--- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json
@@ -1,9 +1,9 @@
{
"format-version" : 2,
- "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512",
+ "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79",
"location" : "s3://warehouse/sales/sales_multi",
"last-sequence-number" : 3,
- "last-updated-ms" : 1716234545865,
+ "last-updated-ms" : 1725396300069,
"last-column-id" : 5,
"current-schema-id" : 0,
"schemas" : [ {
@@ -49,97 +49,97 @@
} ],
"properties" : {
"owner" : "root",
- "created-at" : "2024-05-20T19:49:04.669537174Z",
+ "created-at" : "2024-09-03T20:44:52.796187633Z",
"write.format.default" : "parquet",
"write.parquet.compression-codec" : "zstd"
},
- "current-snapshot-id" : 3247344357341484163,
+ "current-snapshot-id" : 5693547373255393922,
"refs" : {
"main" : {
- "snapshot-id" : 3247344357341484163,
+ "snapshot-id" : 5693547373255393922,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
- "snapshot-id" : 2001582482032951248,
- "timestamp-ms" : 1716234545155,
+ "snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396294486,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "18266",
- "added-files-size" : "176970",
+ "added-records" : "18073",
+ "added-files-size" : "175359",
"changed-partition-count" : "1",
- "total-records" : "18266",
- "total-files-size" : "176970",
+ "total-records" : "18073",
+ "total-files-size" : "175359",
"total-data-files" : "1",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
- "snapshot-id" : 8325605756612719366,
- "parent-snapshot-id" : 2001582482032951248,
- "timestamp-ms" : 1716234545529,
+ "snapshot-id" : 6040260770188877244,
+ "parent-snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396298768,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "36107",
- "added-files-size" : "331675",
+ "added-records" : "36360",
+ "added-files-size" : "333827",
"changed-partition-count" : "1",
- "total-records" : "54373",
- "total-files-size" : "508645",
+ "total-records" : "54433",
+ "total-files-size" : "509186",
"total-data-files" : "2",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
- "snapshot-id" : 3247344357341484163,
- "parent-snapshot-id" : 8325605756612719366,
- "timestamp-ms" : 1716234545865,
+ "snapshot-id" : 5693547373255393922,
+ "parent-snapshot-id" : 6040260770188877244,
+ "timestamp-ms" : 1725396300069,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "18230",
- "added-files-size" : "176482",
+ "added-records" : "18118",
+ "added-files-size" : "175594",
"changed-partition-count" : "1",
- "total-records" : "72603",
- "total-files-size" : "685127",
+ "total-records" : "72551",
+ "total-files-size" : "684780",
"total-data-files" : "3",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"partition-statistics" : [ ],
"snapshot-log" : [ {
- "timestamp-ms" : 1716234545155,
- "snapshot-id" : 2001582482032951248
+ "timestamp-ms" : 1725396294486,
+ "snapshot-id" : 8121674823173822790
}, {
- "timestamp-ms" : 1716234545529,
- "snapshot-id" : 8325605756612719366
+ "timestamp-ms" : 1725396298768,
+ "snapshot-id" : 6040260770188877244
}, {
- "timestamp-ms" : 1716234545865,
- "snapshot-id" : 3247344357341484163
+ "timestamp-ms" : 1725396300069,
+ "snapshot-id" : 5693547373255393922
} ],
"metadata-log" : [ {
- "timestamp-ms" : 1716234545155,
- "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json"
+ "timestamp-ms" : 1725396294486,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json"
}, {
- "timestamp-ms" : 1716234545529,
- "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json"
+ "timestamp-ms" : 1725396298768,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json"
} ]
}
\ No newline at end of file
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json
similarity index 58%
rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json
rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json
index 16cca0f3f8d..f63543c318f 100644
--- a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-366e4cb2-48c8-4f6d-bd6b-9b452de74dc3.metadata.json
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json
@@ -1,9 +1,9 @@
{
"format-version" : 2,
- "table-uuid" : "49ca5b8c-6402-40f9-96b0-4c09f9f1b512",
+ "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79",
"location" : "s3://warehouse/sales/sales_multi",
"last-sequence-number" : 4,
- "last-updated-ms" : 1716234546189,
+ "last-updated-ms" : 1725396301293,
"last-column-id" : 5,
"current-schema-id" : 0,
"schemas" : [ {
@@ -49,124 +49,124 @@
} ],
"properties" : {
"owner" : "root",
- "created-at" : "2024-05-20T19:49:04.669537174Z",
+ "created-at" : "2024-09-03T20:44:52.796187633Z",
"write.format.default" : "parquet",
"write.parquet.compression-codec" : "zstd"
},
- "current-snapshot-id" : 1792185872197984875,
+ "current-snapshot-id" : 3445166522731196553,
"refs" : {
"main" : {
- "snapshot-id" : 1792185872197984875,
+ "snapshot-id" : 3445166522731196553,
"type" : "branch"
}
},
"snapshots" : [ {
"sequence-number" : 1,
- "snapshot-id" : 2001582482032951248,
- "timestamp-ms" : 1716234545155,
+ "snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396294486,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "18266",
- "added-files-size" : "176970",
+ "added-records" : "18073",
+ "added-files-size" : "175359",
"changed-partition-count" : "1",
- "total-records" : "18266",
- "total-files-size" : "176970",
+ "total-records" : "18073",
+ "total-files-size" : "175359",
"total-data-files" : "1",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro",
"schema-id" : 0
}, {
"sequence-number" : 2,
- "snapshot-id" : 8325605756612719366,
- "parent-snapshot-id" : 2001582482032951248,
- "timestamp-ms" : 1716234545529,
+ "snapshot-id" : 6040260770188877244,
+ "parent-snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396298768,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "36107",
- "added-files-size" : "331675",
+ "added-records" : "36360",
+ "added-files-size" : "333827",
"changed-partition-count" : "1",
- "total-records" : "54373",
- "total-files-size" : "508645",
+ "total-records" : "54433",
+ "total-files-size" : "509186",
"total-data-files" : "2",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8325605756612719366-1-02f91282-fecb-4479-bd3b-20a8a3aaa795.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro",
"schema-id" : 0
}, {
"sequence-number" : 3,
- "snapshot-id" : 3247344357341484163,
- "parent-snapshot-id" : 8325605756612719366,
- "timestamp-ms" : 1716234545865,
+ "snapshot-id" : 5693547373255393922,
+ "parent-snapshot-id" : 6040260770188877244,
+ "timestamp-ms" : 1725396300069,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "18230",
- "added-files-size" : "176482",
+ "added-records" : "18118",
+ "added-files-size" : "175594",
"changed-partition-count" : "1",
- "total-records" : "72603",
- "total-files-size" : "685127",
+ "total-records" : "72551",
+ "total-files-size" : "684780",
"total-data-files" : "3",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3247344357341484163-1-77bfad1c-123c-452c-814d-298a1483a99f.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro",
"schema-id" : 0
}, {
"sequence-number" : 4,
- "snapshot-id" : 1792185872197984875,
- "parent-snapshot-id" : 3247344357341484163,
- "timestamp-ms" : 1716234546189,
+ "snapshot-id" : 3445166522731196553,
+ "parent-snapshot-id" : 5693547373255393922,
+ "timestamp-ms" : 1725396301293,
"summary" : {
"operation" : "append",
- "spark.app.id" : "local-1716234462547",
+ "spark.app.id" : "local-1725396284766",
"added-data-files" : "1",
- "added-records" : "27397",
- "added-files-size" : "256855",
+ "added-records" : "27449",
+ "added-files-size" : "257345",
"changed-partition-count" : "1",
"total-records" : "100000",
- "total-files-size" : "941982",
+ "total-files-size" : "942125",
"total-data-files" : "4",
"total-delete-files" : "0",
"total-position-deletes" : "0",
"total-equality-deletes" : "0"
},
- "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-1792185872197984875-1-51be90a2-7294-4ebb-8eee-20d512fff8b0.avro",
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro",
"schema-id" : 0
} ],
"statistics" : [ ],
"partition-statistics" : [ ],
"snapshot-log" : [ {
- "timestamp-ms" : 1716234545155,
- "snapshot-id" : 2001582482032951248
+ "timestamp-ms" : 1725396294486,
+ "snapshot-id" : 8121674823173822790
}, {
- "timestamp-ms" : 1716234545529,
- "snapshot-id" : 8325605756612719366
+ "timestamp-ms" : 1725396298768,
+ "snapshot-id" : 6040260770188877244
}, {
- "timestamp-ms" : 1716234545865,
- "snapshot-id" : 3247344357341484163
+ "timestamp-ms" : 1725396300069,
+ "snapshot-id" : 5693547373255393922
}, {
- "timestamp-ms" : 1716234546189,
- "snapshot-id" : 1792185872197984875
+ "timestamp-ms" : 1725396301293,
+ "snapshot-id" : 3445166522731196553
} ],
"metadata-log" : [ {
- "timestamp-ms" : 1716234545155,
- "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-8bdf65c1-a414-468b-b7b2-78558b2e8c1f.metadata.json"
+ "timestamp-ms" : 1725396294486,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json"
}, {
- "timestamp-ms" : 1716234545529,
- "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-30972e0f-9c8a-4fa9-911d-e356b2b04061.metadata.json"
+ "timestamp-ms" : 1725396298768,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json"
}, {
- "timestamp-ms" : 1716234545865,
- "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-559f7323-a010-4afe-8461-f5261787aae9.metadata.json"
+ "timestamp-ms" : 1725396300069,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json"
} ]
}
\ No newline at end of file
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json
new file mode 100644
index 00000000000..32367a81a58
--- /dev/null
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json
@@ -0,0 +1,186 @@
+{
+ "format-version" : 2,
+ "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79",
+ "location" : "s3://warehouse/sales/sales_multi",
+ "last-sequence-number" : 5,
+ "last-updated-ms" : 1725396302818,
+ "last-column-id" : 5,
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "Region",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 2,
+ "name" : "Item_Type",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 3,
+ "name" : "Units_Sold",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 4,
+ "name" : "Unit_Price",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 5,
+ "name" : "Order_Date",
+ "required" : false,
+ "type" : "timestamptz"
+ } ]
+ } ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "owner" : "root",
+ "created-at" : "2024-09-03T20:44:52.796187633Z",
+ "write.format.default" : "parquet",
+ "write.parquet.compression-codec" : "zstd"
+ },
+ "current-snapshot-id" : 1277776933184906785,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 1277776933184906785,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "sequence-number" : 1,
+ "snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396294486,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "18073",
+ "added-files-size" : "175359",
+ "changed-partition-count" : "1",
+ "total-records" : "18073",
+ "total-files-size" : "175359",
+ "total-data-files" : "1",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro",
+ "schema-id" : 0
+ }, {
+ "sequence-number" : 2,
+ "snapshot-id" : 6040260770188877244,
+ "parent-snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396298768,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "36360",
+ "added-files-size" : "333827",
+ "changed-partition-count" : "1",
+ "total-records" : "54433",
+ "total-files-size" : "509186",
+ "total-data-files" : "2",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro",
+ "schema-id" : 0
+ }, {
+ "sequence-number" : 3,
+ "snapshot-id" : 5693547373255393922,
+ "parent-snapshot-id" : 6040260770188877244,
+ "timestamp-ms" : 1725396300069,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "18118",
+ "added-files-size" : "175594",
+ "changed-partition-count" : "1",
+ "total-records" : "72551",
+ "total-files-size" : "684780",
+ "total-data-files" : "3",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro",
+ "schema-id" : 0
+ }, {
+ "sequence-number" : 4,
+ "snapshot-id" : 3445166522731196553,
+ "parent-snapshot-id" : 5693547373255393922,
+ "timestamp-ms" : 1725396301293,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "27449",
+ "added-files-size" : "257345",
+ "changed-partition-count" : "1",
+ "total-records" : "100000",
+ "total-files-size" : "942125",
+ "total-data-files" : "4",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro",
+ "schema-id" : 0
+ }, {
+ "sequence-number" : 5,
+ "snapshot-id" : 1277776933184906785,
+ "timestamp-ms" : 1725396302818,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "100000",
+ "added-files-size" : "729342",
+ "changed-partition-count" : "1",
+ "total-records" : "100000",
+ "total-files-size" : "729342",
+ "total-data-files" : "1",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-1277776933184906785-1-bb0d7049-0780-4636-972a-7f46bb6c395a.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ ],
+ "partition-statistics" : [ ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1725396302818,
+ "snapshot-id" : 1277776933184906785
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1725396294486,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json"
+ }, {
+ "timestamp-ms" : 1725396298768,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json"
+ }, {
+ "timestamp-ms" : 1725396300069,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json"
+ }, {
+ "timestamp-ms" : 1725396301293,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00005-3476adcd-1647-44c6-930e-df3fe7867d8f.metadata.json b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00005-3476adcd-1647-44c6-930e-df3fe7867d8f.metadata.json
new file mode 100644
index 00000000000..fe4f6176a8d
--- /dev/null
+++ b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/00005-3476adcd-1647-44c6-930e-df3fe7867d8f.metadata.json
@@ -0,0 +1,206 @@
+{
+ "format-version" : 2,
+ "table-uuid" : "533f8120-9943-4b64-a0cf-6cfab18e4a79",
+ "location" : "s3://warehouse/sales/sales_multi",
+ "last-sequence-number" : 6,
+ "last-updated-ms" : 1725396304540,
+ "last-column-id" : 5,
+ "current-schema-id" : 0,
+ "schemas" : [ {
+ "type" : "struct",
+ "schema-id" : 0,
+ "fields" : [ {
+ "id" : 1,
+ "name" : "Region",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 2,
+ "name" : "Item_Type",
+ "required" : false,
+ "type" : "string"
+ }, {
+ "id" : 3,
+ "name" : "Units_Sold",
+ "required" : false,
+ "type" : "int"
+ }, {
+ "id" : 4,
+ "name" : "Unit_Price",
+ "required" : false,
+ "type" : "double"
+ }, {
+ "id" : 5,
+ "name" : "Order_Date",
+ "required" : false,
+ "type" : "timestamptz"
+ } ]
+ } ],
+ "default-spec-id" : 0,
+ "partition-specs" : [ {
+ "spec-id" : 0,
+ "fields" : [ ]
+ } ],
+ "last-partition-id" : 999,
+ "default-sort-order-id" : 0,
+ "sort-orders" : [ {
+ "order-id" : 0,
+ "fields" : [ ]
+ } ],
+ "properties" : {
+ "owner" : "root",
+ "created-at" : "2024-09-03T20:44:52.796187633Z",
+ "write.format.default" : "parquet",
+ "write.parquet.compression-codec" : "zstd"
+ },
+ "current-snapshot-id" : 3825168261540020388,
+ "refs" : {
+ "main" : {
+ "snapshot-id" : 3825168261540020388,
+ "type" : "branch"
+ }
+ },
+ "snapshots" : [ {
+ "sequence-number" : 1,
+ "snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396294486,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "18073",
+ "added-files-size" : "175359",
+ "changed-partition-count" : "1",
+ "total-records" : "18073",
+ "total-files-size" : "175359",
+ "total-data-files" : "1",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-8121674823173822790-1-ddb2a84e-df73-4cb9-a92f-18d737a54489.avro",
+ "schema-id" : 0
+ }, {
+ "sequence-number" : 2,
+ "snapshot-id" : 6040260770188877244,
+ "parent-snapshot-id" : 8121674823173822790,
+ "timestamp-ms" : 1725396298768,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "36360",
+ "added-files-size" : "333827",
+ "changed-partition-count" : "1",
+ "total-records" : "54433",
+ "total-files-size" : "509186",
+ "total-data-files" : "2",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro",
+ "schema-id" : 0
+ }, {
+ "sequence-number" : 3,
+ "snapshot-id" : 5693547373255393922,
+ "parent-snapshot-id" : 6040260770188877244,
+ "timestamp-ms" : 1725396300069,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "18118",
+ "added-files-size" : "175594",
+ "changed-partition-count" : "1",
+ "total-records" : "72551",
+ "total-files-size" : "684780",
+ "total-data-files" : "3",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-5693547373255393922-1-e1bd1067-e40a-4222-bab9-959f5af4dd4d.avro",
+ "schema-id" : 0
+ }, {
+ "sequence-number" : 4,
+ "snapshot-id" : 3445166522731196553,
+ "parent-snapshot-id" : 5693547373255393922,
+ "timestamp-ms" : 1725396301293,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "27449",
+ "added-files-size" : "257345",
+ "changed-partition-count" : "1",
+ "total-records" : "100000",
+ "total-files-size" : "942125",
+ "total-data-files" : "4",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3445166522731196553-1-226480b9-ac20-4445-b67e-7cd3f7ef27d0.avro",
+ "schema-id" : 0
+ }, {
+ "sequence-number" : 5,
+ "snapshot-id" : 1277776933184906785,
+ "timestamp-ms" : 1725396302818,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "added-data-files" : "1",
+ "added-records" : "100000",
+ "added-files-size" : "729342",
+ "changed-partition-count" : "1",
+ "total-records" : "100000",
+ "total-files-size" : "729342",
+ "total-data-files" : "1",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-1277776933184906785-1-bb0d7049-0780-4636-972a-7f46bb6c395a.avro",
+ "schema-id" : 0
+ }, {
+ "sequence-number" : 6,
+ "snapshot-id" : 3825168261540020388,
+ "timestamp-ms" : 1725396304540,
+ "summary" : {
+ "operation" : "append",
+ "spark.app.id" : "local-1725396284766",
+ "changed-partition-count" : "0",
+ "total-records" : "0",
+ "total-files-size" : "0",
+ "total-data-files" : "0",
+ "total-delete-files" : "0",
+ "total-position-deletes" : "0",
+ "total-equality-deletes" : "0"
+ },
+ "manifest-list" : "s3://warehouse/sales/sales_multi/metadata/snap-3825168261540020388-1-bdc6d711-4699-4d21-a7f8-907c7740c70c.avro",
+ "schema-id" : 0
+ } ],
+ "statistics" : [ ],
+ "partition-statistics" : [ ],
+ "snapshot-log" : [ {
+ "timestamp-ms" : 1725396304540,
+ "snapshot-id" : 3825168261540020388
+ } ],
+ "metadata-log" : [ {
+ "timestamp-ms" : 1725396294486,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00000-e433b4ac-9c89-46e8-b0f7-b4ec128b0f5e.metadata.json"
+ }, {
+ "timestamp-ms" : 1725396298768,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00001-1e0eb3be-5f28-4965-b390-ab426779517b.metadata.json"
+ }, {
+ "timestamp-ms" : 1725396300069,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00002-b1745d7e-fa56-47ec-9639-079621849f5a.metadata.json"
+ }, {
+ "timestamp-ms" : 1725396301293,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00003-1d6e8486-a6bf-4d78-b6f0-d927d58feb4c.metadata.json"
+ }, {
+ "timestamp-ms" : 1725396302818,
+ "metadata-file" : "s3://warehouse/sales/sales_multi/metadata/00004-bdf13c5f-b50f-4ccf-94a0-14c7b9e6ccb7.metadata.json"
+ } ]
+}
\ No newline at end of file
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/226480b9-ac20-4445-b67e-7cd3f7ef27d0-m0.avro
similarity index 95%
rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/02f91282-fecb-4479-bd3b-20a8a3aaa795-m0.avro
rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/226480b9-ac20-4445-b67e-7cd3f7ef27d0-m0.avro
index d871fbb9fd098d297903c3e1dcd7f46abe821c6e..2de91cb1f3498958d4ca022d64344afaf83b7f0e 100644
GIT binary patch
delta 315
zcmV-B0mS~mHo-Qq5E_5xdjE7NJO^-6$Ep_b4_~qZoCITHnsjLW_WNsYoR4Q(sPb4oJxQxZ!O^$kEk*Hkw(EhWv+Bsopj&@?4g
z*Ca79SvM&$Em=1?&D_)=%`(kA&D=m22!X&*uOP9gur#%VFTj5>C@|DDn+$=+jh12+&;7^
zcRri2_+|UFX|iu}w7(Sy)
za!IqWu~4_rXAzmpU7I3Y4jBbq4)zwB+#r|oSRjBsllA45iYrW4ZvFLn^>gQUhqVE(
JB^b~{2mlTkiNgQ@
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/4b9446ba-c04a-440a-afad-f090aa27c81a-m0.avro
similarity index 95%
rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/51be90a2-7294-4ebb-8eee-20d512fff8b0-m0.avro
rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/4b9446ba-c04a-440a-afad-f090aa27c81a-m0.avro
index 21c52f9fe1cc6652cda25045a38657547255ea89..d91a8b5dc742a129031b1bdffc135b033a882d58 100644
GIT binary patch
delta 316
zcmV-C0mJ^nHo`Wr5E_3;gM=+}Te^;EkhX-Qz$PRDo&;lJ`tfbu-Fv52Twcz|z`$^X
zvDny3U%x!DC^aL$v^Z72I58)+7(&M9mgbaX>Zc@@Bh`E3mB+S6Vz{bGAz{SA8#Kge>
z65((x&P-$xa!O39RB+4BPhsN#`GZ41Gp7I0*A+?*TmlA`hnC-d!wOcd5L}w18=RPt
zSd^Hj;FwmF#UeBkmYSYgl9&YaARC8(I!G%>r^cfepjOAYWjD^f2Wo{{#{d9HgM=+}
OTe^;EkhX-Qz$PT4#hCs8
delta 316
zcmX?PcF1glfK+|^n;$dY`pvxbVAa)GuO&XFGR(80>uo5;sVVrk`WR&BrGI$NJz|>
zVIkq5B-3|vW~YvmU#I6(uDqW=HFIQo9O?@=GLC(b|Fz5MWTSV&%F92`Zoj{wDO8(f
zy4p)N^)wYR)n}R(%S2N*c9^s*UABChiW;NsiDQZj_%3l>TKRDW>k3o-4;Mcdzi*6M
Lc+HT30X>WWF7=GM
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/bb0d7049-0780-4636-972a-7f46bb6c395a-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/bb0d7049-0780-4636-972a-7f46bb6c395a-m0.avro
new file mode 100644
index 0000000000000000000000000000000000000000..463995b0f52698b291b22bb0b7438348b572b1c3
GIT binary patch
literal 6981
zcmb_gTWl0n7?v&ujnHTj@KWJ8i3xJTuBTzTTxaqimQwVSNd0I}|g|*gvSqv}l
zHg(UHhAbl!aa%XNP;YC|RcVK#t5J$GR$ei*@OE2xd%Kg-9BGT}McwxFoaTCR&h`#+
z)1%X?kX5KL1U&Df!jK;Z)mX-5{$EEReRua!jg3xNFs!3)(r)W+c
z(nLmDRyQ=sU9iYjMr20zWC_a+O{)sBBfyz*U{%ZX0QT@c=Odf3RLJqo{!15GS>%mt
z2y05CL?>1oM|`Jkxw;2cfHIn)d72wXLMQkq8@e|YJPWjvF!3U5Jpsg)y>Y4?st^|q
z#D^HrZ>cg-uzRo%9nigT)H{SoK4FiEH@@6Nfx;%sRgPNq9k}?Gm
z4}JFwD^Re@@??Xg3v%H&79SjEgT^L0AMu+>#n6AnOY3pe&C)?2f
z0yJKcZH%?=A<{F*v*3pjBH-Zq(cS%EQT3*jNR`^;2Y7Ly>Hw&OL_JV-l$u3xEo2-6
zm5^8fR3x9{kV&eM&p?krxN#%d%%A~zAR0T?NPJ5oL0v=u1s(%fRN~<@iq18G?I;h-
zH9@EgK!Owz4ms#|h3c8cvz^qG%|!=GOaWM8M;+NFmK8x%wuv2ef`vytu2;w_L{JOc
zqhG<1$tJSR!>|s(&ZGe5utEk6X8}D;XvcY$tOGisy?UWm=*&X0F6@ZEi1-@`lF-Ij
zleg`~1qpXEv7T-wu#KXm7%PAz*4NI6zqNqGW;=8dDW`cd-V(|(iSvyM=S`2PI#V{;
zo8TS_?jgmnl*qV_UsKNBrJqE*eeTLpGb${5EkpnKr*1l0@>Bu
zk;tw#!EOng{X8u4H`-!@+NnW3!NRIqIasoE7v?b;TBvUe))HYGo5rr?09u43RpK4k
zh%i7ya2A$KAXK0uQbntuV|0k(3GgJLRjQTI)JCw*08ZBIud*$Zn&ZMk-XL({zSx5I
zMTkQ~q8X?k^tQ1ABfGjWT%T$qzd(lriWSf?N`8SpQ|^ns#U)-$ncX5{Mz0q0X0kT~
zmnrcrCr&j9p!Aopf;y#8^3F0YY5?neIjAIh%r^v)8b#?>Wnqg?#{y*e1Pv#Fra;81
z*ik~CqKFyQK);p{4Iw3~O-FBy6piYmVG6S!s;jcLh%hYzJ)<_!3@#yNmCXwJr_X<*
zi5gWx6Uxu>5{g!fs;ZQYYDtClUqaTX{Gw70A;xql&cQ75#{r)t|G)RA~X3EdDOEei9c0`Hio7a}B56)!{x-NL_RJ_&$b;FF@8GWm>39<)#;53C$^f2ZctvM*WHspOhUMyOsgB2zN;o6m06q4^>m
zLXZ#a5?#f7E;PVzF_GIeRYPY_>Ec{PPuLv|-NoM-ix*TF8-)kZVhW#9f)i9=QA8_j
z*3>+jgC$Ee6;?Tj6|I9rq}CPh*Hm7SanW0LXlE;JWu*%>6ps5Ugv>AGl?|tss!F=`_Mc}ynj850)$G1~
zsY}C#@yR)9X83~-k4YmZKb20Ncz^JGf9OwcxFYltbO*f7j%?*E{v*@E|j*c82`KMuB
QXV<;8r&mop7b^qlf1tq^#{d8T
literal 0
HcmV?d00001
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/ddb2a84e-df73-4cb9-a92f-18d737a54489-m0.avro
similarity index 95%
rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/77bfad1c-123c-452c-814d-298a1483a99f-m0.avro
rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/ddb2a84e-df73-4cb9-a92f-18d737a54489-m0.avro
index 9501a6ce37e2dbcae20429b8d3568c73ba343666..3e3c02833a0ee7470e59c3f9e8a9d42accf7081e 100644
GIT binary patch
delta 298
zcmX?PcF1glfK+{#TM-kZ4F8OPtBI?Qx3Q!!&0|SUdR1w)GTH0?{R~$W6`bt)wIevVjOsc(h?FAk{={INJ>aZ
zdX&J>SzsZSuC|H8QdKQfYsb^)%QqA@{5hz&K=N|^i`QJL?0SuEdzZH7?@wf19mb*O
rdx_WYjE9cbncxB+?NdpDlli2@wpuK(M`G>;`YDQowZtgmx^%B#B=8W_kV>X>c2
zbY0(H$Ls6~eO=V4PSws|YzhmqS7j&7n>1@m+MG4BQ&O^)
zWlzdTSd^WalC*Gk(!89snNw!vBqb)U$X>u5l)EB)LPSW!tf)ic^$#ROBwk5=JaF{D
zfg?u_G6+2^&^hOmB3R_*vnr(E^m+fp#|bRInw=etm;JwRUDS&&hG~-0*5`Tq6IoY>
uap?J8;`KY@q2qNXxWGsIRFdFiK4~$z)ya1@Nk6~Gywz!KG6Ms8hyVcaD~6>2
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/e1bd1067-e40a-4222-bab9-959f5af4dd4d-m0.avro
similarity index 95%
rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/a61cde47-0ced-4b74-b91a-9e32dd59b212-m0.avro
rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/e1bd1067-e40a-4222-bab9-959f5af4dd4d-m0.avro
index 462894acbdef4351207c59d84392e303f1c0d9eb..2cda88a5e7816bf1018a9d0e6f34322df26e4d2d 100644
GIT binary patch
delta 316
zcmX?PcF1glfKS9aNo=g{8{z_c^*%C456F=@;nls7g@025xTxyvGE|^|4_4G0|F*G;0=q@gL+8#PuI=;FO>$wYfxOhr=U(P&R
zlDb<%So|_m+O*1?9AU8yg?kd7?)uu<vRC517`OyLqiQf72IrxT!!@mat7kXaWe<^L9#5~<$h1-)~xAP+ot`cqX
zo?Ox_Y%J6*^jSpaa@VE^7n4IlmxH~9CO62XJQfIG&t!dhrQ!&1NdF`Po&;lJTJV4WtIVG9;DTyVC`UW7NYpiQ(o@!*2m~5eIo@$<=
zYhqz+pqrSQlB8>Bl9HHYmX?^5l47Q7pbJuMs8^6!R9Ko?!WVzw7!(-l8nR_B)0DHK
zOk7O!m|2#wvE5tB#_@rb>p&Af15gYo$pXY|K+FNeTtEyGW?*4pW8h%mVqjom;$Q%Y
za5xrcCb9@QB_>rWxaH@kuyHT}{lOui8Pk90>k1_YE`d;u$o041u!2=91eYf11}A1D
z7A58>IHnb4v4}K;rKYErBqjkp$i^X{4$=zJsqv@&1NdF{%kvR~USsEJ~T9{ZGn3-FcPCmx2#Av$sB|AqwM_y@84#SM+)`q5+
z%1=#UmY%7Xp2x@ZfHC@TL{84q_3`r}rp%fcA2nxH{G1t+qj(e_7kd|b8GS7d_7(~j
z@)r6!c@j_18D1ZqQrb%%n+YCiUw23YQl1ei1s|^=WsJ
iq{NOLd}k}FvYv09ynE_Q2`LG_GmjDz5|S7&+yDSb!cAHL
delta 268
zcmZotJg2x}GrO>bv5~2nfvLHvnVF%HxuK=8nc3uH>`IJAi(j&H)TLG3-R)$W3OT-nvnFA)96pqF3U8_XtrY+A&P$hhg#rp~za>9xt~
z%US1)q8A?Int5FK>E{mTq_kvSClA(JEK)n#1=%_@LOEudA99_z-pDDYynMgy@7nFP
z7E>lTzZP=x{CvD4+1@P7<3m!mLx!e7SozwQsP!
KAMRr
z1Y`u=-P-ISsOiVG&WeTAu|zCpm&J}9D=aK_mQL_XTEr2%+2(cs^Z3%P3}ue8j1=?s
z$A{Qhd2>E)|65~Iud%)9&a{*jOB^~_g4c-#s97i^@mNh1OG#5wUn_j6;KWa>-2S%R
zmdShXz4`a*+uEwSH`P75t4xpCgl%QWUEzB|m{ZJ>kxS7b`0eV5oik*lnxDzfxstHm
z+A^X}?-_q}`rN2F^Qv-PuW%V&WRW}`l@n2va-i^NXUmyO6T)2Y$Gg|X?%Q)tf8IIi
zk1x*UYfCL=U7B&wLXt7fH%?WiW#z<)OC$_KS1)?*YoGS}c>8_x_q`RhpMHO2M-L(Z
DuXvvn
delta 360
zcmZ3lv|ee$W_Dpib4w#b3sVboBSTAbOA8YVbJNYo*i|^{jg3sqjZI99O@XALiG_)w
znK8o?i4}Le_up_hoZUY&m9uyP#~hZa=MB9NJBS?kn60xk{@Khsr;qObym-fn-FK`-
zq_})nt!@_4Y2tena&hOilj|;dc0G9V;MIkLuI%=WY_2b6J>CEB$N8w#U9!j9O&;w1
zx8cc-C34g2@9MVg&8oN8+o9wydUR$j1Cl0=v?$ijrQP9=G4f+pIcR_WO(b&zHre8zuH{
z&w5jIFHh^zv@^Eayq@Rerd;NlGnHjp@E5zye5&j8-7cQ}o>Xctt6R!@Zhf72Jw^}#
E05%b$DF6Tf
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3825168261540020388-1-bdc6d711-4699-4d21-a7f8-907c7740c70c.avro
similarity index 92%
rename from extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-2001582482032951248-1-a61cde47-0ced-4b74-b91a-9e32dd59b212.avro
rename to extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-3825168261540020388-1-bdc6d711-4699-4d21-a7f8-907c7740c70c.avro
index 28d439215a04848e617058919ca10091fdd5fbd9..ca67d791ceb320cec91cbf101405a982357071ce 100644
GIT binary patch
delta 54
zcmV-60LlM=A^#t+vj-A0I5IUcHaIdiF*P(WFfuSRIFrQ(BLOzE=?4Y~5QOgc%H<>U
Mi-YW^^k4{~+29lu`~Uy|
delta 184
zcmew_-=Mf*GrN$Hfq|i^g^`Jck%6(1rKzEj$>gK#%8Z7aU$V1t)N8%=5HfzzF`Z}k
zy?O=yU3pA*7^4qIr{LF`kC;Ax^r*wWb2$Y}F1b`_3#GXoO?BQpbYa|1&Q3k!2|
zBNG#bv~}ATe!4O%=z^fsff7f6dPCj)|+?*I2=y;bO~=|yhI^=7j|
z;=)Y6T-~?)X-WitO8v|1q*b23jubAcVzbd=x)Rl_oPIFPEJb93q@-7=)8&sDfm|2g
z>gkom-=F>U&pu(!jeD|%IUS#{qiW@$a-mJ`W!L|{s&%Po;gr}iY7_jU8Oy`O!2
zdwbZPs@{IlnRA!P*cKRQCR}_o@han_IlfsM&S%~(lyE(L{`KeCr|(af=nt#e_qYBJ
HGrFGv(#KhPXNE(`0m>8NFZ$8GZ!clKwY-DO?U}|n^W@cz)ZfI$2
zX2!73OCV%b$@-VAJT=_Qn|*e$9ble%ey5M0A9@yqB*_PohCLGA62Da23qCJm=fvHPyX#lRnK^{xZcVds0rt
zoLTqfu5T`^)_Q%VzUaxcrCawsTEV1oh0&*zagLGT1R3qeCc>cs;>>ol{hna+j@{c{AuIdt6Dvy!gHQYTo5-
zoBn<;-|o9l+PiE*6xEU!a%eVvyw;$8)WI{kN3}BKN4?=`xqaJqFTX1J=w{i2XYbNK
IN~3!k09J*Ln*aa+
diff --git a/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro b/extensions/iceberg/s3/src/test/resources/warehouse/sales/sales_multi/metadata/snap-6040260770188877244-1-4b9446ba-c04a-440a-afad-f090aa27c81a.avro
new file mode 100644
index 0000000000000000000000000000000000000000..790df47dae8eeaab9e9d153b5d693497193327b3
GIT binary patch
literal 4301
zcmbVPTWB0r7_LDOf(f;>p^c?I4z>7@$joN5yIGpr)+h$!g>3uMu&ifi&Ss~~&a5+M
zvaW435a>(QRy0pu&=*CaTA_GpeJNVnO7kKVD(X|AAZio(AOs3}&Rotpv%9kwA9wc5
z_xv2adqY7NwbzH*rDw492ja19lq!G!475cjZO}&I1Tqx-V
zB9BWtGBiiZ&K++$QH8inP6R*>wsmu!{MIa$;LKPm#JXjIBeT<>rsFbrItM&8%}U_}6~LL2Keo`O?FYJo7b
zz;zR|sB`2`Fg*oa2WbFXpl0j1G2Vm#t6C;T-upooM6f6QKOJC|04^gCrZx-YNeyB+
z0+dM&8K^08B`r`m=w%m~Dk_+6rHE`EcO)gb;1uvrfm#`v5mHct4uG15=*_*N`dL!O
z1+I!t^3xUHj8!8|0>Z-~EvbZf(*~-PK;?x}a7GN6b}&oa;vlw}|mS1eYTfR3ZtdqaVuvduIvFDBAXphU8u%{!MNOb_}rbC950*AOo%Q6r&$pc>$58EzL
zs}g`_-IuR29ZE7hh5=ZdU;!F#qv%Ws9mvT%=;)oKkTc(j6q+qKHB^GGfu$^sDJm2h
zAq0brzg;MjiQ+;UW>xaFnDM_DN-yx<0^VDoWZ3|cYkRXv6BJJ@E(o_;{U5~MTfGx2
zi}}S7iA-CCNJFqxGK=hDIk!@QL^0@95LC-Noo(+UrwQyM9~TVm?^&-|l9e
zOt!PD(mm%J0Piw#WV;8;xY-BBEUY=V12LqLgynj$2b?vp3Prk>+c%7A$y$7n|j$XLyz$!47Z+UfwW0JrjiKX@xIy8DKg?
zjRwz42^tQQOQy#LF_Gk0!){>@1P{WLTlf<5Rdj-NE_VdJ{ow2otq!e0Cuh*gj_{0T
zTF`n%gtJEO%p6H4+7^a{vyx>Me4$3YM=YsslHE)fvG+OJjS1mod_P6~M$csN#v@`a
zI4-Rh@f?28Z43r^eGh3k>Ptl1kWms!DxJ(E#+39}CXq-_$eXSm1ZOY*d}rgaU#@TZ
z`ti
zF;iUcFK&70)Q!7yum5^<$N81vkJm0-{s@0Qb^E6;KR9vb$mGh!{-JGKubljD+wzuG
zP#QSY|LkX};q@Q(99y#USI!mQa30yWwsAB6#iy(F{5yxPpZ;+3tnu&mvu`d={dM&d
tc;NjXPrh{T-Ql+fmN!3l`|7>7W}ja>buBV#0lH?Qm|dVY99!#b==xE;i0u;p
zKt*>01^Z^eu#0$rh&p!(a(vVqGeWa)Q(!{@9^#NWJf1Ec5~~Z$B5=oNB5D)>f<3D1T0sO{Ke7}yee#fS#pOdEc#l3g?9?=T0vA*smnYs
z2nrdFQFctpk-n}#l207Upwr_4nTq;OdU6V|n*5{{?Af?i>62n0zh|$&~JEe&o6u$WA)7I1KR^{)S=q
zKP@mq@%8%KAJ>KaU&2hxx(mG+Ui
zm>GE<2F2i4A=nzobWXjG-U8UwcpF=RO{}wUbr&;oE)zpwy4W_F#%621-CQ$TYwc#U
zwWXg29uKp|AqgogvgPK5Onn=P=r0@R9gDz@I7rw0Pp^wkhV2|#e9t8YAij*A+wSQy
zE*gP!&4;iToDkMW2cU>(@degC@1KOlD6)u!U%`_Tt4aePDVl;p43@I`D@fGsU2X*g
zfzc8K(ge%_yND@}WE+AktGToT6{NX@nG4th$+{_6%Za9KC8G#ADfOH}nIl0p_K+n)t%a6ePEvEAo?T
zzJj$7!5gyJB0memi~J~dKr+bkX6ad&AWKgfzAj}z=nOX+GB17XQX!W@k6kd4aBO0?
zhzE)X5z4K6i6tt!B|29+g2aAE_MBE{)?k)1D6=CnqwNyf*oa8h+?|CZnMGS-NF*yP
ztMm&s?>#C>9UpcxU&Qg}=sYGwlF9v)_Zth7RU40-wUD^HUf?;Bpr;rNc>PUiIPOan
z+wc;1ZO?u9&C+H1*3;G1-+#W*{Pf3Hmj77V|K*nJe*1m*@vX1E{<3@f(RbZPCl7DG
z{KiwdP2X)kqvm$atZmnxnR^eus5N%>BJ=dgakTs3VLe)D9G|i?jlCzwzpd=$qlM
rJ@ZfP*U$gn4_|xr
Date: Wed, 4 Sep 2024 09:56:41 -0700
Subject: [PATCH 24/56] Manual and auto refreshing tests for Iceberg.
---
.../iceberg/util/IcebergToolsTest.java | 163 ++++++++-----
.../iceberg/util/IcebergCatalogAdapter.java | 8 -
.../IcebergRefreshingTestTable.java | 218 ++++++++++++++++++
3 files changed, 326 insertions(+), 63 deletions(-)
create mode 100644 extensions/iceberg/src/test/java/io/deephaven/iceberg/TestCatalog/IcebergRefreshingTestTable.java
diff --git a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java
index 335082ea355..66b1a08ef3b 100644
--- a/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java
+++ b/extensions/iceberg/s3/src/test/java/io/deephaven/iceberg/util/IcebergToolsTest.java
@@ -5,10 +5,14 @@
import gnu.trove.list.array.TLongArrayList;
import io.deephaven.base.verify.Assert;
+import io.deephaven.engine.context.ExecutionContext;
import io.deephaven.engine.table.ColumnDefinition;
import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.TableDefinition;
import io.deephaven.engine.table.impl.locations.TableDataException;
+import io.deephaven.engine.testutil.ControlledUpdateGraph;
+import io.deephaven.engine.testutil.GenerateTableUpdates;
+import io.deephaven.engine.testutil.TstUtils;
import io.deephaven.engine.testutil.junit4.EngineCleanup;
import io.deephaven.extensions.s3.S3Instructions;
import io.deephaven.iceberg.TestCatalog.IcebergTestCatalog;
@@ -201,12 +205,12 @@ public void testListNamespaces() {
final Collection namespaceNames =
namespaces.stream().map(Namespace::toString).collect(Collectors.toList());
- Assert.eq(namespaceNames.size(), "namespaceNames.size()", 2, "2 namespace in the catalog");
+ Assert.eq(namespaceNames.size(), "namespaceNames.size()", 2, "namespace in the catalog");
Assert.eqTrue(namespaceNames.contains("sales"), "namespaceNames.contains(sales)");
Assert.eqTrue(namespaceNames.contains("sample"), "namespaceNames.contains(sample)");
final Table table = adapter.namespaces();
- Assert.eq(table.size(), "table.size()", 2, "2 namespace in the catalog");
+ Assert.eq(table.size(), "table.size()", 2, "namespace in the catalog");
Assert.equals(table.getDefinition(), "table.getDefinition()", NAMESPACE_DEFINITION);
}
@@ -217,7 +221,7 @@ public void testListTables() {
final Namespace ns = Namespace.of("sales");
Collection