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 40 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
40 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
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 @@ -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;

Expand Down Expand Up @@ -219,6 +220,36 @@ public final <NOTIFICATION_TYPE> boolean deliverNotification(
return initialSize > 0 && size == 0;
}

/**
* Dispatch a notification to all subscribers. Clean up any GC'd subscriptions.
*
* @param procedure The notification procedure to invoke
* @param firstNotification The first item to deliver
* @param secondNotification The second item to deliver (must be of the same type as {@code firstNotification})
* @param activeOnly Whether to restrict this notification to active subscriptions only
* @return Whether this operation caused the set to become <b>empty</b>
*/
public final <NOTIFICATION_TYPE> boolean deliverNotification(
@NotNull final TriConsumer<LISTENER_TYPE, NOTIFICATION_TYPE, NOTIFICATION_TYPE> procedure,
@Nullable final NOTIFICATION_TYPE firstNotification,
@Nullable final NOTIFICATION_TYPE secondNotification,
final boolean activeOnly) {
final int initialSize = size;
for (int si = 0; si < size;) {
final Entry currentEntry = subscriptions[si];
final LISTENER_TYPE currentListener = currentEntry.getListener();
if (currentListener == null) {
removeAt(si);
continue; // si is not incremented in this case - we'll reconsider the same slot if necessary.
}
if (!activeOnly || currentEntry.isActive()) {
procedure.accept(currentListener, firstNotification, secondNotification);
}
++si;
}
return initialSize > 0 && size == 0;
}

private void removeAt(final int subscriptionIndex) {
final int lastSubscriptionIndex = --size;
subscriptions[subscriptionIndex] = subscriptions[lastSubscriptionIndex];
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,11 +6,7 @@
import io.deephaven.engine.liveness.LivenessReferent;
import io.deephaven.engine.rowset.RowSet;
import io.deephaven.engine.rowset.TrackingWritableRowSet;
import io.deephaven.engine.rowset.WritableRowSet;
import io.deephaven.engine.table.ColumnSource;
import io.deephaven.engine.table.DataIndex;
import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.TableListener;
import io.deephaven.engine.table.*;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableLocation;
import org.jetbrains.annotations.NotNull;
Expand Down Expand Up @@ -53,7 +49,7 @@ public interface ColumnSourceManager extends LivenessReferent {
*
* @return The set of added row keys, to be owned by the caller
*/
WritableRowSet refresh();
TableUpdate refresh();

/**
* Advise this ColumnSourceManager that an error has occurred, and that it will no longer be {@link #refresh()
Expand Down Expand Up @@ -116,8 +112,7 @@ public interface ColumnSourceManager extends LivenessReferent {
/**
* Remove a table location key from the sources.
*
* @return true if the location key was actually removed
* @param tableLocationKey the location key being removed
*/
boolean removeLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey);
void removeLocationKey(@NotNull ImmutableTableLocationKey tableLocationKey);
}
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
import io.deephaven.api.Selectable;
import io.deephaven.api.filter.Filter;
import io.deephaven.base.verify.Assert;
import io.deephaven.engine.liveness.LiveSupplier;
import io.deephaven.engine.table.*;
import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer;
import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
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__";

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

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

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

// TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table
final List<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));
}
partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource(foundLocationKeys,
ImmutableTableLocationKey.class, null));
// Add the tracked keys to the table
// noinspection unchecked,rawtypes
partitionTableColumnSources.add(ArrayBackedColumnSource.getMemoryColumnSource(
(Collection<LiveSupplier>) (Collection) foundLocationKeys,
LiveSupplier.class,
null));

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

// Return the filtered keys
final Iterable<LiveSupplier<ImmutableTableLocationKey>> 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 @@ -4,6 +4,7 @@
package io.deephaven.engine.table.impl;

import io.deephaven.base.verify.Assert;
import io.deephaven.engine.liveness.LiveSupplier;
import io.deephaven.engine.primitive.iterator.CloseableIterator;
import io.deephaven.engine.rowset.*;
import io.deephaven.engine.table.*;
Expand Down Expand Up @@ -163,9 +164,12 @@ protected void instrumentedRefresh() {
processNewLocationsUpdateRoot = null;
removedLocationsComitter = null;
tableLocationProvider.refresh();
try (final RowSet added = sortAndAddLocations(tableLocationProvider.getTableLocationKeys().stream()
.filter(locationKeyMatcher)
.map(tableLocationProvider::getTableLocation))) {

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

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

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

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

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,26 +5,23 @@

import io.deephaven.base.verify.Assert;
import io.deephaven.base.verify.Require;
import io.deephaven.engine.liveness.LiveSupplier;
import io.deephaven.engine.liveness.LivenessScopeStack;
import io.deephaven.engine.rowset.TrackingWritableRowSet;
import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.TableDefinition;
import io.deephaven.engine.table.TableUpdate;
import io.deephaven.engine.table.TableUpdateListener;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableDataException;
import io.deephaven.engine.table.impl.locations.TableLocationProvider;
import io.deephaven.engine.table.impl.locations.TableLocationRemovedException;
import io.deephaven.engine.table.impl.locations.*;
import io.deephaven.engine.updategraph.UpdateSourceRegistrar;
import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder;
import io.deephaven.engine.table.impl.locations.impl.TableLocationSubscriptionBuffer;
import io.deephaven.engine.rowset.RowSet;
import io.deephaven.engine.rowset.RowSetFactory;
import io.deephaven.util.SafeCloseable;
import io.deephaven.util.annotations.TestUseOnly;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
import org.jetbrains.annotations.NotNull;

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

/**
Expand Down Expand Up @@ -101,12 +98,12 @@ public abstract class SourceTable<IMPL_TYPE extends SourceTable<IMPL_TYPE>> exte
definition.getColumns() // This is the *re-written* definition passed to the super-class constructor
);
if (isRefreshing) {
// TODO: managing doesn't work here because columnSourceManager is at zero right now.
manage(columnSourceManager);
}
}

setRefreshing(isRefreshing);
setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE);
}

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

maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys());
maybeAddLocations(locationUpdate.getPendingAddedLocationKeys());
manage(locationBuffer);
try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate =
locationBuffer.processPending()) {
maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys());
maybeAddLocations(locationUpdate.getPendingAddedLocationKeys());
}
updateSourceRegistrar.addSource(locationChangePoller = new LocationChangePoller(locationBuffer));
} else {
locationProvider.refresh();
maybeAddLocations(locationProvider.getTableLocationKeys());
final Collection<LiveSupplier<ImmutableTableLocationKey>> keySuppliers = new ArrayList<>();
// Manage each of the location keys as we see them (since the TLP is not guaranteeing them outside
// the callback)
locationProvider.getTableLocationKeys(ttlk -> {
manage(ttlk);
keySuppliers.add(ttlk);
});
maybeAddLocations(keySuppliers);
// Now we can un-manage the location keys
keySuppliers.forEach(this::unmanage);
}
});
locationsInitialized = true;
}
}

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

private ImmutableTableLocationKey[] maybeRemoveLocations(
@NotNull final Collection<ImmutableTableLocationKey> removedKeys) {
private void maybeRemoveLocations(
@NotNull final Collection<LiveSupplier<ImmutableTableLocationKey>> removedKeys) {
if (removedKeys.isEmpty()) {
return ImmutableTableLocationKey.ZERO_LENGTH_IMMUTABLE_TABLE_LOCATION_KEY_ARRAY;
return;
}

return filterLocationKeys(removedKeys).stream()
.filter(columnSourceManager::removeLocationKey)
.toArray(ImmutableTableLocationKey[]::new);
filterLocationKeys(removedKeys).stream()
.map(LiveSupplier::get)
.forEach(columnSourceManager::removeLocationKey);
}

private void initializeLocationSizes() {
Expand Down Expand Up @@ -216,14 +223,11 @@ private LocationChangePoller(@NotNull final TableLocationSubscriptionBuffer loca

@Override
protected void instrumentedRefresh() {
final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate = locationBuffer.processPending();
final ImmutableTableLocationKey[] removedKeys =
maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys());
if (removedKeys.length > 0) {
throw new TableLocationRemovedException("Source table does not support removed locations",
removedKeys);
try (final TableLocationSubscriptionBuffer.LocationUpdate locationUpdate =
locationBuffer.processPending()) {
maybeRemoveLocations(locationUpdate.getPendingRemovedLocationKeys());
maybeAddLocations(locationUpdate.getPendingAddedLocationKeys());
}
maybeAddLocations(locationUpdate.getPendingAddedLocationKeys());

// This class previously had functionality to notify "location listeners", but it was never used.
// Resurrect from git history if needed.
Expand All @@ -232,13 +236,16 @@ protected void instrumentedRefresh() {
return;
}

final RowSet added = columnSourceManager.refresh();
if (added.isEmpty()) {
final TableUpdate update = columnSourceManager.refresh();
if (update.empty()) {
lbooker42 marked this conversation as resolved.
Show resolved Hide resolved
update.release();
return;
}

rowSet.insert(added);
notifyListeners(added, RowSetFactory.empty(), RowSetFactory.empty());
Assert.eqTrue(update.shifted().empty(), "update.shifted().empty()");
rowSet.remove(update.removed());
rowSet.insert(update.added());
notifyListeners(update);
}

@Override
Expand All @@ -257,8 +264,8 @@ protected void onRefreshError(@NotNull final Exception error) {
* {@link TableLocationProvider}, but not yet incorporated into the table
* @return A sub-collection of the input
*/
protected Collection<ImmutableTableLocationKey> filterLocationKeys(
@NotNull final Collection<ImmutableTableLocationKey> foundLocationKeys) {
protected Collection<LiveSupplier<ImmutableTableLocationKey>> filterLocationKeys(
@NotNull final Collection<LiveSupplier<ImmutableTableLocationKey>> 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.LivenessReferent;
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, LivenessReferent {

/**
* 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 @@ -56,4 +56,9 @@ <PARTITION_VALUE_TYPE extends Comparable<PARTITION_VALUE_TYPE>> PARTITION_VALUE_
* @return An immutable version of this key
*/
ImmutableTableLocationKey makeImmutable();

/**
* Release any cached data associated with this key. This would only be called at EOL for this key.
*/
default void clear() {}
}
Loading
Loading