Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

feat: support refreshing Iceberg tables #5707

Open
wants to merge 59 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 3 commits
Commits
Show all changes
59 commits
Select commit Hold shift + click to select a range
470b09c
Initial commit of refreshing Iceberg.
lbooker42 Jul 2, 2024
a8d957a
Rebased to main.
lbooker42 Jul 2, 2024
264fdb1
Change IcebergInstructions refreshing indicator to enum instead of bo…
lbooker42 Jul 2, 2024
58d0a73
WIP, for review
lbooker42 Jul 3, 2024
e090474
Manual and auto-refreshing working, better documentation.
lbooker42 Jul 23, 2024
57021ad
Addressed more PR comments, some remaining.
lbooker42 Jul 23, 2024
fb882e8
WIP, some PR comments addressed.
lbooker42 Jul 26, 2024
5bbdeb2
WIP, even more PR comments addressed.
lbooker42 Jul 27, 2024
3da205c
Nearly all PR comments addressed.
lbooker42 Jul 27, 2024
91acf9b
merged with main
lbooker42 Jul 27, 2024
08dd329
Adjustment to IcebergInstructions update mode.
lbooker42 Jul 29, 2024
7af0d1d
Added python wrapper for Iceberg refreshing tables.
lbooker42 Jul 29, 2024
2d79c38
Changes to mocked tests for ColumnSourceManager and PartitionAwareSou…
lbooker42 Jul 29, 2024
3809f21
Added DHError handler and add'l documentation to python `snapshots()`…
lbooker42 Jul 30, 2024
5273a15
Fixed typo in JavaDoc
lbooker42 Jul 30, 2024
b9e2c6e
WIP
lbooker42 Jul 31, 2024
9937f79
Suggestion from review
lbooker42 Jul 31, 2024
cd08038
WIP, changes to revert some transaction token code.
lbooker42 Jul 31, 2024
f28325f
Correct logic across multiple transactions.
lbooker42 Aug 21, 2024
2d92b3f
Merged with main
lbooker42 Aug 21, 2024
cd31d82
Moved transaction accumulation to AbstractTableLocationProvider
lbooker42 Aug 23, 2024
d680c0c
Moved transaction accumulation to AbstractTableLocationProvider
lbooker42 Aug 26, 2024
6607fc3
PR comments addressed.
lbooker42 Aug 28, 2024
893336f
Updated to use IcebergTableAdapter and exposed in python. Addressed P…
lbooker42 Aug 30, 2024
68e4546
Incorporated external PR to update PartitioningColumnDataIndex for re…
lbooker42 Aug 30, 2024
273f5c1
Added additional snapshots with removes to IcebergToolsTest resources.
lbooker42 Sep 3, 2024
1e92a19
Merge branch 'main' into lab-iceberg-refreshing
lbooker42 Sep 3, 2024
f72c1b7
Manual and auto refreshing tests for Iceberg.
lbooker42 Sep 4, 2024
5c7ff12
Manual and auto refreshing tests for Iceberg, not passing.
lbooker42 Sep 4, 2024
92eec61
PR comments addressed.
lbooker42 Sep 4, 2024
95194b7
Implemented improved location reference counting in AbstractTableLoca…
lbooker42 Sep 5, 2024
09e2b6e
Fixing doc problem.
lbooker42 Sep 5, 2024
30910dd
For review only, does not compile :(
lbooker42 Sep 12, 2024
dd12240
Compiles now, still many problems
lbooker42 Sep 13, 2024
944dac6
Working through problems.
lbooker42 Sep 13, 2024
912b9f2
Cleanup and minor changes
lbooker42 Sep 13, 2024
e2b6fd0
Refactored ATLP
lbooker42 Sep 18, 2024
72b03be
Merged with main.
lbooker42 Sep 18, 2024
01e50fe
Updated but RCSM still not referenced properly.
lbooker42 Sep 18, 2024
81e88d8
Refreshing tests still need work.
lbooker42 Sep 20, 2024
b37a04e
Better tests and improved liveness management for the TableLocation k…
lbooker42 Sep 25, 2024
dead9c4
Added TLP state (add, append, static, refreshing)
lbooker42 Sep 25, 2024
e5d10e7
Added TLP state (add, append, static, refreshing)
lbooker42 Sep 27, 2024
b30e240
Addressed PR comments, some TODO remaining to address.
lbooker42 Sep 30, 2024
fa9d154
Improved table location management in SourcePartitionedTable
lbooker42 Oct 1, 2024
d807a94
Merge with main
lbooker42 Oct 2, 2024
db2b031
Post-merge cleanup
lbooker42 Oct 2, 2024
746b343
Post-merge cleanup and test updating.
lbooker42 Oct 2, 2024
d69ddcd
Addressed PR comments and test failures.
lbooker42 Oct 4, 2024
06a3bd5
More test failure fixes.
lbooker42 Oct 4, 2024
91ba92c
Liveness management re-ordering in SourcePartitionedTable
lbooker42 Oct 7, 2024
73e8824
Addressing open PR comments.
lbooker42 Oct 9, 2024
be689ef
Many PR comments addressed.
lbooker42 Oct 15, 2024
b15c78a
IcebergTableAdapter synchronization changes and cleanup.
lbooker42 Oct 16, 2024
de9f6ae
PR comments addressed.
lbooker42 Oct 18, 2024
d5c13e6
Fix for iceberg.py to use table adapter
lbooker42 Oct 18, 2024
851260c
Merged with main.
lbooker42 Oct 18, 2024
d1557c5
Addressed PR comments.
lbooker42 Oct 18, 2024
975a013
Close to merging.
lbooker42 Oct 19, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -170,7 +170,7 @@ public synchronized void findKeys(@NotNull final Consumer<IcebergTableLocationKe
/**
* Update the snapshot to the latest snapshot from the catalog if
*/
protected boolean maybeUpdateSnapshot() {
protected synchronized boolean maybeUpdateSnapshot() {
final Snapshot latestSnapshot = tableAdapter.currentSnapshot();
if (latestSnapshot == null) {
return false;
Expand All @@ -183,11 +183,8 @@ protected boolean maybeUpdateSnapshot() {
}

/**
* Update the snapshot to the user specified snapshot. This will fail with an {@link IllegalArgumentException} if
* the input snapshot is not newer (higher in sequence number) than the current snapshot or if the snapshot cannot
* be found.
*
* @param snapshotId the id of the snapshot to load
* Update the snapshot to the user specified snapshot. See
* {@link io.deephaven.iceberg.util.IcebergTable#update(long)} for more details.
*/
protected void updateSnapshot(long snapshotId) {
final List<Snapshot> snapshots = tableAdapter.listSnapshots();
Expand All @@ -205,23 +202,18 @@ protected void updateSnapshot(long snapshotId) {
}

/**
* Update the snapshot to the user specified snapshot. This will fail with an {@link IllegalArgumentException} if
* the input snapshot is not newer (higher in sequence number) than the current snapshot.
*
* @param snapshot the snapshot to load
* Update the snapshot to the user specified snapshot. See
* {@link io.deephaven.iceberg.util.IcebergTable#update(Snapshot)} for more details.
*/
protected void updateSnapshot(Snapshot snapshot) {
protected void updateSnapshot(@NotNull final Snapshot updateSnapshot) {
// Validate that we are not trying to update to an older snapshot.
if (snapshot == null) {
throw new IllegalArgumentException("Input snapshot cannot be null");
}
if (this.snapshot != null && snapshot.sequenceNumber() <= this.snapshot.sequenceNumber()) {
if (snapshot != null && updateSnapshot.sequenceNumber() <= snapshot.sequenceNumber()) {
throw new IllegalArgumentException(
"Update snapshot sequence number (" + snapshot.sequenceNumber()
"Update snapshot sequence number (" + updateSnapshot.sequenceNumber()
+ ") must be higher than the current snapshot sequence number ("
+ this.snapshot.sequenceNumber() + ") for table " + tableAdapter);
+ snapshot.sequenceNumber() + ") for table " + tableAdapter);
}

this.snapshot = snapshot;
snapshot = updateSnapshot;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,9 +51,8 @@ public String getImplementationName() {
}

@Override
public synchronized void refresh() {
// There is no refresh service for this provider, but this is called as part of the initialization process.
refreshLocations();
public void refresh() {
ensureInitialized();
}

@Override
Expand All @@ -72,11 +71,19 @@ public synchronized void update(final long snapshotId) {

@Override
public synchronized void update(final Snapshot snapshot) {
if (snapshot == null) {
throw new IllegalArgumentException("Input snapshot cannot be null");
}
// Update the snapshot.
locationKeyFinder.updateSnapshot(snapshot);
refreshLocations();
}

@Override
protected synchronized void doInitialization() {
refreshLocations();
}

// ------------------------------------------------------------------------------------------------------------------
// SubscriptionAggregator implementation
// ------------------------------------------------------------------------------------------------------------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,13 +7,16 @@
import io.deephaven.engine.table.*;
import io.deephaven.engine.table.impl.QueryTable;
import io.deephaven.engine.table.impl.sources.InMemoryColumnSource;
import io.deephaven.iceberg.internal.DataInstructionsProviderLoader;
import io.deephaven.iceberg.internal.DataInstructionsProviderPlugin;
import io.deephaven.util.annotations.VisibleForTesting;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.Namespace;
import org.apache.iceberg.catalog.SupportsNamespaces;
import org.apache.iceberg.catalog.TableIdentifier;
import org.jetbrains.annotations.NotNull;
import io.deephaven.iceberg.internal.DataInstructionsProviderLoader;
import org.apache.iceberg.rest.RESTCatalog;
import org.apache.iceberg.rest.ResourcePaths;

import java.util.*;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@
//
package io.deephaven.iceberg.util;

import io.deephaven.annotations.CopyableStyle;
import io.deephaven.engine.table.TableDefinition;
import org.immutables.value.Value;
import org.immutables.value.Value.Immutable;
Expand All @@ -15,11 +16,7 @@
* this class may change in the future. As such, callers may wish to explicitly set the values.
*/
@Immutable
@Value.Style(visibility = Value.Style.ImplementationVisibility.PACKAGE,
strictBuilder = true,
weakInterning = true,
jdkOnly = true,
includeHashCode = "getClass().hashCode()")
@CopyableStyle
public abstract class IcebergInstructions {
/**
* The default {@link IcebergInstructions} to use when reading Iceberg data files. Providing this will use system
Expand Down Expand Up @@ -50,7 +47,7 @@ public static Builder builder() {
public abstract Map<String, String> columnRenames();

/**
* Return a copy of this instructions object with the column renames updated to the provided entries.
* Return a copy of this instructions object with the column renames replaced by {@code entries}.
*/
public abstract IcebergInstructions withColumnRenames(Map<String, ? extends String> entries);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,13 +77,22 @@ public synchronized Snapshot currentSnapshot() {
}

/**
* List all {@link Snapshot snapshots} of the Iceberg table.
* Get the current list of all {@link Snapshot snapshots} of the Iceberg table.
*
* @return A list of all snapshots of the given table.
*/
public synchronized List<Snapshot> listSnapshots() {
// Refresh the table to update the snapshot list.
refresh();
return getSnapshots();
}

/**
* Get a list of all {@link Snapshot snapshots} of the Iceberg table (without refreshing).
*
* @return A list of all snapshots of the given table.
*/
private List<Snapshot> getSnapshots() {
final List<Snapshot> snapshots = new ArrayList<>();
table.snapshots().forEach(snapshots::add);
return snapshots;
Expand Down Expand Up @@ -124,6 +133,7 @@ public synchronized List<Snapshot> listSnapshots() {
* @return A Table containing a list of all tables in the given namespace.
*/
public Table snapshots() {
// Retrieve the current list of snapshots
final List<Snapshot> snapshots = listSnapshots();
final long size = snapshots.size();

Expand Down Expand Up @@ -173,9 +183,17 @@ public Table snapshots() {
* @return An Optional<Snapshot> containing the requested snapshot if it exists.
*/
private Optional<Snapshot> snapshot(final long snapshotId) {
return listSnapshots().stream()
Optional<Snapshot> found = getSnapshots().stream()
.filter(snapshot -> snapshot.snapshotId() == snapshotId)
.findFirst();
if (found.isEmpty()) {
// Refresh the table to update the snapshot list, then try again.
refresh();
found = getSnapshots().stream()
.filter(snapshot -> snapshot.snapshotId() == snapshotId)
.findFirst();
}
return found;
}

/**
Expand All @@ -186,17 +204,27 @@ public synchronized Schema currentSchema() {
return table.schema();
}

/**
* Retrieve the current {@link Schema schema} of an Iceberg table.
*/
public synchronized Map<Integer, Schema> schemas() {
refresh();
return table.schemas();
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
}

/**
* Retrieve a specific {@link Schema schema} of an Iceberg table.
*
* @param schemaId The identifier of the schema to load.
*/
public synchronized Schema schema(final int schemaId) {
// TODO: discuss refresh() strategy for this and other functions:
// 1) ALWAYS refresh() before searching for a match (safe, might be slow)
// 2) NEVER refresh() before searching (user should call refresh() manually)
// 3) HYBRID, refresh() if search fails, then re-search
return table.schemas().get(schemaId);
public synchronized Optional<Schema> schema(final int schemaId) {
Schema found = table.schemas().get(schemaId);
if (found == null) {
// Refresh the table to update the snapshot list, then try again.
refresh();
found = table.schemas().get(schemaId);
}
return Optional.ofNullable(found);
}

/**
Expand Down Expand Up @@ -252,21 +280,19 @@ public TableDefinition definition(
@Nullable final Snapshot tableSnapshot,
@Nullable final IcebergInstructions instructions) {

final Snapshot snapshot;
final Schema schema;
final org.apache.iceberg.PartitionSpec partitionSpec;

if (tableSnapshot == null) {
synchronized (this) {
// Refresh only once and record the current snapshot, using its schema and spec.
// Refresh only once and record the current schema and partition spec.
refresh();
snapshot = table.currentSnapshot();
schema = snapshot != null ? schema(snapshot.schemaId()) : table.schema();
schema = table.schema();
partitionSpec = table.spec();
}
} else {
snapshot = tableSnapshot;
schema = schema(tableSnapshot.schemaId());
// Use the schema from the snapshot
schema = schema(tableSnapshot.schemaId()).get();
partitionSpec = table.spec();
}

Expand Down Expand Up @@ -388,15 +414,17 @@ public IcebergTable table(

if (tableSnapshot == null) {
synchronized (this) {
// Refresh only once and record the current snapshot, using its schema and spec.
// Refresh only once and record the current snapshot, schema (which may be newer than the
// snapshot schema), and partition spec.
refresh();
snapshot = table.currentSnapshot();
schema = snapshot != null ? schema(snapshot.schemaId()) : table.schema();
schema = table.schema();
partitionSpec = table.spec();
}
} else {
snapshot = tableSnapshot;
schema = schema(tableSnapshot.schemaId());
// Use the schema from the snapshot
schema = schema(tableSnapshot.schemaId()).get();
partitionSpec = table.spec();
}

Expand Down Expand Up @@ -427,8 +455,7 @@ public IcebergTable table(
finalInstructions, dataInstructionsProviderLoader);
}

if (instructions == null
|| instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) {
if (finalInstructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.STATIC) {
final IcebergTableLocationProviderBase<TableKey, IcebergTableLocationKey> locationProvider =
new IcebergStaticTableLocationProvider<>(
StandaloneTableKey.getInstance(),
Expand All @@ -447,7 +474,7 @@ public IcebergTable table(
final UpdateSourceRegistrar updateSourceRegistrar = ExecutionContext.getContext().getUpdateGraph();
final IcebergTableLocationProviderBase<TableKey, IcebergTableLocationKey> locationProvider;

if (instructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) {
if (finalInstructions.updateMode().updateType() == IcebergUpdateMode.IcebergUpdateType.MANUAL_REFRESHING) {
locationProvider = new IcebergManualRefreshTableLocationProvider<>(
StandaloneTableKey.getInstance(),
keyFinder,
Expand All @@ -460,7 +487,7 @@ public IcebergTable table(
keyFinder,
new IcebergTableLocationFactory(),
TableDataRefreshService.getSharedRefreshService(),
instructions.updateMode().autoRefreshMs(),
finalInstructions.updateMode().autoRefreshMs(),
this,
tableIdentifier);
}
Expand Down
38 changes: 3 additions & 35 deletions py/server/deephaven/experimental/iceberg.py
rcaudy marked this conversation as resolved.
Show resolved Hide resolved
Original file line number Diff line number Diff line change
Expand Up @@ -229,14 +229,9 @@ def table(self, instructions: Optional[IcebergInstructions] = None, snapshot_id:
Table: the table read from the catalog.
"""

if instructions is not None:
instructions_object = instructions.j_object
else:
instructions_object = _JIcebergInstructions.DEFAULT

if snapshot_id is not None:
return IcebergTable(self.j_object.table(snapshot_id, instructions_object))
return IcebergTable(self.j_object.readTable(instructions_object))
if snapshot_id:
return IcebergTable(self.j_object.table(snapshot_id, instructions))
return IcebergTable(self.j_object.table(instructions))

@property
def j_object(self) -> jpy.JType:
Expand Down Expand Up @@ -317,33 +312,6 @@ def snapshots(self, table_identifier: str) -> Table:

return self.j_object.snapshots(table_identifier)

def read_table(self, table_identifier: str, instructions: Optional[IcebergInstructions] = None, snapshot_id: Optional[int] = None) -> IcebergTable:
"""
NOTE: deprecated, use `load_table(table_identifier).table()` instead.

Reads the table from the catalog using the provided instructions. Optionally, a snapshot id can be provided to
read a specific snapshot of the table.

Args:
table_identifier (str): the table to read.
instructions (Optional[IcebergInstructions]): the instructions for reading the table. These instructions
can include column renames, table definition, and specific data instructions for reading the data files
from the provider. If omitted, the table will be read with default instructions.
snapshot_id (Optional[int]): the snapshot id to read; if omitted the most recent snapshot will be selected.

Returns:
Table: the table read from the catalog.
"""

if instructions is not None:
instructions_object = instructions.j_object
else:
instructions_object = _JIcebergInstructions.DEFAULT

if snapshot_id is not None:
return IcebergTable(self.j_object.readTable(table_identifier, snapshot_id, instructions_object))
return IcebergTable(self.j_object.readTable(table_identifier, instructions_object))

@property
def j_object(self) -> jpy.JType:
return self.j_catalog_adapter
Expand Down