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

Merged
merged 63 commits into from
Oct 22, 2024
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
Show all changes
63 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
ad5a225
Changes in the refresh and initialization for Static/Manual/Refreshin…
lbooker42 Oct 21, 2024
e5c8f49
Python fix for definition(), cleanup
lbooker42 Oct 22, 2024
1f44a44
More cleanup.
lbooker42 Oct 22, 2024
300f836
Final pre-merge change.
lbooker42 Oct 22, 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 @@ -3,13 +3,12 @@
//
package io.deephaven.engine.table.impl;

import io.deephaven.engine.liveness.LivenessReferent;
import io.deephaven.engine.liveness.LivenessNode;
import io.deephaven.engine.rowset.RowSet;
import io.deephaven.engine.rowset.TrackingWritableRowSet;
import io.deephaven.engine.rowset.WritableRowSet;
import io.deephaven.engine.table.*;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableLocation;
import io.deephaven.engine.table.impl.locations.TrackedTableLocationKey;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;

Expand All @@ -19,7 +18,7 @@
/**
* Manager for ColumnSources in a Table.
*/
public interface ColumnSourceManager extends LivenessReferent {
public interface ColumnSourceManager extends LivenessNode {
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved

/**
* Get a map of name to {@link ColumnSource} for the column sources maintained by this manager.
Expand Down Expand Up @@ -116,5 +115,5 @@ public interface ColumnSourceManager extends LivenessReferent {
* @return true if the location key was actually removed
* @param tableLocationKey the location key being removed
*/
boolean removeLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey);
boolean removeLocationKey(@NotNull TrackedTableLocationKey tableLocationKey);
}
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@
import io.deephaven.api.filter.Filter;
import io.deephaven.base.verify.Assert;
import io.deephaven.engine.table.*;
import io.deephaven.engine.table.impl.locations.TrackedTableLocationKey;
import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer;
import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder;
Expand Down Expand Up @@ -204,7 +205,7 @@ protected final Table redefine(TableDefinition newDefinitionExternal, TableDefin
reference, null, viewColumns, null);
}

private static final String LOCATION_KEY_COLUMN_NAME = "__PartitionAwareSourceTable_TableLocationKey__";
private static final String TRACKED_KEY_COLUMN_NAME = "__PartitionAwareSourceTable_TrackedTableLocationKey__";
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved

private static <T> ColumnSource<? super T> makePartitionSource(@NotNull final ColumnDefinition<T> columnDefinition,
@NotNull final Collection<ImmutableTableLocationKey> locationKeys) {
Expand All @@ -221,30 +222,39 @@ private static <T> ColumnSource<? super T> makePartitionSource(@NotNull final Co
}

@Override
protected final Collection<ImmutableTableLocationKey> filterLocationKeys(
@NotNull final Collection<ImmutableTableLocationKey> foundLocationKeys) {
protected final Collection<TrackedTableLocationKey> filterLocationKeys(
@NotNull final Collection<TrackedTableLocationKey> foundLocationKeys) {
if (partitioningColumnFilters.length == 0) {
return foundLocationKeys;
}

final Collection<ImmutableTableLocationKey> immutableTableLocationKeys = foundLocationKeys.stream()
.map(TrackedTableLocationKey::getKey)
.collect(Collectors.toList());

// TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Check and see if we can just close this ticket, and maybe delete the todo.

final List<String> partitionTableColumnNames = Stream.concat(
partitioningColumnDefinitions.keySet().stream(),
Stream.of(LOCATION_KEY_COLUMN_NAME)).collect(Collectors.toList());
Stream.of(TRACKED_KEY_COLUMN_NAME)).collect(Collectors.toList());
final List<ColumnSource<?>> partitionTableColumnSources =
new ArrayList<>(partitioningColumnDefinitions.size() + 1);
for (final ColumnDefinition<?> columnDefinition : partitioningColumnDefinitions.values()) {
partitionTableColumnSources.add(makePartitionSource(columnDefinition, foundLocationKeys));
partitionTableColumnSources.add(makePartitionSource(columnDefinition, immutableTableLocationKeys));
}
// Add the tracked keys to the table
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource(foundLocationKeys,
ImmutableTableLocationKey.class, null));
TrackedTableLocationKey.class, null));

final Table filteredColumnPartitionTable = TableTools
.newTable(foundLocationKeys.size(), partitionTableColumnNames, partitionTableColumnSources)
.where(Filter.and(partitioningColumnFilters));
if (filteredColumnPartitionTable.size() == foundLocationKeys.size()) {
return foundLocationKeys;
}
final Iterable<ImmutableTableLocationKey> iterable =
() -> filteredColumnPartitionTable.columnIterator(LOCATION_KEY_COLUMN_NAME);

// Return the filtered keys
final Iterable<TrackedTableLocationKey> iterable =
() -> filteredColumnPartitionTable.columnIterator(TRACKED_KEY_COLUMN_NAME);
return StreamSupport.stream(iterable.spliterator(), false).collect(Collectors.toList());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,9 +163,12 @@ protected void instrumentedRefresh() {
processNewLocationsUpdateRoot = null;
removedLocationsComitter = null;
tableLocationProvider.refresh();
try (final RowSet added = sortAndAddLocations(tableLocationProvider.getTableLocationKeys().stream()
.filter(locationKeyMatcher)
.map(tableLocationProvider::getTableLocation))) {

final Collection<TableLocation> locations = new ArrayList<>();
tableLocationProvider.getTableLocationKeys(tlk -> {
locations.add(tableLocationProvider.getTableLocation(tlk.getKey()));
}, locationKeyMatcher);
try (final RowSet added = sortAndAddLocations(locations.stream())) {
resultRows.insert(added);
}
}
Expand Down Expand Up @@ -213,9 +216,14 @@ private Table makeConstituentTable(@NotNull final TableLocation tableLocation) {
}

private void processPendingLocations(final boolean notifyListeners) {
final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = subscriptionBuffer.processPending();
final RowSet removed = processRemovals(locationUpdate);
final RowSet added = processAdditions(locationUpdate);
final RowSet removed;
final RowSet added;

try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate =
subscriptionBuffer.processPending()) {
removed = processRemovals(locationUpdate);
added = processAdditions(locationUpdate);
}

resultRows.update(added, removed);
if (notifyListeners) {
Expand All @@ -242,6 +250,7 @@ private RowSet processAdditions(final TableLocationSubscriptionBuffer.LocationUp
*/
// TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table
locationUpdate.getPendingAddedLocationKeys().stream()
.map(TrackedTableLocationKey::getKey)
.filter(locationKeyMatcher)
.map(tableLocationProvider::getTableLocation)
.map(PendingLocationState::new)
Expand All @@ -263,6 +272,7 @@ private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpd
final Set<ImmutableTableLocationKey> relevantRemovedLocations =
locationUpdate.getPendingRemovedLocationKeys()
.stream()
.map(TrackedTableLocationKey::getKey)
.filter(locationKeyMatcher)
.collect(Collectors.toSet());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,9 +10,7 @@
import io.deephaven.engine.table.TableDefinition;
import io.deephaven.engine.table.TableUpdate;
import io.deephaven.engine.table.TableUpdateListener;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableDataException;
import io.deephaven.engine.table.impl.locations.TableLocationProvider;
import io.deephaven.engine.table.impl.locations.*;
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder;
import io.deephaven.engine.table.impl.locations.impl.TableLocationSubscriptionBuffer;
Expand All @@ -22,6 +20,7 @@
import org.apache.commons.lang3.mutable.MutableObject;
import org.jetbrains.annotations.NotNull;

import java.util.ArrayList;
import java.util.Collection;

/**
Expand Down Expand Up @@ -140,39 +139,42 @@ private void initializeAvailableLocations() {
if (isRefreshing()) {
final TableLocationSubscriptionBuffer locationBuffer =
new TableLocationSubscriptionBuffer(locationProvider);
final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate =
locationBuffer.processPending();

maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys());
maybeAddLocations(locationUpdate.getPendingAddedLocationKeys());
manage(locationBuffer);
try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate =
locationBuffer.processPending()) {
maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys());
maybeAddLocations(locationUpdate.getPendingAddedLocationKeys());
}
updateSourceRegistrar.addSource(locationChangePoller = new LocationChangePoller(locationBuffer));
} else {
locationProvider.refresh();
maybeAddLocations(locationProvider.getTableLocationKeys());
final Collection<TrackedTableLocationKey> tableLocationKeys = new ArrayList<>();
locationProvider.getTableLocationKeys(tableLocationKeys::add);
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
maybeAddLocations(tableLocationKeys);
}
});
locationsInitialized = true;
}
}

private void maybeAddLocations(@NotNull final Collection<ImmutableTableLocationKey> locationKeys) {
private void maybeAddLocations(@NotNull final Collection<TrackedTableLocationKey> locationKeys) {
if (locationKeys.isEmpty()) {
return;
}
filterLocationKeys(locationKeys)
.parallelStream()
.forEach(lk -> columnSourceManager.addLocation(locationProvider.getTableLocation(lk)));
.forEach(lk -> columnSourceManager.addLocation(locationProvider.getTableLocation(lk.getKey())));
}

private ImmutableTableLocationKey[] maybeRemoveLocations(
@NotNull final Collection<ImmutableTableLocationKey> removedKeys) {
private TrackedTableLocationKey[] maybeRemoveLocations(
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
@NotNull final Collection<TrackedTableLocationKey> removedKeys) {
if (removedKeys.isEmpty()) {
return ImmutableTableLocationKey.ZERO_LENGTH_IMMUTABLE_TABLE_LOCATION_KEY_ARRAY;
return TrackedTableLocationKey.ZERO_LENGTH_TRACKED_TABLE_LOCATION_KEY_ARRAY;
}

return filterLocationKeys(removedKeys).stream()
.filter(columnSourceManager::removeLocationKey)
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
.toArray(ImmutableTableLocationKey[]::new);
.toArray(TrackedTableLocationKey[]::new);
}

private void initializeLocationSizes() {
Expand Down Expand Up @@ -212,10 +214,11 @@ private LocationChangePoller(@NotNull final TableLocationSubscriptionBuffer loca

@Override
protected void instrumentedRefresh() {
final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending();
final ImmutableTableLocationKey[] removedKeys =
maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys());
maybeAddLocations(locationUpdate.getPendingAddedLocationKeys());
try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate =
locationBuffer.processPending()) {
maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys());
maybeAddLocations(locationUpdate.getPendingAddedLocationKeys());
}

// This class previously had functionality to notify "location listeners", but it was never used.
// Resurrect from git history if needed.
Expand Down Expand Up @@ -252,8 +255,8 @@ protected void onRefreshError(@NotNull final Exception error) {
* {@link TableLocationProvider}, but not yet incorporated into the table
* @return A sub-collection of the input
*/
protected Collection<ImmutableTableLocationKey> filterLocationKeys(
@NotNull final Collection<ImmutableTableLocationKey> foundLocationKeys) {
protected Collection<TrackedTableLocationKey> filterLocationKeys(
@NotNull final Collection<TrackedTableLocationKey> foundLocationKeys) {
return foundLocationKeys;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,9 +12,6 @@
*/
@Immutable
public interface ImmutableTableLocationKey extends TableLocationKey {

ImmutableTableLocationKey[] ZERO_LENGTH_IMMUTABLE_TABLE_LOCATION_KEY_ARRAY = new ImmutableTableLocationKey[0];

@FinalDefault
default ImmutableTableLocationKey makeImmutable() {
return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
import io.deephaven.api.SortColumn;
import io.deephaven.base.log.LogOutput;
import io.deephaven.base.log.LogOutputAppendable;
import io.deephaven.engine.liveness.LivenessNode;
import io.deephaven.engine.table.BasicDataIndex;
import io.deephaven.engine.table.Table;
import io.deephaven.io.log.impl.LogOutputStringImpl;
Expand All @@ -21,7 +22,7 @@
* location allows access to columns, size, and possibly other metadata for a single partition that may be included in a
* source table.
*/
public interface TableLocation extends NamedImplementation, LogOutputAppendable, TableLocationState {
public interface TableLocation extends NamedImplementation, LogOutputAppendable, TableLocationState, LivenessNode {
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved

/**
* Listener interface for anything that wants to know about changes to a location.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,8 @@
import org.jetbrains.annotations.Nullable;

import java.util.Collection;
import java.util.function.Consumer;
import java.util.function.Predicate;

/**
* Discovery utility for {@link TableLocation}s for a given table.
Expand Down Expand Up @@ -41,23 +43,24 @@ interface Listener extends BasicTableDataListener {
void endTransaction(@NotNull Object token);

/**
* Notify the listener of a {@link TableLocationKey} encountered while initiating or maintaining the location
* subscription. This should occur at most once per location, but the order of delivery is <i>not</i>
* Notify the listener of a {@link TrackedTableLocationKey} encountered while initiating or maintaining the
* location subscription. This should occur at most once per location, but the order of delivery is <i>not</i>
* guaranteed. This addition is not part of any transaction, and is equivalent to
* {@code handleTableLocationKeyAdded(tableLocationKey, null);} by default.
*
* @param tableLocationKey The new table location key.
*/
void handleTableLocationKeyAdded(@NotNull ImmutableTableLocationKey tableLocationKey);
void handleTableLocationKeyAdded(@NotNull TrackedTableLocationKey tableLocationKey);

/**
* Notify the listener of a {@link TableLocationKey} that has been removed. This removal is not part of any
* transaction, and is equivalent to {@code handleTableLocationKeyRemoved(tableLocationKey, null);} by default.
* Notify the listener of a {@link TrackedTableLocationKey} that has been removed. This removal is not part of
* any transaction, and is equivalent to {@code handleTableLocationKeyRemoved(tableLocationKey, null);} by
* default.
*
* @param tableLocationKey The table location key that was removed.
*/
@SuppressWarnings("unused")
void handleTableLocationKeyRemoved(@NotNull ImmutableTableLocationKey tableLocationKey);
void handleTableLocationKeyRemoved(@NotNull TrackedTableLocationKey tableLocationKey);

/**
* <p>
Expand All @@ -70,8 +73,8 @@ interface Listener extends BasicTableDataListener {
* @param removedKeys Collection of table location keys that were removed.
*/
default void handleTableLocationKeysUpdate(
@NotNull Collection<ImmutableTableLocationKey> addedKeys,
@NotNull Collection<ImmutableTableLocationKey> removedKeys) {
@NotNull Collection<TrackedTableLocationKey> addedKeys,
@NotNull Collection<TrackedTableLocationKey> removedKeys) {
removedKeys.forEach(this::handleTableLocationKeyRemoved);
addedKeys.forEach(this::handleTableLocationKeyAdded);
}
Expand Down Expand Up @@ -128,11 +131,17 @@ default void handleTableLocationKeysUpdate(
* Get this provider's currently known location keys. The locations specified by the keys returned may have null
* size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is
* guaranteed to succeed for all results.
*
* @return A collection of keys for locations available from this provider.
*/
@NotNull
Collection<ImmutableTableLocationKey> getTableLocationKeys();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should consider preserving this method, @Deprecated. Mostly a question of how inconvenient just deleting it would be for @abaranec .

default void getTableLocationKeys(Consumer<TrackedTableLocationKey> consumer) {
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
getTableLocationKeys(consumer, key -> true);
}

/**
* Get this provider's currently known location keys. The locations specified by the keys returned may have null
* size - that is, they may not "exist" for application purposes. {@link #getTableLocation(TableLocationKey)} is
* guaranteed to succeed for all results.
*/
void getTableLocationKeys(Consumer<TrackedTableLocationKey> consumer, Predicate<ImmutableTableLocationKey> filter);

/**
* Check if this provider knows the supplied location key.
Expand Down
Loading