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 52 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 51 commits
Commits
Show all changes
52 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
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 @@ -3,14 +3,10 @@
//
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.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 All @@ -22,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 @@ -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,18 @@ 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);

/**
* Get a map of Table attributes that can be applied to the output source table, given the update modes of the
* underlying table location provider.
*
* @param tableUpdateMode The update mode of the table location set
* @param tableLocationUpdateMode The update mode of the table location rows
*/
Map<String, Object> getTableAttributes(
@NotNull TableUpdateMode tableUpdateMode,
@NotNull TableUpdateMode tableLocationUpdateMode);
}
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 KEY_SUPPLIER_COLUMN_NAME = "__PartitionAwareSourceTable_KeySupplier__";

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
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(KEY_SUPPLIER_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 key suppliers 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(KEY_SUPPLIER_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.*;
import io.deephaven.engine.primitive.iterator.CloseableIterator;
import io.deephaven.engine.rowset.*;
import io.deephaven.engine.table.*;
Expand All @@ -15,7 +16,9 @@
import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource;
import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl;
import io.deephaven.engine.table.iterators.ChunkedObjectColumnIterator;
import io.deephaven.engine.updategraph.NotificationQueue;
import io.deephaven.engine.updategraph.UpdateCommitter;
import io.deephaven.engine.updategraph.UpdateGraph;
import io.deephaven.engine.updategraph.UpdateSourceCombiner;
import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedNode;
import io.deephaven.util.datastructures.linked.IntrusiveDoublyLinkedQueue;
Expand Down Expand Up @@ -73,7 +76,8 @@ public SourcePartitionedTable(
false);
}

private static final class UnderlyingTableMaintainer {
private static final class UnderlyingTableMaintainer extends ReferenceCountedLivenessNode
implements NotificationQueue.Dependency {

private final TableDefinition constituentDefinition;
private final UnaryOperator<Table> applyTablePermissions;
Expand Down Expand Up @@ -103,13 +107,14 @@ private UnderlyingTableMaintainer(
final boolean refreshLocations,
final boolean refreshSizes,
@NotNull final Predicate<ImmutableTableLocationKey> locationKeyMatcher) {
super(false);

this.constituentDefinition = constituentDefinition;
this.applyTablePermissions = applyTablePermissions;
this.tableLocationProvider = tableLocationProvider;
this.refreshSizes = refreshSizes;
this.locationKeyMatcher = locationKeyMatcher;

// noinspection resource
resultRows = RowSetFactory.empty().toTracking();
resultTableLocationKeys = ArrayBackedColumnSource.getMemoryColumnSource(TableLocationKey.class, null);
resultLocationTables = ArrayBackedColumnSource.getMemoryColumnSource(Table.class, null);
Expand All @@ -123,13 +128,19 @@ private UnderlyingTableMaintainer(
if (needToRefreshLocations || refreshSizes) {
result.setRefreshing(true);
refreshCombiner = new UpdateSourceCombiner(result.getUpdateGraph());
result.addParentReference(refreshCombiner);
result.addParentReference(this);
manage(refreshCombiner);
} else {
refreshCombiner = null;
}

if (needToRefreshLocations) {
resultTableLocationKeys.startTrackingPrevValues();
resultLocationTables.startTrackingPrevValues();

subscriptionBuffer = new TableLocationSubscriptionBuffer(tableLocationProvider);
manage(subscriptionBuffer);

pendingLocationStates = new IntrusiveDoublyLinkedQueue<>(
IntrusiveDoublyLinkedNode.Adapter.<PendingLocationState>getInstance());
readyLocationStates = new IntrusiveDoublyLinkedQueue<>(
Expand All @@ -143,7 +154,6 @@ protected void instrumentedRefresh() {
processPendingLocations(true);
}
};
result.addParentReference(processNewLocationsUpdateRoot);
refreshCombiner.addSource(processNewLocationsUpdateRoot);

this.removedLocationsComitter = new UpdateCommitter<>(
Expand All @@ -154,7 +164,6 @@ protected void instrumentedRefresh() {
removedConstituents.forEach(result::unmanage);
removedConstituents = null;
});

processPendingLocations(false);
} else {
subscriptionBuffer = null;
Expand All @@ -163,9 +172,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 @@ -204,18 +216,30 @@ private Table makeConstituentTable(@NotNull final TableLocation tableLocation) {
constituentDefinition,
"SingleLocationSourceTable-" + tableLocation,
RegionedTableComponentFactoryImpl.INSTANCE,
new SingleTableLocationProvider(tableLocation),
new SingleTableLocationProvider(tableLocation, refreshSizes
? tableLocationProvider.getLocationUpdateMode()
: TableUpdateMode.STATIC),
refreshSizes ? refreshCombiner : null);

// Transfer management to the constituent CSM. NOTE: this is likely to end up double-managed
// after the CSM adds the location to the table, but that's acceptable.
constituent.columnSourceManager.manage(tableLocation);
unmanage(tableLocation);

// Be careful to propagate the systemic attribute properly to child tables
constituent.setAttribute(Table.SYSTEMIC_TABLE_ATTRIBUTE, result.isSystemicObject());
return applyTablePermissions.apply(constituent);
}

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,8 +266,10 @@ 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)
.peek(this::manage)
.map(PendingLocationState::new)
.forEach(pendingLocationStates::offer);
for (final Iterator<PendingLocationState> iter = pendingLocationStates.iterator(); iter.hasNext();) {
Expand All @@ -263,13 +289,24 @@ private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpd
final Set<ImmutableTableLocationKey> relevantRemovedLocations =
locationUpdate.getPendingRemovedLocationKeys()
.stream()
.map(LiveSupplier::get)
.filter(locationKeyMatcher)
.collect(Collectors.toSet());

if (relevantRemovedLocations.isEmpty()) {
return RowSetFactory.empty();
}

// Iterate through the pending locations and remove any that are in the removed set.
for (final Iterator<PendingLocationState> iter = pendingLocationStates.iterator(); iter.hasNext();) {
final PendingLocationState pendingLocationState = iter.next();
if (relevantRemovedLocations.contains(pendingLocationState.location.getKey())) {
iter.remove();
// Release the state and unmanage the location
unmanage(pendingLocationState.release());
}
}

// At the end of the cycle we need to make sure we unmanage any removed constituents.
this.removedConstituents = new ArrayList<>(relevantRemovedLocations.size());
final RowSetBuilderSequential deleteBuilder = RowSetFactory.builderSequential();
Expand Down Expand Up @@ -306,6 +343,22 @@ private RowSet processRemovals(final TableLocationSubscriptionBuffer.LocationUpd
resultLocationTables.setNull(deletedRows);
return deletedRows;
}

@Override
public boolean satisfied(final long step) {
if (refreshCombiner == null) {
throw new UnsupportedOperationException("This method should not be called when result is static");
}
return refreshCombiner.satisfied(step);
}

@Override
public UpdateGraph getUpdateGraph() {
if (refreshCombiner == null) {
throw new UnsupportedOperationException("This method should not be called when result is static");
}
return refreshCombiner.getUpdateGraph();
}
}

private static final class PendingLocationState extends IntrusiveDoublyLinkedNode.Impl<PendingLocationState> {
Expand Down
Loading