diff --git a/Util/src/main/java/io/deephaven/util/SafeCloseableList.java b/Util/src/main/java/io/deephaven/util/SafeCloseableList.java
index de8d54a7459..ee47f5fba6a 100644
--- a/Util/src/main/java/io/deephaven/util/SafeCloseableList.java
+++ b/Util/src/main/java/io/deephaven/util/SafeCloseableList.java
@@ -10,6 +10,12 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import java.util.function.BinaryOperator;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collector;
/**
* {@link SafeCloseable} that will close an internal list of other {@link SafeCloseable}s.
@@ -57,4 +63,35 @@ public final void close() {
}
list.clear();
}
+
+ public static final Collector COLLECTOR = new Collector<>() {
+
+ @Override
+ public Supplier supplier() {
+ return SafeCloseableList::new;
+ }
+
+ @Override
+ public BiConsumer accumulator() {
+ return SafeCloseableList::add;
+ }
+
+ @Override
+ public BinaryOperator combiner() {
+ return (left, right) -> {
+ left.addAll(right.list);
+ return left;
+ };
+ }
+
+ @Override
+ public Function finisher() {
+ return a -> a;
+ }
+
+ @Override
+ public Set characteristics() {
+ return Set.of(Characteristics.IDENTITY_FINISH);
+ }
+ };
}
diff --git a/docker/registry/selenium/gradle.properties b/docker/registry/selenium/gradle.properties
index 168b04db3dc..9f3323a59cc 100644
--- a/docker/registry/selenium/gradle.properties
+++ b/docker/registry/selenium/gradle.properties
@@ -1,3 +1,4 @@
io.deephaven.project.ProjectType=DOCKER_REGISTRY
deephaven.registry.imageName=selenium/standalone-firefox:4.16.1-20231219
deephaven.registry.imageId=selenium/standalone-firefox@sha256:a405fe92b3ce5d7eb31a07e1f99be3d628fdc0e5bdc81febd8dc11786edef024
+deephaven.registry.platform=linux/amd64
diff --git a/engine/api/src/main/java/io/deephaven/engine/table/BasicDataIndex.java b/engine/api/src/main/java/io/deephaven/engine/table/BasicDataIndex.java
new file mode 100644
index 00000000000..13ff41aa000
--- /dev/null
+++ b/engine/api/src/main/java/io/deephaven/engine/table/BasicDataIndex.java
@@ -0,0 +1,118 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.engine.table;
+
+import io.deephaven.engine.liveness.LivenessReferent;
+import io.deephaven.engine.rowset.RowSet;
+import io.deephaven.util.annotations.FinalDefault;
+import org.jetbrains.annotations.NotNull;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Implementations of BasicDataIndex provide a data index for a {@link Table}. The index is itself a {@link Table} with
+ * columns corresponding to the indexed column(s) ("key" columns) and a column of {@link RowSet RowSets} that contain
+ * the row keys for each unique combination of key values (that is, the "group" or "bucket"). The index itself is a
+ * Table containing the key column(s) and the RowSets associated with each unique combination of values. Implementations
+ * may be loaded from persistent storage or created at runtime, e.g. via aggregations.
+ */
+public interface BasicDataIndex extends LivenessReferent {
+
+ /**
+ * Get a map from indexed {@link ColumnSource ColumnSources} to key column names for the index {@link #table()
+ * table}. This map must be ordered in the same order presented by {@link #keyColumnNames()} and used for lookup
+ * keys.
+ *
+ * @return A map designating the key column names for each indexed {@link ColumnSource}
+ */
+ @NotNull
+ Map, String> keyColumnNamesByIndexedColumn();
+
+ /**
+ * Get a list of the key column names for the index {@link #table() table}.
+ *
+ * @return The key column names
+ */
+ @NotNull
+ List keyColumnNames();
+
+ /**
+ * Get the {@link RowSet} column name for the index {@link #table() table}.
+ *
+ * @return The {@link RowSet} column name
+ */
+ @NotNull
+ String rowSetColumnName();
+
+ /**
+ * Get the key {@link ColumnSource ColumnSources} of the index {@link #table() table}.
+ *
+ * @return An array of the key {@link ColumnSource ColumnSources}, to be owned by the caller
+ */
+ @FinalDefault
+ @NotNull
+ default ColumnSource>[] keyColumns() {
+ final Table indexTable = table();
+ return keyColumnNames().stream()
+ .map(indexTable::getColumnSource)
+ .toArray(ColumnSource[]::new);
+ }
+
+ /**
+ * Get the key {@link ColumnSource ColumnSources} of the index {@link #table() table} in the relative order of
+ * {@code indexedColumnSources}.
+ *
+ * @param indexedColumnSources The indexed {@link ColumnSource ColumnSources} in the desired order; must match the
+ * keys of {@link #keyColumnNamesByIndexedColumn()}
+ * @return An array of the key {@link ColumnSource ColumnSources} in the specified order, to be owned by the caller
+ */
+ @FinalDefault
+ @NotNull
+ default ColumnSource>[] keyColumns(@NotNull final ColumnSource>[] indexedColumnSources) {
+ final Table indexTable = table();
+ final Map, String> keyColumnNamesByIndexedColumn = keyColumnNamesByIndexedColumn();
+ // Verify that the provided columns match the indexed columns.
+ if (keyColumnNamesByIndexedColumn.size() != indexedColumnSources.length
+ || !keyColumnNamesByIndexedColumn.keySet().containsAll(Arrays.asList(indexedColumnSources))) {
+ throw new IllegalArgumentException(String.format(
+ "The provided columns %s do not match the index's indexed columns %s",
+ Arrays.toString(indexedColumnSources),
+ keyColumnNamesByIndexedColumn.keySet()));
+ }
+ return Arrays.stream(indexedColumnSources)
+ .map(keyColumnNamesByIndexedColumn::get)
+ .map(indexTable::getColumnSource)
+ .toArray(ColumnSource[]::new);
+ }
+
+ /**
+ * Get the {@link RowSet} {@link ColumnSource} of the index {@link #table() table}.
+ *
+ * @return The {@link RowSet} {@link ColumnSource}
+ */
+ @FinalDefault
+ @NotNull
+ default ColumnSource rowSetColumn() {
+ return table().getColumnSource(rowSetColumnName(), RowSet.class);
+ }
+
+ /**
+ * Get the {@link Table} backing this data index.
+ *
+ * @return The {@link Table}
+ */
+ @NotNull
+ Table table();
+
+ /**
+ * Whether the index {@link #table()} {@link Table#isRefreshing() is refreshing}. Some transformations will force
+ * the index to become static even when the source table is refreshing.
+ *
+ * @return {@code true} if the index {@link #table()} {@link Table#isRefreshing() is refreshing}, {@code false}
+ * otherwise
+ */
+ boolean isRefreshing();
+}
diff --git a/engine/api/src/main/java/io/deephaven/engine/table/ColumnDefinition.java b/engine/api/src/main/java/io/deephaven/engine/table/ColumnDefinition.java
index ac59ed460dd..f5dbdd8cf4b 100644
--- a/engine/api/src/main/java/io/deephaven/engine/table/ColumnDefinition.java
+++ b/engine/api/src/main/java/io/deephaven/engine/table/ColumnDefinition.java
@@ -55,12 +55,6 @@ public enum ColumnType {
*/
Normal,
- /**
- * A column that has "grouping" metadata associated with it, possibly allowing for indexed filters, joins, and
- * aggregations.
- */
- Grouping,
-
/**
* A column that helps define underlying partitions in the storage of the data, which consequently may also be
* used for very efficient filtering.
@@ -393,10 +387,6 @@ public ColumnDefinition withPartitioning() {
return isPartitioning() ? this : new ColumnDefinition<>(name, dataType, componentType, ColumnType.Partitioning);
}
- public ColumnDefinition withGrouping() {
- return isGrouping() ? this : new ColumnDefinition<>(name, dataType, componentType, ColumnType.Grouping);
- }
-
public ColumnDefinition withNormal() {
return columnType == ColumnType.Normal
? this
@@ -414,16 +404,12 @@ public ColumnDefinition> withName(@NotNull final String newName) {
return newName.equals(name) ? this : new ColumnDefinition<>(newName, dataType, componentType, columnType);
}
- public boolean isGrouping() {
- return (columnType == ColumnType.Grouping);
- }
-
public boolean isPartitioning() {
return (columnType == ColumnType.Partitioning);
}
public boolean isDirect() {
- return (columnType == ColumnType.Normal || columnType == ColumnType.Grouping);
+ return (columnType == ColumnType.Normal);
}
/**
diff --git a/engine/api/src/main/java/io/deephaven/engine/table/ColumnSource.java b/engine/api/src/main/java/io/deephaven/engine/table/ColumnSource.java
index bc574cad9ce..23fafe1248e 100644
--- a/engine/api/src/main/java/io/deephaven/engine/table/ColumnSource.java
+++ b/engine/api/src/main/java/io/deephaven/engine/table/ColumnSource.java
@@ -4,16 +4,15 @@
package io.deephaven.engine.table;
import io.deephaven.base.verify.Require;
-import io.deephaven.chunk.attributes.Values;
import io.deephaven.chunk.ChunkType;
-import io.deephaven.engine.rowset.WritableRowSet;
+import io.deephaven.chunk.attributes.Values;
import io.deephaven.engine.rowset.RowSet;
+import io.deephaven.engine.rowset.WritableRowSet;
import io.deephaven.util.annotations.FinalDefault;
import org.jetbrains.annotations.NotNull;
import javax.annotation.Nullable;
import javax.annotation.OverridingMethodsMustInvokeSuper;
-import java.util.Map;
/**
* A "source" for column data - allows cell values to be looked up by (long) keys.
@@ -40,10 +39,26 @@ default ChunkType getChunkType() {
return ChunkType.fromElementType(dataType);
}
+ /**
+ * Return a {@link RowSet row set} where the values in the column source match the given keys.
+ *
+ * @param invertMatch Whether to invert the match, i.e. return the rows where the values do not match the given keys
+ * @param usePrev Whether to use the previous values for the ColumnSource
+ * @param caseInsensitive Whether to perform a case insensitive match
+ * @param dataIndex An optional data index that can be used to accelerate the match (the index table must be
+ * included in snapshot controls or otherwise guaranteed to be current)
+ * @param mapper Restrict results to this row set
+ * @param keys The keys to match in the column
+ *
+ * @return The rows that match the given keys
+ */
WritableRowSet match(
- boolean invertMatch, boolean usePrev, boolean caseInsensitive, @NotNull RowSet mapper, Object... keys);
-
- Map getValuesMapping(RowSet subRange);
+ boolean invertMatch,
+ boolean usePrev,
+ boolean caseInsensitive,
+ @Nullable final DataIndex dataIndex,
+ @NotNull RowSet mapper,
+ Object... keys);
/**
* ColumnSource implementations that track previous values have the option to not actually start tracking previous
@@ -59,21 +74,6 @@ default void startTrackingPrevValues() {
}
}
- /**
- * Compute grouping information for all keys present in this column source.
- *
- * @return A map from distinct data values to a RowSet that contains those values
- */
- Map getGroupToRange();
-
- /**
- * Compute grouping information for (at least) all keys present in rowSet.
- *
- * @param rowSet The RowSet to consider
- * @return A map from distinct data values to a RowSet that contains those values
- */
- Map getGroupToRange(RowSet rowSet);
-
/**
* Determine if this column source is immutable, meaning that the values at a given row key never change.
*
diff --git a/engine/api/src/main/java/io/deephaven/engine/table/DataIndex.java b/engine/api/src/main/java/io/deephaven/engine/table/DataIndex.java
new file mode 100644
index 00000000000..b257bdbc8d0
--- /dev/null
+++ b/engine/api/src/main/java/io/deephaven/engine/table/DataIndex.java
@@ -0,0 +1,149 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.engine.table;
+
+import io.deephaven.base.verify.Assert;
+import io.deephaven.engine.liveness.LivenessReferent;
+import io.deephaven.engine.rowset.RowSequence;
+import io.deephaven.util.annotations.FinalDefault;
+import org.jetbrains.annotations.NotNull;
+
+import java.util.*;
+
+/**
+ * Expansion of {@link BasicDataIndex} to include methods for transforming the index and fast retrieval of row keys from
+ * lookup keys.
+ */
+public interface DataIndex extends BasicDataIndex {
+ /**
+ * Provides a lookup function from a lookup key to the row key in the index table. Lookup keys consist of
+ * reinterpreted values and are specified as follows:
+ *
+ *
No key columns
+ *
"Empty" keys are signified by any zero-length {@code Object[]}
+ *
One key column
+ *
Singular keys are (boxed, if needed) objects
+ *
Multiple key columns
+ *
Compound keys are {@code Object[]} of (boxed, if needed) objects, in the order of the index's key
+ * columns
+ *
+ */
+ interface RowKeyLookup {
+
+ /**
+ * Get the row key in the index table for the provided lookup key.
+ *
+ * @param key The key to lookup
+ * @return The result row key, or {@link RowSequence#NULL_ROW_KEY} if the key is not found.
+ */
+ long apply(Object key, boolean usePrev);
+ }
+
+ /**
+ * Build a {@link RowKeyLookup lookup function} of row keys for this index. If {@link #isRefreshing()} is
+ * {@code true}, this lookup function is only guaranteed to be accurate for the current cycle. Lookup keys should be
+ * in the order of the index's key columns.
+ *
+ * @return A function that provides map-like lookup of index {@link #table()} row keys from an index lookup key
+ */
+ @NotNull
+ RowKeyLookup rowKeyLookup();
+
+ /**
+ * Return a {@link RowKeyLookup lookup function} function of index row keys for this index. If
+ * {@link #isRefreshing()} is {@code true}, this lookup function is only guaranteed to be accurate for the current
+ * cycle. Lookup keys should be in the order of {@code lookupColumns}.
+ *
+ * @param lookupColumns The {@link ColumnSource ColumnSources} to use for the lookup key
+ * @return A function that provides map-like lookup of index {@link #table()} row keys from an index lookup key. The
+ * result must not be used concurrently by more than one thread.
+ */
+ @NotNull
+ @FinalDefault
+ default RowKeyLookup rowKeyLookup(@NotNull final ColumnSource>[] lookupColumns) {
+ final Map, String> keyColumnNamesByIndexedColumn = keyColumnNamesByIndexedColumn();
+ final ColumnSource>[] indexedColumns =
+ keyColumnNamesByIndexedColumn.keySet().toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY);
+
+ final Runnable onMismatch = () -> {
+ throw new IllegalArgumentException(String.format(
+ "The provided lookup columns %s do not match the indexed columns %s",
+ Arrays.toString(lookupColumns), Arrays.toString(indexedColumns)));
+ };
+
+ if (indexedColumns.length != lookupColumns.length) {
+ onMismatch.run();
+ }
+
+ // We will need to create an appropriately mapped index lookup key for each caller-supplied key. Let's create an
+ // int[] mapping the offset into the index's lookup key to the offset into the caller's lookup key.
+ final int[] indexToCallerOffsetMap = new int[lookupColumns.length];
+ boolean sameOrder = true;
+ // This is an N^2 loop but N is expected to be very small and this is called only at creation.
+ for (int ii = 0; ii < indexedColumns.length; ++ii) {
+ boolean found = false;
+ for (int jj = 0; jj < lookupColumns.length; ++jj) {
+ if (indexedColumns[ii] == lookupColumns[jj]) {
+ indexToCallerOffsetMap[ii] = jj;
+ sameOrder &= ii == jj;
+ found = true;
+ break;
+ }
+ }
+ if (!found) {
+ onMismatch.run();
+ }
+ }
+
+ if (sameOrder) {
+ return rowKeyLookup();
+ }
+ Assert.neq(indexToCallerOffsetMap.length, "indexToCallerOffsetMap.length", 1);
+
+ return new RowKeyLookup() {
+ // This is the complex key we need to provide to our lookup function.
+ final Object[] indexKey = new Object[indexToCallerOffsetMap.length];
+
+ @Override
+ public long apply(final Object callerKey, final boolean usePrev) {
+ // This is the complex key provided by the caller.
+ final Object[] callerKeys = (Object[]) callerKey;
+
+ // Assign the caller-supplied keys to the lookup function key in the appropriate order.
+ for (int ii = 0; ii < indexKey.length; ++ii) {
+ indexKey[ii] = callerKeys[indexToCallerOffsetMap[ii]];
+ }
+
+ return rowKeyLookup().apply(indexKey, usePrev);
+ }
+ };
+ }
+
+ /**
+ * Transform and return a new {@link BasicDataIndex} with the provided transform operations applied. Some
+ * transformations will force the result to be a static snapshot even when this DataIndex {@link #isRefreshing() is
+ * refreshing}.
+ *
+ * @param transformer The {@link DataIndexTransformer} specifying the desired transformations
+ *
+ * @return The transformed {@link BasicDataIndex}
+ */
+ @NotNull
+ BasicDataIndex transform(@NotNull DataIndexTransformer transformer);
+
+ /**
+ * Create a new {@link DataIndex} using the same index {@link #table()}, with the indexed columns in
+ * {@link #keyColumnNamesByIndexedColumn()} remapped according to {@code oldToNewColumnMap}. This is used when it is
+ * known that an operation has produced new columns that are equivalent to the old indexed columns. The result index
+ * may keep {@code this} index reachable and
+ * {@link io.deephaven.engine.liveness.LivenessManager#manage(LivenessReferent) live} for its own lifetime, if
+ * necessary.
+ *
+ * @param oldToNewColumnMap Map from the old indexed {@link ColumnSource ColumnSources} to the new indexed
+ * ColumnSources
+ *
+ * @return The remapped {@link DataIndex}
+ */
+ DataIndex remapKeyColumns(@NotNull Map, ColumnSource>> oldToNewColumnMap);
+}
diff --git a/engine/api/src/main/java/io/deephaven/engine/table/DataIndexTransformer.java b/engine/api/src/main/java/io/deephaven/engine/table/DataIndexTransformer.java
new file mode 100644
index 00000000000..989b7dbe245
--- /dev/null
+++ b/engine/api/src/main/java/io/deephaven/engine/table/DataIndexTransformer.java
@@ -0,0 +1,112 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.engine.table;
+
+import io.deephaven.annotations.BuildableStyle;
+import io.deephaven.engine.rowset.RowSet;
+import io.deephaven.util.annotations.FinalDefault;
+import org.immutables.value.Value.Check;
+import org.immutables.value.Value.Default;
+import org.immutables.value.Value.Immutable;
+
+import java.util.Optional;
+
+/**
+ * A transformation to apply to an existing {@link DataIndex data index} in order to produce a transformed
+ * {@link BasicDataIndex}.
+ */
+@Immutable
+@BuildableStyle
+public interface DataIndexTransformer {
+
+ /**
+ * A {@link RowSet} to {@link RowSet#intersect(RowSet) intersect} with input RowSets when producing output RowSets.
+ * If present, the result {@link BasicDataIndex} will be a static snapshot. This is the first transformation applied
+ * if present.
+ */
+ Optional intersectRowSet();
+
+ /*
+ * A {@link RowSet} to {@link RowSet#invert(RowSet) invert} the input RowSets with when producing output RowSets. If
+ * present, the result {@link BasicDataIndex} will be a static snapshot. This is always applied after {@link
+ * #intersectRowSet() if present.
+ */
+ Optional invertRowSet();
+
+ /**
+ * Whether to sort the output {@link BasicDataIndex BasicDataIndex's} {@link BasicDataIndex#table() table} by the
+ * first row key in each output {@link RowSet}. This is always applied after {@link #intersectRowSet()} and
+ * {@link #invertRowSet()} if present. Note that when sorting a {@link BasicDataIndex#isRefreshing() refreshing}
+ * index, operations that rely on the transformed index must be sure to depend on the transformed index,
+ * and not the input index, for correct satisfaction.
+ */
+ @Default
+ default boolean sortByFirstRowKey() {
+ return false;
+ }
+
+ /**
+ * @return Whether the set of transformations will force the result index table to be a static snapshot.
+ */
+ @FinalDefault
+ default boolean snapshotResult() {
+ return intersectRowSet().isPresent() || invertRowSet().isPresent();
+ }
+
+ @Check
+ default void checkNotEmpty() {
+ if (intersectRowSet().isEmpty() && invertRowSet().isEmpty() && !sortByFirstRowKey()) {
+ throw new IllegalArgumentException("DataIndexTransformer must specify at least one transformation");
+ }
+ }
+
+ /**
+ * Create a {@link DataIndexTransformer.Builder builder} that specifies transformations to apply to an existing
+ * {@link DataIndex data index}.
+ *
+ * When multiple transformations are specified, they are applied in a specific order:
+ *
+ *
Intersect the index {@link RowSet RowSets} with the supplied RowSet. Note that the result will always be a
+ * static snapshot.
+ *
Invert the index {@link RowSet RowSets} with the supplied RowSet. Note that the result will always be a
+ * static snapshot.
+ *
Sort the index table by the first row key within each {@link RowSet}. Be careful to use the correct
+ * dependency for satisfaction!
+ *
+ *
+ *
+ * @return A new {@link DataIndexTransformer} builder.
+ */
+ static Builder builder() {
+ return ImmutableDataIndexTransformer.builder();
+ }
+
+ interface Builder {
+
+ /**
+ * Intersect the index {@link RowSet RowSets} with {@code rowSet}. All
+ * {@link DataIndex#transform(DataIndexTransformer) transformations} using the resulting DataIndexTransformer
+ * should be {@link BasicDataIndex#table() materialized} before {@code rowSet} is {@link RowSet#close() closed}.
+ * The result {@link BasicDataIndex} will be a static snapshot.
+ */
+ Builder intersectRowSet(RowSet rowSet);
+
+ /**
+ * Invert the index {@link RowSet RowSets} with the supplied RowSet. All
+ * {@link DataIndex#transform(DataIndexTransformer) transformations} using the resulting DataIndexTransformer
+ * should be {@link BasicDataIndex#table() materialized} before {@code rowSet} is {@link RowSet#close() closed}.
+ * The result {@link BasicDataIndex} will be a static snapshot.
+ */
+ Builder invertRowSet(RowSet rowSet);
+
+ /**
+ * Whether to sort the index table by the first row key within each {@link RowSet}. Defaults to {@code false}.
+ * Be careful to use the correct dependency for satisfaction!
+ */
+ @SuppressWarnings("unused")
+ Builder sortByFirstRowKey(boolean sort);
+
+ DataIndexTransformer build();
+ }
+}
diff --git a/engine/api/src/main/java/io/deephaven/engine/table/TableDefinition.java b/engine/api/src/main/java/io/deephaven/engine/table/TableDefinition.java
index 1933471cfef..050fd54790e 100644
--- a/engine/api/src/main/java/io/deephaven/engine/table/TableDefinition.java
+++ b/engine/api/src/main/java/io/deephaven/engine/table/TableDefinition.java
@@ -221,22 +221,6 @@ public List> getPartitioningColumns() {
return getColumnStream().filter(ColumnDefinition::isPartitioning).collect(Collectors.toList());
}
- /**
- * @return A list of {@link ColumnDefinition column definitions} for all {@link ColumnDefinition.ColumnType#Grouping
- * grouping} columns in the same relative order as the column definitions list
- */
- public List> getGroupingColumns() {
- return getColumnStream().filter(ColumnDefinition::isGrouping).collect(Collectors.toList());
- }
-
- /**
- * @return An array containing the names of all {@link ColumnDefinition.ColumnType#Grouping grouping} columns in the
- * same relative order as the column definitions list
- */
- public String[] getGroupingColumnNamesArray() {
- return getColumnStream().filter(ColumnDefinition::isGrouping).map(ColumnDefinition::getName)
- .toArray(String[]::new);
- }
/**
* @return The column names as a list in the same order as {@link #getColumns()}
diff --git a/engine/api/src/main/java/io/deephaven/engine/table/TupleSource.java b/engine/api/src/main/java/io/deephaven/engine/table/TupleSource.java
index 23b0fb387fd..8fa53269a4f 100644
--- a/engine/api/src/main/java/io/deephaven/engine/table/TupleSource.java
+++ b/engine/api/src/main/java/io/deephaven/engine/table/TupleSource.java
@@ -6,20 +6,11 @@
import io.deephaven.chunk.attributes.Values;
import org.jetbrains.annotations.NotNull;
-import java.util.List;
-
/**
* Factory to produce immutable tuples from a long row key.
*/
public interface TupleSource extends TupleExporter, ChunkSource.WithPrev {
- /**
- * Get the {@link ColumnSource}s backing this tuple source.
- *
- * @return The column sources
- */
- List> getColumnSources();
-
/**
* Create a tuple for key column values at the supplied row key.
*
diff --git a/engine/api/src/main/java/io/deephaven/engine/table/iterators/ChunkedColumnIterator.java b/engine/api/src/main/java/io/deephaven/engine/table/iterators/ChunkedColumnIterator.java
index 71d08190b09..5832555c5d1 100644
--- a/engine/api/src/main/java/io/deephaven/engine/table/iterators/ChunkedColumnIterator.java
+++ b/engine/api/src/main/java/io/deephaven/engine/table/iterators/ChunkedColumnIterator.java
@@ -14,6 +14,8 @@
import java.util.NoSuchElementException;
+import static io.deephaven.chunk.util.pools.ChunkPoolConstants.SMALLEST_POOLED_CHUNK_CAPACITY;
+
/**
* Iteration support for values supplied by a {@link ChunkSource}. Implementations retrieve {@link Chunk chunks} of
* values at a time in a common Deephaven engine retrieval pattern. This is expected to be high throughput relative to
@@ -167,7 +169,8 @@ public static ColumnIterator make(
public static ColumnIterator make(
@NotNull final ChunkSource extends Any> chunkSource,
@NotNull final RowSequence rowSequence,
- final int chunkSize) {
+ int chunkSize) {
+ chunkSize = Math.max((int) Math.min(chunkSize, rowSequence.size()), SMALLEST_POOLED_CHUNK_CAPACITY);
final ColumnIterator> result;
switch (chunkSource.getChunkType()) {
case Char:
diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSequence.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSequence.java
index 97113ffcfc9..2f6d72103fc 100644
--- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSequence.java
+++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSequence.java
@@ -191,6 +191,7 @@ default void forAllRowKeyRanges(LongRangeConsumer lrc) {
* Using any {@code RowSequence} methods after {@code close()} is an error and may produce exceptions or undefined
* results.
*/
+ @Override
default void close() {}
/**
diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java
index 7236f613329..1859a0f8c69 100644
--- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java
+++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSet.java
@@ -18,7 +18,15 @@
*/
public interface RowSet extends RowSequence, LongSizedDataStructure, SafeCloseable, LogOutputAppendable {
- void close();
+ /**
+ *
+ * Free any resources associated with this object.
+ *
+ * Using any {@code RowSet} methods after {@code close()} is an error and may produce exceptions or undefined
+ * results.
+ */
+ @Override
+ default void close() {}
/**
* Make a new {@link WritableRowSet} with the same row keys as {@code this} that is safe for further mutation. As in
@@ -126,9 +134,7 @@ default WritableRowSet invert(RowSet keys) {
/**
* Returns true if a RowSet has any overlap.
*/
- default boolean overlaps(@NotNull RowSet rowSet) {
- return intersect(rowSet).isNonempty();
- }
+ boolean overlaps(@NotNull RowSet rowSet);
/**
* Returns true if this RowSet has any overlap with the provided range.
diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/TrackingRowSet.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/TrackingRowSet.java
index 0455cd97a62..f7bd4bf485c 100644
--- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/TrackingRowSet.java
+++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/TrackingRowSet.java
@@ -91,12 +91,6 @@ default int intSizePrev() {
* Minimal interface for optional, opaque indexer objects hosted by TrackingRowSet instances.
*/
interface Indexer {
-
- /**
- * Callback for the host TrackingRowSet to report a modification that may invalidate cached indexing
- * information.
- */
- void rowSetChanged();
}
/**
@@ -108,6 +102,13 @@ interface Indexer {
*/
INDEXER_TYPE indexer(@NotNull Function indexerFactory);
+ /**
+ * Get an opaque {@link Indexer} object previously associated with this TrackingRowSet.
+ *
+ * @return An opaque indexer object associated with this TrackingRowSet, or {@code null} if none has been set
+ */
+ INDEXER_TYPE indexer();
+
@Override
default TrackingWritableRowSet writableCast() {
return (TrackingWritableRowSet) this;
diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/TrackingWritableRowSetImpl.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/TrackingWritableRowSetImpl.java
index ce7ec8129c6..8ca5daaf710 100644
--- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/TrackingWritableRowSetImpl.java
+++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/impl/TrackingWritableRowSetImpl.java
@@ -76,16 +76,14 @@ public INDEXER_TYPE indexer(
}
@Override
- public void preMutationHook() {
- checkAndGetPrev();
+ public INDEXER_TYPE indexer() {
+ // noinspection unchecked
+ return (INDEXER_TYPE) indexer;
}
@Override
- protected void postMutationHook() {
- final TrackingRowSet.Indexer localIndexer = indexer;
- if (localIndexer != null) {
- localIndexer.rowSetChanged();
- }
+ protected void preMutationHook() {
+ checkAndGetPrev();
}
@Override
@@ -162,12 +160,12 @@ private static class UnmodifiableRowSetImpl extends WritableRowSetImpl {
public UnmodifiableRowSetImpl() {}
@Override
- public final void preMutationHook() {
+ protected final void preMutationHook() {
throw new UnsupportedOperationException("Unmodifiable view must never be mutated");
}
@Override
- public final void postMutationHook() {
+ protected final void postMutationHook() {
throw new UnsupportedOperationException("Unmodifiable view must never be mutated");
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/AbstractColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/AbstractColumnSource.java
index 45cf7ffd191..65d434d6598 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/AbstractColumnSource.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/AbstractColumnSource.java
@@ -5,32 +5,36 @@
import io.deephaven.base.string.cache.CharSequenceUtils;
import io.deephaven.base.verify.Assert;
-import io.deephaven.chunk.attributes.Values;
import io.deephaven.chunk.Chunk;
+import io.deephaven.chunk.LongChunk;
+import io.deephaven.chunk.ObjectChunk;
import io.deephaven.chunk.WritableChunk;
+import io.deephaven.chunk.attributes.Values;
import io.deephaven.engine.context.ExecutionContext;
+import io.deephaven.engine.primitive.iterator.CloseableIterator;
import io.deephaven.engine.rowset.*;
-import io.deephaven.engine.rowset.RowSetFactory;
+import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys;
import io.deephaven.engine.table.ColumnSource;
+import io.deephaven.engine.table.DataIndex;
+import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.impl.chunkfillers.ChunkFiller;
import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter;
import io.deephaven.engine.table.impl.chunkfilter.ChunkMatchFilterFactory;
import io.deephaven.engine.table.impl.sources.UnboxedLongBackedColumnSource;
+import io.deephaven.engine.table.iterators.ChunkedColumnIterator;
import io.deephaven.engine.updategraph.UpdateGraph;
-import io.deephaven.vector.*;
import io.deephaven.hash.KeyedObjectHashSet;
import io.deephaven.hash.KeyedObjectKey;
+import io.deephaven.util.SafeCloseable;
import io.deephaven.util.annotations.VisibleForTesting;
import io.deephaven.util.type.TypeUtils;
+import io.deephaven.vector.*;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
import java.time.Instant;
import java.time.ZonedDateTime;
import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
public abstract class AbstractColumnSource implements
ColumnSource,
@@ -42,16 +46,15 @@ public abstract class AbstractColumnSource implements
*/
public static final long USE_RANGES_AVERAGE_RUN_LENGTH = 5;
+ private static final int CHUNK_SIZE = 1 << 11;
+
protected final Class type;
protected final Class> componentType;
protected final UpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph();
- protected volatile Map groupToRange;
- protected volatile List> rowSetIndexerKey;
-
protected AbstractColumnSource(@NotNull final Class type) {
- this(type, Object.class);
+ this(type, null);
}
public AbstractColumnSource(@NotNull final Class type, @Nullable final Class> elementType) {
@@ -84,7 +87,7 @@ public AbstractColumnSource(@NotNull final Class type, @Nullable final Class<
} else if (ShortVector.class.isAssignableFrom(type)) {
componentType = short.class;
} else {
- componentType = elementType;
+ componentType = elementType == null ? Object.class : elementType;
}
} else {
componentType = null;
@@ -106,71 +109,85 @@ public ColumnSource getPrevSource() {
return new PrevColumnSource<>(this);
}
- @Override
- public List> getColumnSources() {
- List> localRowSetIndexerKey;
- if ((localRowSetIndexerKey = rowSetIndexerKey) == null) {
- synchronized (this) {
- if ((localRowSetIndexerKey = rowSetIndexerKey) == null) {
- rowSetIndexerKey = localRowSetIndexerKey = Collections.singletonList(this);
- }
- }
- }
- return localRowSetIndexerKey;
- }
-
- @Override
- public Map getGroupToRange() {
- return groupToRange;
- }
-
- @Override
- public Map getGroupToRange(RowSet rowSet) {
- return groupToRange;
- }
-
- public final void setGroupToRange(@Nullable Map groupToRange) {
- this.groupToRange = groupToRange;
- }
-
@Override
public WritableRowSet match(
final boolean invertMatch,
final boolean usePrev,
final boolean caseInsensitive,
+ @Nullable final DataIndex dataIndex,
@NotNull final RowSet mapper,
final Object... keys) {
- final Map groupToRange = (isImmutable() || !usePrev) ? getGroupToRange(mapper) : null;
- if (groupToRange != null) {
- RowSetBuilderRandom allInMatchingGroups = RowSetFactory.builderRandom();
- if (caseInsensitive && (type == String.class)) {
- KeyedObjectHashSet keySet = new KeyedObjectHashSet<>(new CIStringKey());
+ if (dataIndex != null) {
+ final Table indexTable = dataIndex.table();
+ final RowSet matchingIndexRows;
+ if (caseInsensitive && type == String.class) {
+ // Linear scan through the index table, accumulating index row keys for case-insensitive matches
+ final RowSetBuilderSequential matchingIndexRowsBuilder = RowSetFactory.builderSequential();
+
+ // noinspection rawtypes
+ final KeyedObjectHashSet keySet = new KeyedObjectHashSet<>(new CIStringKey());
+ // noinspection unchecked
Collections.addAll(keySet, keys);
- for (Map.Entry ent : groupToRange.entrySet()) {
- if (keySet.containsKey(ent.getKey())) {
- allInMatchingGroups.addRowSet(ent.getValue());
+ final RowSet indexRowSet = usePrev ? indexTable.getRowSet().prev() : indexTable.getRowSet();
+ final ColumnSource> indexKeySource =
+ indexTable.getColumnSource(dataIndex.keyColumnNames().get(0), String.class);
+
+ final int chunkSize = (int) Math.min(CHUNK_SIZE, indexRowSet.size());
+ try (final RowSequence.Iterator indexRowSetIterator = indexRowSet.getRowSequenceIterator();
+ final GetContext indexKeyGetContext = indexKeySource.makeGetContext(chunkSize)) {
+ while (indexRowSetIterator.hasMore()) {
+ final RowSequence chunkIndexRows = indexRowSetIterator.getNextRowSequenceWithLength(chunkSize);
+ final ObjectChunk chunkKeys = (usePrev
+ ? indexKeySource.getPrevChunk(indexKeyGetContext, chunkIndexRows)
+ : indexKeySource.getChunk(indexKeyGetContext, chunkIndexRows)).asObjectChunk();
+ final LongChunk chunkRowKeys = chunkIndexRows.asRowKeyChunk();
+ final int thisChunkSize = chunkKeys.size();
+ for (int ii = 0; ii < thisChunkSize; ++ii) {
+ final String key = chunkKeys.get(ii);
+ if (keySet.containsKey(key)) {
+ matchingIndexRowsBuilder.appendKey(chunkRowKeys.get(ii));
+ }
+ }
}
}
+ matchingIndexRows = matchingIndexRowsBuilder.build();
} else {
+ // Use the lookup function to get the index row keys for the matching keys
+ final RowSetBuilderRandom matchingIndexRowsBuilder = RowSetFactory.builderRandom();
+
+ final DataIndex.RowKeyLookup rowKeyLookup = dataIndex.rowKeyLookup();
for (Object key : keys) {
- RowSet range = groupToRange.get(key);
- if (range != null) {
- allInMatchingGroups.addRowSet(range);
+ final long rowKey = rowKeyLookup.apply(key, usePrev);
+ if (rowKey != RowSequence.NULL_ROW_KEY) {
+ matchingIndexRowsBuilder.addKey(rowKey);
}
}
+ matchingIndexRows = matchingIndexRowsBuilder.build();
}
- final WritableRowSet matchingValues;
- try (final RowSet matchingGroups = allInMatchingGroups.build()) {
- if (invertMatch) {
- matchingValues = mapper.minus(matchingGroups);
- } else {
- matchingValues = mapper.intersect(matchingGroups);
+ try (final SafeCloseable ignored = matchingIndexRows) {
+ final WritableRowSet filtered = invertMatch ? mapper.copy() : RowSetFactory.empty();
+ if (matchingIndexRows.isNonempty()) {
+ final ColumnSource indexRowSetSource = usePrev
+ ? dataIndex.rowSetColumn().getPrevSource()
+ : dataIndex.rowSetColumn();
+ try (final CloseableIterator matchingIndexRowSetIterator =
+ ChunkedColumnIterator.make(indexRowSetSource, matchingIndexRows)) {
+ matchingIndexRowSetIterator.forEachRemaining((final RowSet matchingRowSet) -> {
+ if (invertMatch) {
+ filtered.remove(matchingRowSet);
+ } else {
+ try (final RowSet intersected = matchingRowSet.intersect(mapper)) {
+ filtered.insert(intersected);
+ }
+ }
+ });
+ }
}
+ return filtered;
}
- return matchingValues;
} else {
return ChunkFilter.applyChunkFilter(mapper, this, usePrev,
ChunkMatchFilterFactory.getChunkFilter(type, caseInsensitive, invertMatch, keys));
@@ -194,44 +211,6 @@ public boolean equalKey(String s, String s2) {
}
}
- @Override
- public Map getValuesMapping(RowSet subRange) {
- Map result = new LinkedHashMap<>();
- final Map groupToRange = getGroupToRange();
-
- // if we have a grouping we can use it to avoid iterating the entire subRange. The issue is that our grouping
- // could be bigger than the RowSet we care about, by a very large margin. In this case we could be spinning
- // on RowSet intersect operations that are actually useless. This check says that if our subRange is smaller
- // than the number of keys in our grouping, we should just fetch the keys instead and generate the grouping
- // from scratch.
- boolean useGroupToRange = (groupToRange != null) && (groupToRange.size() < subRange.size());
- if (useGroupToRange) {
- for (Map.Entry typeEntry : groupToRange.entrySet()) {
- RowSet mapping = subRange.intersect(typeEntry.getValue());
- if (mapping.size() > 0) {
- result.put(typeEntry.getKey(), mapping);
- }
- }
- } else {
- Map valueToIndexSet = new LinkedHashMap<>();
-
- for (RowSet.Iterator it = subRange.iterator(); it.hasNext();) {
- long key = it.nextLong();
- T value = get(key);
- RowSetBuilderSequential indexes = valueToIndexSet.get(value);
- if (indexes == null) {
- indexes = RowSetFactory.builderSequential();
- }
- indexes.appendKey(key);
- valueToIndexSet.put(value, indexes);
- }
- for (Map.Entry entry : valueToIndexSet.entrySet()) {
- result.put(entry.getKey(), entry.getValue().build());
- }
- }
- return result;
- }
-
@Override
public void fillChunk(@NotNull final FillContext context, @NotNull final WritableChunk super Values> destination,
@NotNull final RowSequence rowSequence) {
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/AddOnlyToBlinkTableAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/AddOnlyToBlinkTableAdapter.java
index 2b705590bd5..a2053e29e1f 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/AddOnlyToBlinkTableAdapter.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/AddOnlyToBlinkTableAdapter.java
@@ -53,7 +53,7 @@ public static Table toBlink(@NotNull final Table table) {
final OperationSnapshotControl snapshotControl =
coalesced.createSnapshotControlIfRefreshing(OperationSnapshotControl::new);
- // noinspection DataFlowIssue swapListener cannot be null here, since we know the table is refreshing
+ // noinspection DataFlowIssue snapshotControl cannot be null here, since we know the table is refreshing
ConstructSnapshot.callDataSnapshotFunction("addOnlyToBlink", snapshotControl,
(final boolean usePrev, final long beforeClockValue) -> {
// Start with the same rows as the original table
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/AsOfJoinHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/AsOfJoinHelper.java
index c9006b15c5c..aad2f6fc6e0 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/AsOfJoinHelper.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/AsOfJoinHelper.java
@@ -3,7 +3,6 @@
//
package io.deephaven.engine.table.impl;
-import io.deephaven.base.Pair;
import io.deephaven.base.verify.Assert;
import io.deephaven.chunk.*;
import io.deephaven.chunk.attributes.Any;
@@ -21,6 +20,7 @@
import io.deephaven.engine.table.impl.asofjoin.StaticHashedAsOfJoinStateManager;
import io.deephaven.engine.table.impl.by.typed.TypedHasherFactory;
import io.deephaven.engine.table.impl.asofjoin.BucketedChunkedAjMergedListener;
+import io.deephaven.engine.table.DataIndex;
import io.deephaven.engine.table.impl.join.JoinListenerRecorder;
import io.deephaven.engine.table.impl.asofjoin.ZeroKeyChunkedAjMergedListener;
import io.deephaven.engine.table.impl.sort.LongSortKernel;
@@ -36,7 +36,6 @@
import io.deephaven.engine.table.impl.util.compact.LongCompactKernel;
import io.deephaven.util.SafeCloseable;
import io.deephaven.util.SafeCloseableList;
-import org.apache.commons.lang3.mutable.MutableInt;
import org.jetbrains.annotations.NotNull;
import java.util.*;
@@ -63,7 +62,7 @@ static Table asOfJoin(JoinControl control, QueryTable leftTable, QueryTable righ
checkColumnConflicts(leftTable, columnsToAdd);
- if (!leftTable.isRefreshing() && leftTable.size() == 0) {
+ if (!leftTable.isRefreshing() && leftTable.isEmpty()) {
return makeResult(leftTable, rightTable, new SingleValueRowRedirection(RowSequence.NULL_ROW_KEY),
columnsToAdd, false);
}
@@ -104,18 +103,16 @@ static Table asOfJoin(JoinControl control, QueryTable leftTable, QueryTable righ
if (rightTable.isRefreshing()) {
if (leftTable.isRefreshing()) {
return bothIncrementalAj(control, leftTable, rightTable, columnsToMatch, columnsToAdd, order,
- disallowExactMatch, stampPair,
- originalLeftSources, leftSources, rightSources, leftStampSource, originalRightStampSource,
- rightStampSource, rowRedirection);
+ disallowExactMatch, stampPair, originalLeftSources, leftSources, originalRightSources,
+ rightSources, leftStampSource, originalRightStampSource, rightStampSource, rowRedirection);
}
return rightTickingLeftStaticAj(control, leftTable, rightTable, columnsToMatch, columnsToAdd, order,
- disallowExactMatch, stampPair, originalLeftSources, leftSources, rightSources, leftStampSource,
- originalRightStampSource, rightStampSource,
- rowRedirection);
+ disallowExactMatch, stampPair, originalLeftSources, leftSources, originalRightSources, rightSources,
+ leftStampSource, originalRightStampSource, rightStampSource, rowRedirection);
} else {
return rightStaticAj(control, leftTable, rightTable, columnsToMatch, columnsToAdd, order,
- disallowExactMatch, stampPair, originalLeftSources, leftSources, rightSources, leftStampSource,
- originalRightStampSource, rightStampSource, rowRedirection);
+ disallowExactMatch, stampPair, originalLeftSources, leftSources, originalRightSources, rightSources,
+ leftStampSource, originalRightStampSource, rightStampSource, rowRedirection);
}
}
@@ -130,134 +127,79 @@ private static Table rightStaticAj(JoinControl control,
MatchPair stampPair,
ColumnSource>[] originalLeftSources,
ColumnSource>[] leftSources,
+ ColumnSource>[] originalRightSources,
ColumnSource>[] rightSources,
ColumnSource> leftStampSource,
ColumnSource> originalRightStampSource,
ColumnSource> rightStampSource,
WritableRowRedirection rowRedirection) {
- final IntegerArraySource slots = new IntegerArraySource();
- final int slotCount;
-
- final boolean buildLeft;
- final int size;
-
- final Map, RowSet> leftGrouping;
- final Map, RowSet> rightGrouping;
-
- if (control.useGrouping(leftTable, leftSources)) {
- leftGrouping = leftSources[0].getGroupToRange(leftTable.getRowSet());
- final int leftSize = leftGrouping.size();
-
- if (control.useGrouping(rightTable, rightSources)) {
- rightGrouping = rightSources[0].getGroupToRange(rightTable.getRowSet());
- final int rightSize = rightGrouping.size();
- buildLeft = leftSize < rightSize;
- size = buildLeft ? control.tableSize(leftSize) : control.tableSize(rightSize);
- } else {
- buildLeft = true;
- size = control.tableSize(leftSize);
- rightGrouping = null;
- }
- } else if (control.useGrouping(rightTable, rightSources)) {
- rightGrouping = rightSources[0].getGroupToRange(rightTable.getRowSet());
- leftGrouping = null;
-
- final int rightSize = rightGrouping.size();
- buildLeft = !leftTable.isRefreshing() && leftTable.size() < rightSize;
- size = control.tableSize(Math.min(leftTable.size(), rightSize));
+ // region This block is mostly copied to other entry points
+ final DataIndex leftDataIndex = control.dataIndexToUse(leftTable, originalLeftSources);
+ final Table leftDataIndexTable = leftDataIndex == null ? null : leftDataIndex.table();
+ final DataIndex rightDataIndex = control.dataIndexToUse(rightTable, originalRightSources);
+ final Table rightDataIndexTable = rightDataIndex == null ? null : rightDataIndex.table();
+ final JoinControl.BuildParameters buildParameters =
+ control.buildParameters(leftTable, leftDataIndexTable, rightTable, rightDataIndexTable);
+
+ final RowSet leftRowSetToUse;
+ final ColumnSource>[] leftSourcesToUse;
+ final RowSet rightRowSetToUse;
+ final ColumnSource>[] rightSourcesToUse;
+
+ if (leftDataIndexTable != null) {
+ Assert.neq(buildParameters.firstBuildFrom(), "build from", JoinControl.BuildParameters.From.LeftInput);
+ leftRowSetToUse = leftDataIndexTable.getRowSet();
+ leftSourcesToUse =
+ ReinterpretUtils.maybeConvertToPrimitive(leftDataIndex.keyColumns(originalLeftSources));
} else {
- buildLeft = !leftTable.isRefreshing() && control.buildLeft(leftTable, rightTable);
- size = control.initialBuildSize();
- leftGrouping = rightGrouping = null;
+ leftRowSetToUse = leftTable.getRowSet();
+ leftSourcesToUse = leftSources;
}
-
- final StaticHashedAsOfJoinStateManager asOfJoinStateManager;
- if (buildLeft) {
- asOfJoinStateManager =
- TypedHasherFactory.make(StaticAsOfJoinStateManagerTypedBase.class,
- leftSources, originalLeftSources, size,
- control.getMaximumLoadFactor(), control.getTargetLoadFactor());
+ if (rightDataIndexTable != null) {
+ Assert.neq(buildParameters.firstBuildFrom(), "build from", JoinControl.BuildParameters.From.RightInput);
+ rightRowSetToUse = rightDataIndexTable.getRowSet();
+ rightSourcesToUse =
+ ReinterpretUtils.maybeConvertToPrimitive(rightDataIndex.keyColumns(originalRightSources));
} else {
- asOfJoinStateManager =
- TypedHasherFactory.make(StaticAsOfJoinStateManagerTypedBase.class,
- leftSources, originalLeftSources, size,
- control.getMaximumLoadFactor(), control.getTargetLoadFactor());
+ rightRowSetToUse = rightTable.getRowSet();
+ rightSourcesToUse = rightSources;
}
+ // endregion This block is mostly copied to other entry points
+ final StaticHashedAsOfJoinStateManager asOfJoinStateManager = TypedHasherFactory.make(
+ StaticAsOfJoinStateManagerTypedBase.class, leftSources, originalLeftSources,
+ buildParameters.hashTableSize(), control.getMaximumLoadFactor(), control.getTargetLoadFactor());
+ final IntegerArraySource slots = new IntegerArraySource();
- final Pair, ObjectArraySource> leftGroupedSources;
- final int leftGroupingSize;
- if (leftGrouping != null) {
- final MutableInt groupSize = new MutableInt();
- // noinspection unchecked,rawtypes
- leftGroupedSources = GroupingUtils.groupingToFlatSources((ColumnSource) leftSources[0], leftGrouping,
- leftTable.getRowSet(), groupSize);
- leftGroupingSize = groupSize.intValue();
- } else {
- leftGroupedSources = null;
- leftGroupingSize = 0;
- }
-
- final Pair, ObjectArraySource> rightGroupedSources;
- final int rightGroupingSize;
- if (rightGrouping != null) {
- final MutableInt groupSize = new MutableInt();
- // noinspection unchecked,rawtypes
- rightGroupedSources = GroupingUtils.groupingToFlatSources((ColumnSource) rightSources[0],
- rightGrouping, rightTable.getRowSet(), groupSize);
- rightGroupingSize = groupSize.intValue();
- } else {
- rightGroupedSources = null;
- rightGroupingSize = 0;
+ final int slotCount;
+ switch (buildParameters.firstBuildFrom()) {
+ case LeftInput:
+ case LeftDataIndex:
+ slotCount = asOfJoinStateManager.buildFromLeftSide(leftRowSetToUse, leftSourcesToUse, slots);
+ asOfJoinStateManager.probeRight(rightRowSetToUse, rightSourcesToUse);
+ break;
+ case RightInput:
+ case RightDataIndex:
+ slotCount = asOfJoinStateManager.buildFromRightSide(rightRowSetToUse, rightSourcesToUse, slots);
+ asOfJoinStateManager.probeLeft(leftRowSetToUse, leftSourcesToUse);
+ break;
+ default:
+ // noinspection DataFlowIssue
+ throw Assert.statementNeverExecuted("Unknown build from " + buildParameters.firstBuildFrom());
}
- if (buildLeft) {
- if (leftGroupedSources == null) {
- slotCount = asOfJoinStateManager.buildFromLeftSide(leftTable.getRowSet(), leftSources, slots);
- } else {
- slotCount = asOfJoinStateManager.buildFromLeftSide(RowSetFactory.flat(leftGroupingSize),
- new ColumnSource[] {leftGroupedSources.getFirst()}, slots);
- }
- if (rightGroupedSources == null) {
- asOfJoinStateManager.probeRight(rightTable.getRowSet(), rightSources);
- } else {
- asOfJoinStateManager.probeRight(RowSetFactory.flat(rightGroupingSize),
- new ColumnSource[] {rightGroupedSources.getFirst()});
- }
+ if (rightDataIndex != null) {
+ asOfJoinStateManager.populateRightRowSetsFromIndexTable(slots, slotCount, rightDataIndex.rowSetColumn());
} else {
- if (rightGroupedSources == null) {
- slotCount = asOfJoinStateManager.buildFromRightSide(rightTable.getRowSet(), rightSources, slots);
- } else {
- slotCount =
- asOfJoinStateManager.buildFromRightSide(RowSetFactory.flat(rightGroupingSize),
- new ColumnSource[] {rightGroupedSources.getFirst()}, slots);
- }
- if (leftGroupedSources == null) {
- asOfJoinStateManager.probeLeft(leftTable.getRowSet(), leftSources);
- } else {
- asOfJoinStateManager.probeLeft(RowSetFactory.flat(leftGroupingSize),
- new ColumnSource[] {leftGroupedSources.getFirst()});
- }
+ asOfJoinStateManager.convertRightBuildersToRowSet(slots, slotCount);
}
-
- final ArrayValuesCache arrayValuesCache;
- if (leftTable.isRefreshing()) {
- if (rightGroupedSources != null) {
- asOfJoinStateManager.convertRightGrouping(slots, slotCount, rightGroupedSources.getSecond());
- } else {
- asOfJoinStateManager.convertRightBuildersToIndex(slots, slotCount);
- }
- arrayValuesCache = new ArrayValuesCache(asOfJoinStateManager.getTableSize());
- } else {
- arrayValuesCache = null;
- if (rightGroupedSources != null) {
- asOfJoinStateManager.convertRightGrouping(slots, slotCount, rightGroupedSources.getSecond());
- } else {
- asOfJoinStateManager.convertRightBuildersToIndex(slots, slotCount);
- }
- }
-
+ final ArrayValuesCache arrayValuesCache = leftTable.isRefreshing()
+ ? new ArrayValuesCache(asOfJoinStateManager.getTableSize())
+ : null;
+ final ColumnSource leftDataIndexRowSetColumn = leftDataIndexTable != null
+ ? leftDataIndex.rowSetColumn()
+ : null;
try (final AsOfStampContext stampContext = new AsOfStampContext(order, disallowExactMatch, leftStampSource,
rightStampSource, originalRightStampSource);
final ResettableWritableLongChunk keyChunk =
@@ -266,21 +208,19 @@ private static Table rightStaticAj(JoinControl control,
rightStampSource.getChunkType().makeResettableWritableChunk()) {
for (int slotIndex = 0; slotIndex < slotCount; ++slotIndex) {
final int slot = slots.getInt(slotIndex);
- RowSet leftRowSet = asOfJoinStateManager.getLeftIndex(slot);
+ RowSet leftRowSet = asOfJoinStateManager.getLeftRowSet(slot);
if (leftRowSet == null || leftRowSet.isEmpty()) {
continue;
}
- final RowSet rightRowSet = asOfJoinStateManager.getRightIndex(slot);
+ final RowSet rightRowSet = asOfJoinStateManager.getRightRowset(slot);
if (rightRowSet == null || rightRowSet.isEmpty()) {
continue;
}
- if (leftGroupedSources != null) {
- if (leftRowSet.size() != 1) {
- throw new IllegalStateException("Groupings should have exactly one row key!");
- }
- leftRowSet = leftGroupedSources.getSecond().get(leftRowSet.get(0));
+ if (leftDataIndexRowSetColumn != null) {
+ Assert.eq(leftRowSet.size(), "Indexed left row set size", 1);
+ leftRowSet = leftDataIndexRowSetColumn.get(leftRowSet.get(0));
}
if (arrayValuesCache != null) {
@@ -347,8 +287,8 @@ public void onUpdate(TableUpdate upstream) {
for (int ii = 0; ii < slotCount; ++ii) {
final int slot = updatedSlots.getInt(ii);
- final RowSet leftRowSet = asOfJoinStateManager.getLeftIndex(slot);
- final RowSet rightRowSet = asOfJoinStateManager.getRightIndex(slot);
+ final RowSet leftRowSet = asOfJoinStateManager.getLeftRowSet(slot);
+ final RowSet rightRowSet = asOfJoinStateManager.getRightRowset(slot);
assert arrayValuesCache != null;
processLeftSlotWithRightCache(stampContext, leftRowSet, rightRowSet, rowRedirection,
rightStampSource, keyChunk, valuesChunk, arrayValuesCache, slot);
@@ -422,7 +362,6 @@ private static void processLeftSlotWithRightCache(AsOfStampContext stampContext,
keyChunk.resetFromTypedArray(keyIndices, 0, rightSize);
valuesChunk.resetFromArray(rightStampArray, 0, rightSize);
- // noinspection unchecked
stampContext.getAndCompactStamps(rightRowSet, keyChunk, valuesChunk);
if (keyChunk.size() < rightSize) {
@@ -443,7 +382,6 @@ private static void processLeftSlotWithRightCache(AsOfStampContext stampContext,
valuesChunk.resetFromArray(arrayValuesCache.getValues(slot), 0, rightStampKeys.length);
}
- // noinspection unchecked
stampContext.processEntry(leftRowSet, valuesChunk, keyChunk, rowRedirection);
}
@@ -462,7 +400,7 @@ private static void getCachedLeftStampsAndKeys(RightIncrementalHashedAsOfJoinSta
final long[] leftStampKeys = arrayValuesCache.getKeys(slot);
if (leftStampKeys == null) {
if (leftRowSet == null) {
- leftRowSet = asOfJoinStateManager.getAndClearLeftIndex(slot);
+ leftRowSet = asOfJoinStateManager.getAndClearLeftRowSet(slot);
if (leftRowSet == null) {
leftRowSet = RowSetFactory.empty();
}
@@ -475,13 +413,10 @@ private static void getCachedLeftStampsAndKeys(RightIncrementalHashedAsOfJoinSta
keyChunk.resetFromTypedArray(keyIndices, 0, leftSize);
valuesChunk.resetFromArray(leftStampArray, 0, leftSize);
- // noinspection unchecked
leftRowSet.fillRowKeyChunk(keyChunk);
- // noinspection unchecked
leftStampSource.fillChunk(fillContext.ensureCapacity(leftSize), valuesChunk, leftRowSet);
- // noinspection unchecked
sortContext.ensureCapacity(leftSize).sort(keyChunk, valuesChunk);
arrayValuesCache.setKeysAndValues(slot, keyIndices, leftStampArray);
@@ -517,6 +452,7 @@ private static Table rightTickingLeftStaticAj(JoinControl control,
MatchPair stampPair,
ColumnSource>[] originalLeftSources,
ColumnSource>[] leftSources,
+ ColumnSource>[] originalRightSources,
ColumnSource>[] rightSources,
ColumnSource> leftStampSource,
ColumnSource> originalRightStampSource,
@@ -533,16 +469,53 @@ private static Table rightTickingLeftStaticAj(JoinControl control,
SegmentedSortedArray.makeFactory(stampChunkType, reverse, control.rightSsaNodeSize());
final ChunkSsaStamp chunkSsaStamp = ChunkSsaStamp.make(stampChunkType, reverse);
- final int tableSize = control.initialBuildSize();
-
- final RightIncrementalHashedAsOfJoinStateManager asOfJoinStateManager =
- TypedHasherFactory.make(RightIncrementalAsOfJoinStateManagerTypedBase.class,
- leftSources, originalLeftSources, tableSize,
- control.getMaximumLoadFactor(), control.getTargetLoadFactor());
+ // region This block is mostly copied from rightStaticAj
+ final DataIndex leftDataIndex = control.dataIndexToUse(leftTable, originalLeftSources);
+ final Table leftDataIndexTable = leftDataIndex == null ? null : leftDataIndex.table();
+ final DataIndex rightDataIndex = control.dataIndexToUse(rightTable, originalRightSources);
+ final Table rightDataIndexTable = rightDataIndex == null ? null : rightDataIndex.table();
+ final JoinControl.BuildParameters buildParameters =
+ control.buildParameters(leftTable, leftDataIndexTable, rightTable, rightDataIndexTable);
+
+ final RowSet leftRowSetToUse;
+ final ColumnSource>[] leftSourcesToUse;
+ final RowSet rightRowSetToUse;
+ final ColumnSource>[] rightSourcesToUse;
+
+ if (leftDataIndexTable != null) {
+ Assert.eq(buildParameters.firstBuildFrom(), "build from", JoinControl.BuildParameters.From.LeftDataIndex);
+ leftRowSetToUse = leftDataIndexTable.getRowSet();
+ leftSourcesToUse =
+ ReinterpretUtils.maybeConvertToPrimitive(leftDataIndex.keyColumns(originalLeftSources));
+ } else {
+ Assert.eq(buildParameters.firstBuildFrom(), "build from", JoinControl.BuildParameters.From.LeftInput);
+ leftRowSetToUse = leftTable.getRowSet();
+ leftSourcesToUse = leftSources;
+ }
+ if (rightDataIndexTable != null) {
+ rightRowSetToUse = rightDataIndexTable.getRowSet();
+ rightSourcesToUse =
+ ReinterpretUtils.maybeConvertToPrimitive(rightDataIndex.keyColumns(originalRightSources));
+ } else {
+ rightRowSetToUse = rightTable.getRowSet();
+ rightSourcesToUse = rightSources;
+ }
+ // endregion This block is mostly copied from rightStaticAj
+ final RightIncrementalHashedAsOfJoinStateManager asOfJoinStateManager = TypedHasherFactory.make(
+ RightIncrementalAsOfJoinStateManagerTypedBase.class, leftSources, originalLeftSources,
+ buildParameters.hashTableSize(), control.getMaximumLoadFactor(), control.getTargetLoadFactor());
final IntegerArraySource slots = new IntegerArraySource();
- final int slotCount = asOfJoinStateManager.buildFromLeftSide(leftTable.getRowSet(), leftSources, slots);
- asOfJoinStateManager.probeRightInitial(rightTable.getRowSet(), rightSources);
+
+ final int slotCount = asOfJoinStateManager.buildFromLeftSide(leftRowSetToUse, leftSourcesToUse, slots);
+ asOfJoinStateManager.probeRightInitial(rightRowSetToUse, rightSourcesToUse);
+
+ if (leftDataIndex != null) {
+ asOfJoinStateManager.populateLeftRowSetsFromIndexTable(slots, slotCount, leftDataIndex.rowSetColumn());
+ }
+ if (rightDataIndex != null) {
+ asOfJoinStateManager.populateRightRowSetsFromIndexTable(slots, slotCount, rightDataIndex.rowSetColumn());
+ }
final ArrayValuesCache leftValuesCache = new ArrayValuesCache(asOfJoinStateManager.getTableSize());
final SizedSafeCloseable> sortContext =
@@ -562,17 +535,17 @@ private static Table rightTickingLeftStaticAj(JoinControl control,
rightStampSource.getChunkType().makeResettableWritableChunk()) {
for (int slotIndex = 0; slotIndex < slotCount; ++slotIndex) {
final int slot = slots.getInt(slotIndex);
- final RowSet leftRowSet = asOfJoinStateManager.getAndClearLeftIndex(slot);
+ final RowSet leftRowSet = asOfJoinStateManager.getAndClearLeftRowSet(slot);
assert leftRowSet != null;
- assert leftRowSet.size() > 0;
+ assert leftRowSet.isNonempty();
- final SegmentedSortedArray rightSsa = asOfJoinStateManager.getRightSsa(slot, (rightIndex) -> {
+ final SegmentedSortedArray rightSsa = asOfJoinStateManager.getRightSsa(slot, (rightRowSet) -> {
final SegmentedSortedArray ssa = ssaFactory.get();
- final int slotSize = rightIndex.intSize();
+ final int slotSize = rightRowSet.intSize();
if (slotSize > 0) {
rightStampSource.fillChunk(rightStampFillContext.ensureCapacity(slotSize),
- rightValues.ensureCapacity(slotSize), rightIndex);
- rightIndex.fillRowKeyChunk(rightKeyIndices.ensureCapacity(slotSize));
+ rightValues.ensureCapacity(slotSize), rightRowSet);
+ rightRowSet.fillRowKeyChunk(rightKeyIndices.ensureCapacity(slotSize));
sortContext.ensureCapacity(slotSize).sort(rightKeyIndices.get(), rightValues.get());
ssa.insert(rightValues.get(), rightKeyIndices.get());
}
@@ -589,7 +562,6 @@ private static Table rightTickingLeftStaticAj(JoinControl control,
final WritableLongChunk rightKeysForLeftChunk =
rightKeysForLeft.ensureCapacity(leftRowSet.intSize());
- // noinspection unchecked
chunkSsaStamp.processEntry(leftValuesChunk, leftKeyChunk, rightSsa, rightKeysForLeftChunk,
disallowExactMatch);
@@ -684,7 +656,6 @@ public void onUpdate(TableUpdate upstream) {
rightSsa.removeAndGetPrior(rightValues.get(), rightKeyIndices.get(), priorRedirections.get());
- // noinspection unchecked
chunkSsaStamp.processRemovals(leftValuesChunk, leftKeyChunk, rightValues.get(),
rightKeyIndices.get(), priorRedirections.get(), rowRedirection, modifiedBuilder,
disallowExactMatch);
@@ -712,7 +683,7 @@ public void onUpdate(TableUpdate upstream) {
continue;
}
- final int shiftedSlots = asOfJoinStateManager.gatherShiftIndex(rowSetToShift,
+ final int shiftedSlots = asOfJoinStateManager.gatherShiftRowSet(rowSetToShift,
rightSources, slots, sequentialBuilders);
rowSetToShift.close();
@@ -741,14 +712,12 @@ public void onUpdate(TableUpdate upstream) {
rightValues.get());
if (sit.polarityReversed()) {
- // noinspection unchecked
chunkSsaStamp.applyShift(leftValuesChunk, leftKeyChunk,
rightValues.get(), rightKeyIndices.get(), sit.shiftDelta(),
rowRedirection, disallowExactMatch);
rightSsa.applyShiftReverse(rightValues.get(), rightKeyIndices.get(),
sit.shiftDelta());
} else {
- // noinspection unchecked
chunkSsaStamp.applyShift(leftValuesChunk, leftKeyChunk,
rightValues.get(), rightKeyIndices.get(), sit.shiftDelta(),
rowRedirection, disallowExactMatch);
@@ -858,7 +827,6 @@ public void onUpdate(TableUpdate upstream) {
leftStampFillContext, sortContext, leftKeyChunk, leftValuesChunk,
leftValuesCache, slot);
- // noinspection unchecked
chunkSsaStamp.findModified(0, leftValuesChunk, leftKeyChunk, rowRedirection,
rightValues.get(), rightKeyIndices.get(), modifiedBuilder, disallowExactMatch);
}
@@ -903,6 +871,7 @@ private static Table bothIncrementalAj(JoinControl control,
MatchPair stampPair,
ColumnSource>[] originalLeftSources,
ColumnSource>[] leftSources,
+ ColumnSource>[] originalRightSources,
ColumnSource>[] rightSources,
ColumnSource> leftStampSource,
ColumnSource> originalRightStampSource,
@@ -915,16 +884,53 @@ private static Table bothIncrementalAj(JoinControl control,
SegmentedSortedArray.makeFactory(stampChunkType, reverse, control.rightSsaNodeSize());
final SsaSsaStamp ssaSsaStamp = SsaSsaStamp.make(stampChunkType, reverse);
- final int tableSize = control.initialBuildSize();
-
- final RightIncrementalHashedAsOfJoinStateManager asOfJoinStateManager =
- TypedHasherFactory.make(RightIncrementalAsOfJoinStateManagerTypedBase.class,
- leftSources, originalLeftSources, tableSize,
- control.getMaximumLoadFactor(), control.getTargetLoadFactor());
+ // region This block is mostly copied from rightStaticAj
+ final DataIndex leftDataIndex = control.dataIndexToUse(leftTable, originalLeftSources);
+ final Table leftDataIndexTable = leftDataIndex == null ? null : leftDataIndex.table();
+ final DataIndex rightDataIndex = control.dataIndexToUse(rightTable, originalRightSources);
+ final Table rightDataIndexTable = rightDataIndex == null ? null : rightDataIndex.table();
+ final JoinControl.BuildParameters buildParameters =
+ control.buildParameters(leftTable, leftDataIndexTable, rightTable, rightDataIndexTable);
+
+ final RowSet leftRowSetToUse;
+ final ColumnSource>[] leftSourcesToUse;
+ final RowSet rightRowSetToUse;
+ final ColumnSource>[] rightSourcesToUse;
+
+ if (leftDataIndexTable != null) {
+ Assert.neq(buildParameters.firstBuildFrom(), "build from", JoinControl.BuildParameters.From.LeftInput);
+ leftRowSetToUse = leftDataIndexTable.getRowSet();
+ leftSourcesToUse =
+ ReinterpretUtils.maybeConvertToPrimitive(leftDataIndex.keyColumns(originalLeftSources));
+ } else {
+ leftRowSetToUse = leftTable.getRowSet();
+ leftSourcesToUse = leftSources;
+ }
+ if (rightDataIndexTable != null) {
+ Assert.neq(buildParameters.firstBuildFrom(), "build from", JoinControl.BuildParameters.From.RightInput);
+ rightRowSetToUse = rightDataIndexTable.getRowSet();
+ rightSourcesToUse =
+ ReinterpretUtils.maybeConvertToPrimitive(rightDataIndex.keyColumns(originalRightSources));
+ } else {
+ rightRowSetToUse = rightTable.getRowSet();
+ rightSourcesToUse = rightSources;
+ }
+ // endregion This block is mostly copied from rightStaticAj
+ final RightIncrementalHashedAsOfJoinStateManager asOfJoinStateManager = TypedHasherFactory.make(
+ RightIncrementalAsOfJoinStateManagerTypedBase.class, leftSources, originalLeftSources,
+ buildParameters.hashTableSize(), control.getMaximumLoadFactor(), control.getTargetLoadFactor());
final IntegerArraySource slots = new IntegerArraySource();
- int slotCount = asOfJoinStateManager.buildFromLeftSide(leftTable.getRowSet(), leftSources, slots);
- slotCount = asOfJoinStateManager.buildFromRightSide(rightTable.getRowSet(), rightSources, slots, slotCount);
+
+ int slotCount = asOfJoinStateManager.buildFromLeftSide(leftRowSetToUse, leftSourcesToUse, slots);
+ slotCount = asOfJoinStateManager.buildFromRightSide(rightRowSetToUse, rightSourcesToUse, slots, slotCount);
+
+ if (leftDataIndex != null) {
+ asOfJoinStateManager.populateLeftRowSetsFromIndexTable(slots, slotCount, leftDataIndex.rowSetColumn());
+ }
+ if (rightDataIndex != null) {
+ asOfJoinStateManager.populateRightRowSetsFromIndexTable(slots, slotCount, rightDataIndex.rowSetColumn());
+ }
// These contexts and chunks will be closed when the SSA factory itself is closed by the destroy function of the
// BucketedChunkedAjMergedListener
@@ -946,13 +952,13 @@ public void close() {
}
@Override
- public SegmentedSortedArray apply(RowSet rightIndex) {
+ public SegmentedSortedArray apply(RowSet rightRowSet) {
final SegmentedSortedArray ssa = ssaFactory.get();
- final int slotSize = rightIndex.intSize();
+ final int slotSize = rightRowSet.intSize();
if (slotSize > 0) {
- rightIndex.fillRowKeyChunk(rightStampKeys.ensureCapacity(slotSize));
+ rightRowSet.fillRowKeyChunk(rightStampKeys.ensureCapacity(slotSize));
rightStampSource.fillChunk(rightStampFillContext.ensureCapacity(slotSize),
- rightStampValues.ensureCapacity(slotSize), rightIndex);
+ rightStampValues.ensureCapacity(slotSize), rightRowSet);
sortKernel.ensureCapacity(slotSize).sort(rightStampKeys.get(), rightStampValues.get());
ssa.insert(rightStampValues.get(), rightStampKeys.get());
}
@@ -967,15 +973,15 @@ public void close() {
}
@Override
- public SegmentedSortedArray apply(RowSet leftIndex) {
+ public SegmentedSortedArray apply(RowSet leftRowSet) {
final SegmentedSortedArray ssa = ssaFactory.get();
- final int slotSize = leftIndex.intSize();
+ final int slotSize = leftRowSet.intSize();
if (slotSize > 0) {
leftStampSource.fillChunk(leftStampFillContext.ensureCapacity(slotSize),
- leftStampValues.ensureCapacity(slotSize), leftIndex);
+ leftStampValues.ensureCapacity(slotSize), leftRowSet);
- leftIndex.fillRowKeyChunk(leftStampKeys.ensureCapacity(slotSize));
+ leftRowSet.fillRowKeyChunk(leftStampKeys.ensureCapacity(slotSize));
sortKernel.ensureCapacity(slotSize).sort(leftStampKeys.get(), leftStampValues.get());
@@ -1430,7 +1436,6 @@ private static Table zeroKeyAjRightStatic(QueryTable leftTable, Table rightTable
rightStampSource.getChunkType().makeWritableChunk(rightRowSet.intSize());
try (final SafeCloseableList chunksToClose = new SafeCloseableList(rightStampKeys, rightStampValues)) {
- final Supplier keyStringSupplier = () -> "[] (zero key columns)";
try (final AsOfStampContext stampContext = new AsOfStampContext(order, disallowExactMatch, leftStampSource,
rightStampSource, originalRightStampSource)) {
stampContext.getAndCompactStamps(rightRowSet, rightStampKeys, rightStampValues);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BothIncrementalNaturalJoinStateManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BothIncrementalNaturalJoinStateManager.java
index 87af4f81848..2fb559afe31 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BothIncrementalNaturalJoinStateManager.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BothIncrementalNaturalJoinStateManager.java
@@ -21,6 +21,9 @@ public interface BothIncrementalNaturalJoinStateManager extends IncrementalNatur
void compactAll();
+ WritableRowRedirection buildIndexedRowRedirection(QueryTable leftTable, boolean exactMatch, InitialBuildContext ibc,
+ ColumnSource indexRowSets, JoinControl.RedirectionType redirectionType);
+
WritableRowRedirection buildRowRedirectionFromRedirections(QueryTable leftTable, boolean exactMatch,
InitialBuildContext ibc, JoinControl.RedirectionType redirectionType);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketingContext.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketingContext.java
index 0d70c98f3a5..ed9083ce1eb 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketingContext.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BucketingContext.java
@@ -3,7 +3,10 @@
//
package io.deephaven.engine.table.impl;
+import io.deephaven.base.verify.Assert;
import io.deephaven.chunk.attributes.Values;
+import io.deephaven.engine.rowset.RowSet;
+import io.deephaven.engine.table.DataIndex;
import io.deephaven.engine.table.Table;
import io.deephaven.util.BooleanUtils;
import io.deephaven.util.datastructures.LongSizedDataStructure;
@@ -16,6 +19,7 @@
import io.deephaven.engine.table.impl.sources.regioned.SymbolTableSource;
import io.deephaven.util.SafeCloseable;
import io.deephaven.util.type.TypeUtils;
+import org.jetbrains.annotations.NotNull;
import java.time.Instant;
import java.util.Arrays;
@@ -26,26 +30,46 @@
import static io.deephaven.engine.table.impl.MatchPair.matchString;
class BucketingContext implements SafeCloseable {
- final int keyColumnCount;
- final boolean useLeftGrouping;
- final boolean useRightGrouping;
+
final String listenerDescription;
final ColumnSource>[] leftSources;
- final ColumnSource>[] rightSources;
final ColumnSource>[] originalLeftSources;
+ final ColumnSource>[] rightSources;
+ final ColumnSource>[] originalRightSources;
+
+ final int keyColumnCount;
- ToIntFunctor uniqueFunctor = null;
- boolean uniqueValues = false;
- long maximumUniqueValue = Integer.MAX_VALUE;
- long minimumUniqueValue = Integer.MIN_VALUE;
+ final Table leftDataIndexTable;
+ final ColumnSource>[] leftDataIndexSources;
+ final ColumnSource>[] originalLeftDataIndexSources;
+ final ColumnSource leftDataIndexRowSetSource;
+ final Table rightDataIndexTable;
+ final ColumnSource>[] rightDataIndexSources;
+ final ColumnSource>[] originalRightDataIndexSources;
+ final ColumnSource rightDataIndexRowSetSource;
- BucketingContext(final String listenerPrefix, final QueryTable leftTable, final QueryTable rightTable,
- MatchPair[] columnsToMatch, MatchPair[] columnsToAdd, JoinControl control) {
- final Set leftKeys = leftTable.getDefinition().getColumnNameSet();
+ final JoinControl.BuildParameters buildParameters;
+
+ final boolean uniqueValues;
+ final long minimumUniqueValue;
+ final long maximumUniqueValue;
+ final ToIntFunctor uniqueFunctor;
+
+ BucketingContext(
+ @NotNull final String listenerPrefix,
+ @NotNull final QueryTable leftTable,
+ @NotNull final QueryTable rightTable,
+ @NotNull final MatchPair[] columnsToMatch,
+ @NotNull final MatchPair[] columnsToAdd,
+ @NotNull final JoinControl control,
+ final boolean uniqueRightValues,
+ final boolean useDataIndexes) {
+
+ final Set leftColumnNames = leftTable.getDefinition().getColumnNameSet();
final List conflicts = Arrays.stream(columnsToAdd)
.map(MatchPair::leftColumn)
- .filter(leftKeys::contains)
+ .filter(leftColumnNames::contains)
.collect(Collectors.toList());
if (!conflicts.isEmpty()) {
throw new RuntimeException("Conflicting column names " + conflicts);
@@ -54,18 +78,59 @@ class BucketingContext implements SafeCloseable {
listenerDescription =
listenerPrefix + "(" + matchString(columnsToMatch) + ", " + matchString(columnsToAdd) + ")";
- leftSources = Arrays.stream(columnsToMatch).map(mp -> leftTable.getColumnSource(mp.leftColumn))
- .toArray(ColumnSource[]::new);
- rightSources = Arrays.stream(columnsToMatch).map(mp -> rightTable.getColumnSource(mp.rightColumn))
+ leftSources = Arrays.stream(columnsToMatch)
+ .map(mp -> leftTable.getColumnSource(mp.leftColumn))
.toArray(ColumnSource[]::new);
originalLeftSources = Arrays.copyOf(leftSources, leftSources.length);
+ rightSources = Arrays.stream(columnsToMatch)
+ .map(mp -> rightTable.getColumnSource(mp.rightColumn))
+ .toArray(ColumnSource[]::new);
+ originalRightSources = Arrays.copyOf(rightSources, rightSources.length);
keyColumnCount = leftSources.length;
- useLeftGrouping = JoinControl.useGrouping(leftTable, leftSources);
- // note that the naturalJoin operation ignores this field, because there is never any point to reading or
- // processing grouping information when we have a single row on the right side. Cross join just doesn't support
- // grouping at all (yuck).
- useRightGrouping = JoinControl.useGrouping(rightTable, rightSources);
+
+ final DataIndex leftDataIndex = useDataIndexes ? control.dataIndexToUse(leftTable, originalLeftSources) : null;
+ leftDataIndexTable = leftDataIndex == null ? null : leftDataIndex.table();
+ final DataIndex rightDataIndex;
+
+ if (uniqueRightValues) {
+ rightDataIndex = null;
+ rightDataIndexTable = null;
+ buildParameters = control.buildParametersForUniqueRights(leftTable, leftDataIndexTable, rightTable);
+ } else {
+ rightDataIndex = useDataIndexes ? control.dataIndexToUse(rightTable, originalRightSources) : null;
+ rightDataIndexTable = rightDataIndex == null ? null : rightDataIndex.table();
+ buildParameters = control.buildParameters(leftTable, leftDataIndexTable, rightTable, rightDataIndexTable);
+ }
+
+ if (leftDataIndexTable == null) {
+ leftDataIndexSources = null;
+ originalLeftDataIndexSources = null;
+ leftDataIndexRowSetSource = null;
+ } else {
+ leftDataIndexSources = Arrays.stream(columnsToMatch)
+ .map(mp -> leftDataIndexTable.getColumnSource(mp.leftColumn))
+ .toArray(ColumnSource[]::new);
+ originalLeftDataIndexSources = Arrays.copyOf(leftDataIndexSources, leftDataIndexSources.length);
+ leftDataIndexRowSetSource = leftDataIndex.rowSetColumn();
+ }
+
+ if (rightDataIndexTable == null) {
+ rightDataIndexSources = null;
+ originalRightDataIndexSources = null;
+ rightDataIndexRowSetSource = null;
+ } else {
+ rightDataIndexSources = Arrays.stream(columnsToMatch)
+ .map(mp -> rightDataIndexTable.getColumnSource(mp.rightColumn))
+ .toArray(ColumnSource[]::new);
+ originalRightDataIndexSources = Arrays.copyOf(rightDataIndexSources, rightDataIndexSources.length);
+ rightDataIndexRowSetSource = rightDataIndex.rowSetColumn();
+ }
+
+ boolean localUniqueValues = false;
+ long localMinimumUniqueValue = Integer.MIN_VALUE;
+ long localMaximumUniqueValue = Integer.MAX_VALUE;
+ ToIntFunctor localUniqueFunctor = null;
for (int ii = 0; ii < keyColumnCount; ++ii) {
final Class> leftType = TypeUtils.getUnboxedTypeIfBoxed(leftSources[ii].getType());
@@ -78,23 +143,51 @@ class BucketingContext implements SafeCloseable {
if (leftType == Instant.class) {
// noinspection unchecked
leftSources[ii] = ReinterpretUtils.instantToLongSource((ColumnSource) leftSources[ii]);
+ if (leftDataIndexTable != null) {
+ // noinspection unchecked
+ leftDataIndexSources[ii] =
+ ReinterpretUtils.instantToLongSource((ColumnSource) leftDataIndexSources[ii]);
+ }
// noinspection unchecked
rightSources[ii] = ReinterpretUtils.instantToLongSource((ColumnSource) rightSources[ii]);
+ if (rightDataIndexTable != null) {
+ // noinspection unchecked
+ rightDataIndexSources[ii] =
+ ReinterpretUtils.instantToLongSource((ColumnSource) rightDataIndexSources[ii]);
+ }
} else if (leftType == boolean.class || leftType == Boolean.class) {
// noinspection unchecked
leftSources[ii] = ReinterpretUtils.booleanToByteSource((ColumnSource) leftSources[ii]);
+ if (leftDataIndexTable != null) {
+ // noinspection unchecked
+ leftDataIndexSources[ii] =
+ ReinterpretUtils.booleanToByteSource((ColumnSource) leftDataIndexSources[ii]);
+ }
// noinspection unchecked
rightSources[ii] = ReinterpretUtils.booleanToByteSource((ColumnSource) rightSources[ii]);
+ if (rightDataIndexTable != null) {
+ // noinspection unchecked
+ rightDataIndexSources[ii] =
+ ReinterpretUtils.booleanToByteSource((ColumnSource) rightDataIndexSources[ii]);
+ }
if (leftSources.length == 1) {
- uniqueValues = true;
- maximumUniqueValue = BooleanUtils.TRUE_BOOLEAN_AS_BYTE;
- minimumUniqueValue = BooleanUtils.NULL_BOOLEAN_AS_BYTE;
- uniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Byte,
+ localUniqueValues = true;
+ localMinimumUniqueValue = BooleanUtils.NULL_BOOLEAN_AS_BYTE;
+ localMaximumUniqueValue = BooleanUtils.TRUE_BOOLEAN_AS_BYTE;
+ localUniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Byte,
JoinControl.CHUNK_SIZE, -BooleanUtils.NULL_BOOLEAN_AS_BYTE);
}
} else if (leftType == String.class) {
- if (control.considerSymbolTables(leftTable, rightTable, useLeftGrouping, useRightGrouping,
+ if (control.considerSymbolTables(leftTable, rightTable,
+ leftDataIndexTable != null, rightDataIndexTable != null,
leftSources[ii], rightSources[ii])) {
+ // If we're considering symbol tables, we cannot be using data indexes. Firstly, there's no point,
+ // since we expect that using the data indexes will be faster. Secondly, if one side is refreshing,
+ // we don't have a good way to ensure that the data index table on that side uses the same symbol
+ // keys as the original table.
+ Assert.eqNull(leftDataIndexTable, "leftDataIndexTable");
+ Assert.eqNull(rightDataIndexTable, "rightDataIndexTable");
+
final SymbolTableSource> leftSymbolTableSource = (SymbolTableSource>) leftSources[ii];
final SymbolTableSource> rightSymbolTableSource = (SymbolTableSource>) rightSources[ii];
@@ -103,8 +196,9 @@ class BucketingContext implements SafeCloseable {
final Table rightSymbolTable = rightSymbolTableSource.getStaticSymbolTable(rightTable.getRowSet(),
control.useSymbolTableLookupCaching());
- if (control.useSymbolTables(leftTable.size(), leftSymbolTable.size(), rightTable.size(),
- rightSymbolTable.size())) {
+ if (control.useSymbolTables(
+ leftTable.size(), leftSymbolTable.size(),
+ rightTable.size(), rightSymbolTable.size())) {
final SymbolTableCombiner symbolTableCombiner =
new SymbolTableCombiner(new ColumnSource[] {leftSources[ii]}, SymbolTableCombiner
.hashTableSize(Math.max(leftSymbolTable.size(), rightSymbolTable.size())));
@@ -129,40 +223,44 @@ class BucketingContext implements SafeCloseable {
new SymbolTableToUniqueIdSource(rightSourceAsLong, rightSymbolMapper);
if (leftSources.length == 1) {
- uniqueValues = true;
- maximumUniqueValue = symbolTableCombiner.getMaximumIdentifier();
- minimumUniqueValue = 0;
- uniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Int,
+ localUniqueValues = true;
+ localMinimumUniqueValue = 0;
+ localMaximumUniqueValue = symbolTableCombiner.getMaximumIdentifier();
+ localUniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Int,
JoinControl.CHUNK_SIZE, 0);
}
}
}
} else if (leftType == byte.class) {
if (leftSources.length == 1) {
- uniqueValues = true;
- maximumUniqueValue = Byte.MAX_VALUE;
- minimumUniqueValue = Byte.MIN_VALUE;
- uniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Byte,
+ localUniqueValues = true;
+ localMinimumUniqueValue = Byte.MIN_VALUE;
+ localMaximumUniqueValue = Byte.MAX_VALUE;
+ localUniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Byte,
JoinControl.CHUNK_SIZE, -Byte.MIN_VALUE);
}
} else if (leftType == char.class) {
if (leftSources.length == 1) {
- uniqueValues = true;
- maximumUniqueValue = Character.MAX_VALUE;
- minimumUniqueValue = Character.MIN_VALUE;
- uniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Char,
+ localUniqueValues = true;
+ localMinimumUniqueValue = Character.MIN_VALUE;
+ localMaximumUniqueValue = Character.MAX_VALUE;
+ localUniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Char,
JoinControl.CHUNK_SIZE, -Character.MIN_VALUE);
}
} else if (leftType == short.class) {
if (leftSources.length == 1) {
- uniqueValues = true;
- maximumUniqueValue = Short.MAX_VALUE;
- minimumUniqueValue = Short.MIN_VALUE;
- uniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Short,
+ localUniqueValues = true;
+ localMinimumUniqueValue = Short.MIN_VALUE;
+ localMaximumUniqueValue = Short.MAX_VALUE;
+ localUniqueFunctor = ToIntegerCast.makeToIntegerCast(ChunkType.Short,
JoinControl.CHUNK_SIZE, -Short.MIN_VALUE);
}
}
}
+ this.uniqueValues = localUniqueValues;
+ this.minimumUniqueValue = localMinimumUniqueValue;
+ this.maximumUniqueValue = localMaximumUniqueValue;
+ this.uniqueFunctor = localUniqueFunctor;
}
@Override
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/CodecLookup.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/CodecLookup.java
index 9278cafd07e..9d6e3b449b6 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/CodecLookup.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/CodecLookup.java
@@ -3,14 +3,13 @@
//
package io.deephaven.engine.table.impl;
+import io.deephaven.engine.rowset.RowSet;
import io.deephaven.engine.table.ColumnDefinition;
+import io.deephaven.engine.table.impl.dataindex.RowSetCodec;
+import io.deephaven.util.codec.*;
import io.deephaven.vector.ObjectVector;
import io.deephaven.vector.Vector;
import io.deephaven.stringset.StringSet;
-import io.deephaven.util.codec.CodecCache;
-import io.deephaven.util.codec.ExternalizableCodec;
-import io.deephaven.util.codec.ObjectCodec;
-import io.deephaven.util.codec.SerializableCodec;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
@@ -145,6 +144,10 @@ public static ObjectCodec lookup(@NotNull final Class dataTyp
* @return The default {@link ObjectCodec}
*/
public static ObjectCodec getDefaultCodec(@NotNull final Class dataType) {
+ // TODO (https://github.com/deephaven/deephaven-core/issues/5262): Eliminate reliance on RowSetCodec
+ if (dataType.equals(RowSet.class)) {
+ return CodecCache.DEFAULT.getCodec(RowSetCodec.class.getName(), null);
+ }
if (Externalizable.class.isAssignableFrom(dataType)) {
return CodecCache.DEFAULT.getCodec(ExternalizableCodec.class.getName(), dataType.getName());
}
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 b5315bb6f90..1d5ba841b7a 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
@@ -3,11 +3,18 @@
//
package io.deephaven.engine.table.impl;
+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.impl.locations.ImmutableTableLocationKey;
import io.deephaven.engine.table.impl.locations.TableLocation;
-import io.deephaven.engine.table.impl.sources.DeferredGroupingColumnSource;
import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
import java.util.Collection;
import java.util.Map;
@@ -15,20 +22,14 @@
/**
* Manager for ColumnSources in a Table.
*/
-public interface ColumnSourceManager {
+public interface ColumnSourceManager extends LivenessReferent {
/**
- * Get a map of name to {@link DeferredGroupingColumnSource} for the column sources maintained by this manager.
+ * Get a map of name to {@link ColumnSource} for the column sources maintained by this manager.
*
* @return An unmodifiable view of the column source map maintained by this manager.
*/
- Map> getColumnSources();
-
- /**
- * Turn off column grouping, and clear the groupings on all GROUPING column sources. Note that this does *not*
- * affect PARTITIONING columns.
- */
- void disableGrouping();
+ Map> getColumnSources();
/**
* Add a table location to the list to be checked in run().
@@ -37,13 +38,33 @@ public interface ColumnSourceManager {
*/
void addLocation(@NotNull TableLocation tableLocation);
+ /**
+ * Observe initial sizes for the previously added table locations, and update the managed column sources
+ * accordingly. Create any {@link DataIndex data indexes} that may be derived from the locations.
+ *
+ * @return The initial set of initially-available row keys, to be owned by the caller. This row set will have a
+ * {@link io.deephaven.engine.table.impl.indexer.DataIndexer data indexer} populated with any data indexes
+ * that were created.
+ */
+ TrackingWritableRowSet initialize();
+
/**
* Observe size changes in the previously added table locations, and update the managed column sources accordingly.
- *
- * @return The RowSet of added keys
+ *
+ * @return The set of added row keys, to be owned by the caller
*/
WritableRowSet refresh();
+ /**
+ * Advise this ColumnSourceManager that an error has occurred, and that it will no longer be {@link #refresh()
+ * refreshed}. This method should ensure that the error is delivered to downstream {@link TableListener listeners}
+ * if appropriate.
+ *
+ * @param error The error that occurred
+ * @param entry The failing node's entry, if known
+ */
+ void deliverError(@NotNull Throwable error, @Nullable TableListener.Entry entry);
+
/**
* Get the added locations, first the ones that have been "included" (found to exist with non-zero size) in order of
* inclusion, then the remainder in order of discovery.
@@ -60,6 +81,30 @@ public interface ColumnSourceManager {
@SuppressWarnings("unused")
Collection includedLocations();
+ /**
+ * Get the added locations that have been found to exist and have non-zero size as a table containing the
+ * {@link io.deephaven.engine.rowset.RowSet row sets} for each location. May only be called after
+ * {@link #initialize()}. The returned table will also have columns corresponding to the partitions found in the
+ * locations, for the convenience of many downstream operations.
+ *
+ * @return The added locations that have been found to exist and have non-zero size
+ */
+ Table locationTable();
+
+ /**
+ * Get the name of the column that contains the {@link TableLocation} values from {@link #locationTable()}.
+ *
+ * @return The name of the location column
+ */
+ String locationColumnName();
+
+ /**
+ * Get the name of the column that contains the {@link RowSet} values from {@link #locationTable()}.
+ *
+ * @return The name of the row set column
+ */
+ String rowSetColumnName();
+
/**
* Report whether this ColumnSourceManager has no locations that have been "included" (i.e. found to exist with
* non-zero size).
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/CrossJoinHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/CrossJoinHelper.java
index 288ee21a73f..7bbdc6d0400 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/CrossJoinHelper.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/CrossJoinHelper.java
@@ -9,7 +9,6 @@
import io.deephaven.engine.rowset.*;
import io.deephaven.engine.rowset.RowSetFactory;
import io.deephaven.engine.table.*;
-import io.deephaven.datastructures.util.CollectionUtil;
import io.deephaven.engine.table.impl.join.JoinListenerRecorder;
import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder;
import io.deephaven.engine.table.impl.sources.BitMaskingColumnSource;
@@ -29,6 +28,7 @@
import java.util.function.Function;
import java.util.function.LongConsumer;
+import static io.deephaven.engine.table.impl.JoinControl.BuildParameters.From.LeftInput;
import static io.deephaven.engine.table.impl.MatchPair.matchString;
/**
@@ -138,8 +138,8 @@ private static QueryTable internalJoin(
final boolean leftOuterJoin) {
QueryTable.checkInitiateBinaryOperation(leftTable, rightTable);
- try (final BucketingContext bucketingContext =
- new BucketingContext("join", leftTable, rightTable, columnsToMatch, columnsToAdd, control)) {
+ try (final BucketingContext bucketingContext = new BucketingContext("join",
+ leftTable, rightTable, columnsToMatch, columnsToAdd, control, false, false)) {
// TODO: if we have a single column of unique values, and the range is small, we can use a simplified table
// if (!rightTable.isRefreshing()
// && control.useUniqueTable(uniqueValues, maximumUniqueValue, minumumUniqueValue)){ (etc)
@@ -165,7 +165,8 @@ private static QueryTable internalJoin(
jsm.setMaximumLoadFactor(control.getMaximumLoadFactor());
jsm.setTargetLoadFactor(control.getTargetLoadFactor());
- final WritableRowSet resultRowSet = control.buildLeft(leftTable, rightTable)
+ // noinspection resource
+ final WritableRowSet resultRowSet = bucketingContext.buildParameters.firstBuildFrom() == LeftInput
? jsm.buildFromLeft(leftTable, bucketingContext.leftSources, rightTable,
bucketingContext.rightSources)
: jsm.buildFromRight(leftTable, bucketingContext.leftSources, rightTable,
@@ -187,6 +188,7 @@ private static QueryTable internalJoin(
jsm.setMaximumLoadFactor(control.getMaximumLoadFactor());
jsm.setTargetLoadFactor(control.getTargetLoadFactor());
+ // noinspection resource
final TrackingWritableRowSet resultRowSet =
jsm.buildLeftTicking(leftTable, rightTable, bucketingContext.rightSources).toTracking();
final QueryTable resultTable = makeResult(leftTable, rightTable, columnsToAdd, jsm, resultRowSet,
@@ -287,6 +289,7 @@ public void onUpdate(final TableUpdate upstream) {
jsm.setMaximumLoadFactor(control.getMaximumLoadFactor());
jsm.setTargetLoadFactor(control.getTargetLoadFactor());
+ // noinspection resource
final TrackingWritableRowSet resultRowSet = jsm.build(leftTable, rightTable).toTracking();
final QueryTable resultTable = makeResult(leftTable, rightTable, columnsToAdd, jsm, resultRowSet,
@@ -402,7 +405,7 @@ protected void process() {
final RowSetBuilderRandom leftRowsToVisitForRightRmBuilder = RowSetFactory.builderRandom();
tracker.forAllModifiedSlots(slotState -> {
- if (slotState.leftRowSet.size() > 0 && slotState.rightRemoved.isNonempty()) {
+ if (!slotState.leftRowSet.isEmpty() && slotState.rightRemoved.isNonempty()) {
leftRowsToVisitForRightRmBuilder.addRowSet(slotState.leftRowSet);
}
});
@@ -441,7 +444,7 @@ protected void process() {
leftOuterJoin ? RowSetFactory.builderRandom() : null;
final RowSetBuilderRandom modsToVisit = RowSetFactory.builderRandom();
tracker.forAllModifiedSlots(slotState -> {
- if (slotState.leftRowSet.size() == 0) {
+ if (slotState.leftRowSet.isEmpty()) {
return;
}
if (slotState.rightAdded.isNonempty()) {
@@ -504,7 +507,7 @@ protected void process() {
// removals might generate shifts, so let's add those to our RowSet
final RowSetBuilderRandom rmsToVisit = RowSetFactory.builderRandom();
tracker.forAllModifiedSlots(slotState -> {
- if (slotState.leftRowSet.size() > 0 && slotState.rightRemoved.isNonempty()) {
+ if (!slotState.leftRowSet.isEmpty() && slotState.rightRemoved.isNonempty()) {
rmsToVisit.addRowSet(slotState.leftRowSet);
}
});
@@ -1025,7 +1028,7 @@ private static QueryTable zeroKeyColumnsJoin(
try (final WritableRowSet currRight = rightTable.getRowSet().copy()) {
final RowSetBuilderSequential builder = RowSetFactory.builderSequential();
final MutableLong currRightShift = new MutableLong();
- if (currRight.size() == 0) {
+ if (currRight.isEmpty()) {
if (leftOuterJoin) {
leftTable.getRowSet().forAllRowKeys((currIdx) -> {
final long currResultIdx = currIdx << crossJoinState.getNumShiftBits();
@@ -1041,6 +1044,7 @@ private static QueryTable zeroKeyColumnsJoin(
});
crossJoinState.setRightEmpty(false);
}
+ // noinspection resource
resultRowSet = builder.build().toTracking();
}
@@ -1343,14 +1347,14 @@ protected void process() {
leftTable.addUpdateListener(leftRecorder);
rightTable.addUpdateListener(rightRecorder);
result.addParentReference(mergedListener);
- } else if (leftTable.isRefreshing() && rightTable.size() > 0) {
+ } else if (leftTable.isRefreshing() && !rightTable.isEmpty()) {
leftTable.addUpdateListener(new BaseTable.ListenerImpl(listenerDescription, leftTable, result) {
@Override
public void onUpdate(final TableUpdate upstream) {
onUpdate.accept(upstream, null);
}
});
- } else if (rightTable.isRefreshing() && leftTable.size() > 0) {
+ } else if (rightTable.isRefreshing() && !leftTable.isEmpty()) {
rightTable.addUpdateListener(new BaseTable.ListenerImpl(listenerDescription, rightTable, result) {
@Override
public void onUpdate(final TableUpdate upstream) {
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/GroupingUtils.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/GroupingUtils.java
deleted file mode 100644
index b5b4475b6f0..00000000000
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/GroupingUtils.java
+++ /dev/null
@@ -1,184 +0,0 @@
-//
-// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
-//
-package io.deephaven.engine.table.impl;
-
-import io.deephaven.base.Pair;
-import io.deephaven.base.verify.Assert;
-import io.deephaven.engine.rowset.RowSet;
-import io.deephaven.engine.rowset.TrackingRowSet;
-import io.deephaven.engine.rowset.TrackingWritableRowSet;
-import io.deephaven.engine.rowset.WritableRowSet;
-import io.deephaven.engine.table.ColumnSource;
-import io.deephaven.engine.table.WritableColumnSource;
-import io.deephaven.engine.table.impl.indexer.RowSetIndexer;
-import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource;
-import io.deephaven.engine.table.impl.sources.InMemoryColumnSource;
-import io.deephaven.engine.table.impl.sources.ObjectArraySource;
-import org.apache.commons.lang3.mutable.MutableInt;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.function.BiConsumer;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import static io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource.getMemoryColumnSource;
-
-/**
- * Utilities for creating or interacting with grouping information.
- */
-public class GroupingUtils {
-
- /**
- * Get a map from unique, boxed values in this column to a long[2] range of keys.
- *
- * @param rowSet The RowSet that defines the column along with the column source
- * @param columnSource The column source that defines the column along with the RowSet
- * @return A new value to range map (i.e. grouping metadata)
- */
- public static Map getValueToRangeMap(@NotNull final TrackingRowSet rowSet,
- @Nullable final ColumnSource columnSource) {
- final long size = rowSet.size();
- if (columnSource == null) {
- return Collections.singletonMap(null, new long[] {0, size});
- }
- // noinspection unchecked
- return ((Map) RowSetIndexer.of(rowSet).getGrouping(columnSource)).entrySet().stream()
- .sorted(java.util.Comparator.comparingLong(e -> e.getValue().firstRowKey())).collect(Collectors.toMap(
- Map.Entry::getKey,
- new Function<>() {
- private long prevLastKey = -1L;
- private long currentSize = 0;
-
- @Override
- public long[] apply(@NotNull final Map.Entry entry) {
- final RowSet rowSet = entry.getValue();
- Assert.gt(rowSet.firstRowKey(), "rowSet.firstRowKey()", prevLastKey, "prevLastKey");
- prevLastKey = rowSet.lastRowKey();
- return new long[] {currentSize, currentSize += rowSet.size()};
- }
- },
- Assert::neverInvoked,
- LinkedHashMap::new));
- }
-
- /**
- * Consume all groups in a group-to-RowSet map.
- *
- * @param groupToRowSet The group-to-RowSet map to consume
- * @param groupConsumer Consumer for responsive groups
- */
- public static void forEachGroup(@NotNull final Map groupToRowSet,
- @NotNull final BiConsumer groupConsumer) {
- groupToRowSet.entrySet().stream()
- .filter(kie -> kie.getValue().isNonempty())
- .sorted(java.util.Comparator.comparingLong(kie -> kie.getValue().firstRowKey()))
- .forEachOrdered(kie -> groupConsumer.accept(kie.getKey(), kie.getValue().copy()));
- }
-
- /**
- * Convert a group-to-RowSet map to a pair of flat in-memory column sources, one for the keys and one for the
- * indexes.
- *
- * @param originalKeyColumnSource The key column source whose contents are reflected by the group-to-RowSet map
- * (used for typing, only)
- * @param groupToRowSet The group-to-RowSet map to convert
- * @return A pair of a flat key column source and a flat RowSet column source
- */
- @SuppressWarnings("unused")
- public static Pair, ObjectArraySource> groupingToFlatSources(
- @NotNull final ColumnSource originalKeyColumnSource, @NotNull final Map groupToRowSet) {
- final int numGroups = groupToRowSet.size();
- final WritableColumnSource resultKeyColumnSource = getMemoryColumnSource(
- numGroups, originalKeyColumnSource.getType(), originalKeyColumnSource.getComponentType());
- final ObjectArraySource resultIndexColumnSource =
- new ObjectArraySource<>(TrackingWritableRowSet.class);
- resultIndexColumnSource.ensureCapacity(numGroups);
-
- final MutableInt processedGroupCount = new MutableInt(0);
- forEachGroup(groupToRowSet, (final TYPE key, final WritableRowSet rowSet) -> {
- final long groupIndex = processedGroupCount.longValue();
- resultKeyColumnSource.set(groupIndex, key);
- resultIndexColumnSource.set(groupIndex, rowSet.toTracking());
- processedGroupCount.increment();
- });
- Assert.eq(processedGroupCount.intValue(), "processedGroupCount.intValue()", numGroups, "numGroups");
- return new Pair<>(resultKeyColumnSource, resultIndexColumnSource);
- }
-
- /**
- * Consume all responsive groups in a group-to-RowSet map.
- *
- * @param groupToRowSet The group-to-RowSet map to consume
- * @param intersect Limit indices to values contained within intersect, eliminating empty result groups
- * @param groupConsumer Consumer for responsive groups
- */
- public static void forEachResponsiveGroup(@NotNull final Map groupToRowSet,
- @NotNull final RowSet intersect,
- @NotNull final BiConsumer groupConsumer) {
- groupToRowSet.entrySet().stream()
- .map(kie -> new Pair<>(kie.getKey(), kie.getValue().intersect(intersect)))
- .filter(kip -> kip.getSecond().isNonempty())
- .sorted(java.util.Comparator.comparingLong(kip -> kip.getSecond().firstRowKey()))
- .forEachOrdered(kip -> groupConsumer.accept(kip.getFirst(), kip.getSecond().copy()));
- }
-
- /**
- * Convert a group-to-RowSet map to a pair of flat in-memory column sources, one for the keys and one for the
- * indexes.
- *
- * @param originalKeyColumnSource The key column source whose contents are reflected by the group-to-RowSet map
- * (used for typing, only)
- * @param groupToRowSet The group-to-RowSet map to convert
- * @param intersect Limit returned indices to values contained within intersect
- * @param responsiveGroups Set to the number of responsive groups on exit
- * @return A pair of a flat key column source and a flat rowSet column source
- */
- public static Pair, ObjectArraySource> groupingToFlatSources(
- @NotNull final ColumnSource originalKeyColumnSource,
- @NotNull final Map groupToRowSet,
- @NotNull final RowSet intersect,
- @NotNull final MutableInt responsiveGroups) {
- final int numGroups = groupToRowSet.size();
- final WritableColumnSource resultKeyColumnSource = getMemoryColumnSource(
- numGroups, originalKeyColumnSource.getType(), originalKeyColumnSource.getComponentType());
- final ObjectArraySource resultIndexColumnSource =
- new ObjectArraySource<>(TrackingWritableRowSet.class);
- resultIndexColumnSource.ensureCapacity(numGroups);
-
- responsiveGroups.setValue(0);
- forEachResponsiveGroup(groupToRowSet, intersect, (final TYPE key, final WritableRowSet rowSet) -> {
- final long groupIndex = responsiveGroups.longValue();
- resultKeyColumnSource.set(groupIndex, key);
- resultIndexColumnSource.set(groupIndex, rowSet.toTracking());
- responsiveGroups.increment();
- });
-
- return new Pair<>(resultKeyColumnSource, resultIndexColumnSource);
- }
-
- /**
- * Convert a group-to-RowSet map to a flat, immutable, in-memory column of keys.
- *
- * @param originalKeyColumnSource The key column source whose contents are reflected by the group-to-RowSet map
- * (used for typing, only)
- * @param groupToRowSet The group-to-RowSet map to convert
- * @return A flat, immutable, in-memory column of keys
- */
- public static WritableColumnSource groupingKeysToImmutableFlatSource(
- @NotNull final ColumnSource originalKeyColumnSource,
- @NotNull final Map groupToRowSet) {
- final WritableColumnSource destination = InMemoryColumnSource.makeImmutableSource(
- originalKeyColumnSource.getType(), originalKeyColumnSource.getComponentType());
- destination.ensureCapacity(groupToRowSet.size());
- int ri = 0;
- for (final TYPE key : groupToRowSet.keySet()) {
- destination.set(ri++, key);
- }
- return destination;
- }
-}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/InstrumentedTableUpdateSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/InstrumentedTableUpdateSource.java
index 5e1c7a79a63..8913001f9ca 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/InstrumentedTableUpdateSource.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/InstrumentedTableUpdateSource.java
@@ -32,7 +32,7 @@ public InstrumentedTableUpdateSource(final BaseTable> table, final String desc
}
@Override
- protected final void onRefreshError(@NotNull final Exception error) {
+ protected void onRefreshError(@NotNull final Exception error) {
final BaseTable> table = tableReference.get();
if (table == null) {
return;
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/JoinControl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/JoinControl.java
index 652f8bdc571..279aa73fc1d 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/JoinControl.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/JoinControl.java
@@ -3,13 +3,18 @@
//
package io.deephaven.engine.table.impl;
+import io.deephaven.engine.liveness.LivenessScopeStack;
+import io.deephaven.engine.table.DataIndex;
import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.ColumnSource;
-import io.deephaven.engine.table.impl.indexer.RowSetIndexer;
+import io.deephaven.engine.table.impl.indexer.DataIndexer;
import io.deephaven.engine.table.impl.sources.regioned.SymbolTableSource;
import io.deephaven.engine.table.impl.sources.sparse.SparseConstants;
import io.deephaven.util.annotations.VisibleForTesting;
import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static io.deephaven.engine.table.impl.JoinControl.BuildParameters.From.*;
@VisibleForTesting
public class JoinControl {
@@ -33,14 +38,6 @@ int tableSize(final long initialCapacity) {
Math.min(MAX_TABLE_SIZE, Long.highestOneBit(initialCapacity) * 2)));
}
- int tableSizeForRightBuild(Table rightTable) {
- return tableSize(rightTable.size());
- }
-
- int tableSizeForLeftBuild(Table leftTable) {
- return tableSize(leftTable.size());
- }
-
double getMaximumLoadFactor() {
return DEFAULT_MAX_LOAD_FACTOR;
}
@@ -49,18 +46,170 @@ int tableSizeForLeftBuild(Table leftTable) {
return DEFAULT_TARGET_LOAD_FACTOR;
}
- static boolean useGrouping(Table leftTable, ColumnSource>[] leftSources) {
- return !leftTable.isRefreshing() && leftSources.length == 1
- && RowSetIndexer.of(leftTable.getRowSet()).hasGrouping(leftSources[0]);
+ @Nullable
+ DataIndex dataIndexToUse(Table table, ColumnSource>[] sources) {
+ final DataIndexer indexer = DataIndexer.existingOf(table.getRowSet());
+ return indexer == null ? null
+ : LivenessScopeStack.computeEnclosed(
+ // DataIndexer will only give us valid, live data indexes.
+ () -> indexer.getDataIndex(sources),
+ // Ensure that we use an enclosing scope to manage the data index if needed.
+ table::isRefreshing,
+ // Don't keep the data index managed. Joins hold the update graph lock, so the index can't go
+ // stale,
+ // and we'll only use it during instantiation.
+ di -> false);
+ }
+
+ static final class BuildParameters {
+
+ enum From {
+ LeftInput, LeftDataIndex, RightInput, RightDataIndex
+ }
+
+ private final From firstBuildFrom;
+ private final int hashTableSize;
+
+ BuildParameters(
+ @NotNull final JoinControl.BuildParameters.From firstBuildFrom,
+ final int hashTableSize) {
+ this.firstBuildFrom = firstBuildFrom;
+ this.hashTableSize = hashTableSize;
+ }
+
+ From firstBuildFrom() {
+ return firstBuildFrom;
+ }
+
+ int hashTableSize() {
+ return hashTableSize;
+ }
+ }
+
+ /**
+ * Join Control Goals:
+ *
+ *
Keep the hash table small
+ *
Avoid rehashing
+ *
+ * To meet these goals, we:
+ *
+ *
Both sides refreshing
+ *
Build from a data index if available. Size the hash table for the expected highest cardinality side to
+ * minimize rehashing. Prefer building from the expected highest cardinality side first, even though it's probably
+ * irrelevant.
+ *
One side refreshing
+ *
Always build from the static side. Use a data index if that side has one.
+ *
Both sides static
+ *
Build from the expected lowest cardinality side to avoid creating unnecessary states. Use a data index if
+ * available on expected lowest cardinality side.
+ *
+ */
+ BuildParameters buildParameters(
+ @NotNull final Table leftTable, @Nullable final Table leftDataIndexTable,
+ @NotNull final Table rightTable, @Nullable final Table rightDataIndexTable) {
+
+ // If we're going to do our initial build from a data index, choose the hash table size accordingly.
+ // Else, choose our default initial size.
+ final int leftBuildSize = leftDataIndexTable != null
+ ? tableSize(leftDataIndexTable.size())
+ : initialBuildSize();
+ final int rightBuildSize = rightDataIndexTable != null
+ ? tableSize(rightDataIndexTable.size())
+ : initialBuildSize();
+
+ final BuildParameters.From firstBuildFrom;
+ final int hashTableSize;
+ if (leftTable.isRefreshing() && rightTable.isRefreshing()) {
+ // Both refreshing: build from largest available data index, or largest table if no indexes
+ if (leftDataIndexTable != null && rightDataIndexTable != null) {
+ firstBuildFrom = leftDataIndexTable.size() >= rightDataIndexTable.size()
+ ? LeftDataIndex
+ : RightDataIndex;
+ } else if (leftDataIndexTable != null) {
+ firstBuildFrom = LeftDataIndex;
+ } else if (rightDataIndexTable != null) {
+ firstBuildFrom = RightDataIndex;
+ } else {
+ firstBuildFrom = leftTable.size() >= rightTable.size()
+ ? LeftInput
+ : RightInput;
+ }
+
+ // We need to hold states from both sides. We'll need a table at least big enough for the largest side.
+ hashTableSize = Math.max(leftBuildSize, rightBuildSize);
+
+ } else if (leftTable.isRefreshing()) {
+ // Left refreshing, right static: build from right data index if available, else right table
+ firstBuildFrom = rightDataIndexTable != null
+ ? RightDataIndex
+ : RightInput;
+
+ // We need to hold states from right, only.
+ hashTableSize = rightBuildSize;
+
+ } else if (rightTable.isRefreshing()) {
+ // Left static, right refreshing: build from left data index if available, else left table
+ firstBuildFrom = leftDataIndexTable != null
+ ? LeftDataIndex
+ : LeftInput;
+
+ // We need to hold states from left, only.
+ hashTableSize = leftBuildSize;
+
+ } else {
+ // Both static: build from smallest available data index or smallest table; ties go to smallest table
+ if (leftDataIndexTable != null && rightDataIndexTable != null) {
+ firstBuildFrom = leftDataIndexTable.size() <= rightDataIndexTable.size()
+ ? LeftDataIndex
+ : RightDataIndex;
+ } else if (leftDataIndexTable != null) {
+ firstBuildFrom = leftDataIndexTable.size() < rightTable.size()
+ ? LeftDataIndex
+ : RightInput;
+ } else if (rightDataIndexTable != null) {
+ firstBuildFrom = leftTable.size() <= rightDataIndexTable.size()
+ ? LeftInput
+ : RightDataIndex;
+ } else {
+ firstBuildFrom = leftTable.size() <= rightTable.size()
+ ? LeftInput
+ : RightInput;
+ }
+
+ switch (firstBuildFrom) {
+ case LeftDataIndex: // Fall through
+ case LeftInput:
+ hashTableSize = leftBuildSize;
+ break;
+ case RightDataIndex: // Fall through
+ case RightInput:
+ hashTableSize = rightBuildSize;
+ break;
+ default:
+ throw new IllegalStateException("Unexpected first build from " + firstBuildFrom);
+ }
+ }
+ return new BuildParameters(firstBuildFrom, hashTableSize);
}
- boolean buildLeft(QueryTable leftTable, Table rightTable) {
- return !leftTable.isRefreshing() && leftTable.size() <= rightTable.size();
+ /**
+ * Same as {@link #buildParameters(Table, Table, Table, Table)}, but it's assumed that all RHS rows are unique.
+ * That, is the RHS table is treated like its own data index table in some respects.
+ */
+ BuildParameters buildParametersForUniqueRights(
+ @NotNull final Table leftTable, @Nullable Table leftDataIndexTable, @NotNull final Table rightTable) {
+ // Treat rightTable as its own data index table, and then fix up the firstBuildFrom to not be RightDataIndex.
+ final BuildParameters result = buildParameters(leftTable, leftDataIndexTable, rightTable, rightTable);
+ return result.firstBuildFrom() == RightDataIndex
+ ? new BuildParameters(RightInput, result.hashTableSize())
+ : result;
}
- boolean considerSymbolTables(QueryTable leftTable, @SuppressWarnings("unused") QueryTable rightTable,
- boolean useLeftGrouping, boolean useRightGrouping, ColumnSource> leftSource,
- ColumnSource> rightSource) {
+ boolean considerSymbolTables(
+ Table leftTable, Table rightTable,
+ boolean useLeftGrouping, boolean useRightGrouping,
+ ColumnSource> leftSource, ColumnSource> rightSource) {
return !leftTable.isRefreshing() && !useLeftGrouping && leftSource.getType() == String.class
&& !rightTable.isRefreshing() && !useRightGrouping && rightSource.getType() == String.class
&& leftSource instanceof SymbolTableSource && rightSource instanceof SymbolTableSource
@@ -73,7 +222,7 @@ boolean useSymbolTableLookupCaching() {
}
boolean useSymbolTables(long leftSize, long leftSymbolSize, long rightSize, long rightSymbolSize) {
- final long proposedSymbolSize = Math.min(rightSymbolSize, leftSymbolSize);
+ final long proposedSymbolSize = Math.min(leftSymbolSize, rightSymbolSize);
return proposedSymbolSize <= leftSize / 2 || proposedSymbolSize <= rightSize / 2;
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/NaturalJoinHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/NaturalJoinHelper.java
index 72aca557cc3..3448858b8ad 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/NaturalJoinHelper.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/NaturalJoinHelper.java
@@ -3,7 +3,6 @@
//
package io.deephaven.engine.table.impl;
-import io.deephaven.base.Pair;
import io.deephaven.base.verify.Assert;
import io.deephaven.datastructures.util.CollectionUtil;
import io.deephaven.engine.rowset.*;
@@ -23,6 +22,8 @@
import java.util.LinkedHashMap;
import java.util.Map;
+import static io.deephaven.engine.table.impl.JoinControl.BuildParameters.From.*;
+
class NaturalJoinHelper {
private NaturalJoinHelper() {} // static use only
@@ -54,232 +55,209 @@ private static QueryTable naturalJoinInternal(QueryTable leftTable, QueryTable r
MatchPair[] columnsToMatch, MatchPair[] columnsToAdd, boolean exactMatch, JoinControl control) {
QueryTable.checkInitiateBinaryOperation(leftTable, rightTable);
- try (final BucketingContext bucketingContext =
- new BucketingContext("naturalJoin", leftTable, rightTable, columnsToMatch, columnsToAdd, control)) {
+ try (final BucketingContext bc = new BucketingContext("naturalJoin",
+ leftTable, rightTable, columnsToMatch, columnsToAdd, control, true, true)) {
+ final JoinControl.BuildParameters.From firstBuildFrom = bc.buildParameters.firstBuildFrom();
+ final int initialHashTableSize = bc.buildParameters.hashTableSize();
// if we have a single column of unique values, and the range is small, we can use a simplified table
// TODO: SimpleUniqueStaticNaturalJoinManager, but not static!
- if (!rightTable.isRefreshing() && control.useUniqueTable(bucketingContext.uniqueValues,
- bucketingContext.maximumUniqueValue, bucketingContext.minimumUniqueValue)) {
- Assert.neqNull(bucketingContext.uniqueFunctor, "uniqueFunctor");
+ if (!rightTable.isRefreshing()
+ && control.useUniqueTable(bc.uniqueValues, bc.maximumUniqueValue, bc.minimumUniqueValue)) {
+ Assert.neqNull(bc.uniqueFunctor, "uniqueFunctor");
final SimpleUniqueStaticNaturalJoinStateManager jsm = new SimpleUniqueStaticNaturalJoinStateManager(
- bucketingContext.originalLeftSources, bucketingContext.uniqueValuesRange(),
- bucketingContext.uniqueFunctor);
- jsm.setRightSide(rightTable.getRowSet(), bucketingContext.rightSources[0]);
+ bc.originalLeftSources, bc.uniqueValuesRange(), bc.uniqueFunctor);
+ jsm.setRightSide(rightTable.getRowSet(), bc.rightSources[0]);
final LongArraySource leftRedirections = new LongArraySource();
leftRedirections.ensureCapacity(leftTable.getRowSet().size());
- jsm.decorateLeftSide(leftTable.getRowSet(), bucketingContext.leftSources, leftRedirections);
+ jsm.decorateLeftSide(leftTable.getRowSet(), bc.leftSources, leftRedirections);
final WritableRowRedirection rowRedirection = jsm.buildRowRedirection(leftTable, exactMatch,
leftRedirections, control.getRedirectionType(leftTable));
final QueryTable result = makeResult(leftTable, rightTable, columnsToAdd, rowRedirection, true);
-
- leftTable
- .addUpdateListener(new LeftTickingListener(bucketingContext.listenerDescription, columnsToMatch,
- columnsToAdd, leftTable, result, rowRedirection, jsm, bucketingContext.leftSources));
-
+ if (leftTable.isRefreshing()) {
+ leftTable.addUpdateListener(new LeftTickingListener(bc.listenerDescription, columnsToMatch,
+ columnsToAdd, leftTable, result, rowRedirection, jsm, bc.leftSources));
+ }
return result;
}
- if (bucketingContext.leftSources.length == 0) {
- return zeroKeyColumnsJoin(leftTable, rightTable, columnsToAdd, exactMatch,
- bucketingContext.listenerDescription);
+ if (bc.leftSources.length == 0) {
+ return zeroKeyColumnsJoin(leftTable, rightTable, columnsToAdd, exactMatch, bc.listenerDescription);
}
final WritableRowRedirection rowRedirection;
- if (rightTable.isRefreshing()) {
- if (leftTable.isRefreshing()) {
- if (bucketingContext.useLeftGrouping) {
- throw new UnsupportedOperationException(
- "Grouping is not supported with ticking chunked naturalJoin!");
- }
- // the right side is unique, so we should have a state for it; the left side can have many
- // duplicates
- // so we would prefer to have a smaller table
- final int tableSize = control.tableSizeForRightBuild(rightTable);
+ if (leftTable.isRefreshing() && rightTable.isRefreshing()) {
+ // We always build right first, regardless of the build parameters. This is probably irrelevant.
- final BothIncrementalNaturalJoinStateManager jsm =
- TypedHasherFactory.make(IncrementalNaturalJoinStateManagerTypedBase.class,
- bucketingContext.leftSources, bucketingContext.originalLeftSources,
- tableSize, control.getMaximumLoadFactor(),
- control.getTargetLoadFactor());
- jsm.buildFromRightSide(rightTable, bucketingContext.rightSources);
+ final BothIncrementalNaturalJoinStateManager jsm = TypedHasherFactory.make(
+ IncrementalNaturalJoinStateManagerTypedBase.class, bc.leftSources, bc.originalLeftSources,
+ initialHashTableSize, control.getMaximumLoadFactor(),
+ control.getTargetLoadFactor());
+ jsm.buildFromRightSide(rightTable, bc.rightSources);
- try (final BothIncrementalNaturalJoinStateManager.InitialBuildContext ibc =
- jsm.makeInitialBuildContext()) {
- jsm.decorateLeftSide(leftTable.getRowSet(), bucketingContext.leftSources, ibc);
+ try (final BothIncrementalNaturalJoinStateManager.InitialBuildContext ibc =
+ jsm.makeInitialBuildContext()) {
+ if (bc.leftDataIndexTable != null) {
+ jsm.decorateLeftSide(bc.leftDataIndexTable.getRowSet(), bc.leftDataIndexSources, ibc);
+ rowRedirection = jsm.buildIndexedRowRedirection(leftTable, exactMatch, ibc,
+ bc.leftDataIndexRowSetSource, control.getRedirectionType(leftTable));
+ } else {
+ jsm.decorateLeftSide(leftTable.getRowSet(), bc.leftSources, ibc);
jsm.compactAll();
-
rowRedirection = jsm.buildRowRedirectionFromRedirections(leftTable, exactMatch, ibc,
control.getRedirectionType(leftTable));
}
+ }
- final QueryTable result = makeResult(leftTable, rightTable, columnsToAdd, rowRedirection, true);
+ final QueryTable result = makeResult(leftTable, rightTable, columnsToAdd, rowRedirection, true);
- final JoinListenerRecorder leftRecorder =
- new JoinListenerRecorder(true, bucketingContext.listenerDescription, leftTable, result);
- final JoinListenerRecorder rightRecorder =
- new JoinListenerRecorder(false, bucketingContext.listenerDescription, rightTable, result);
+ final JoinListenerRecorder leftRecorder =
+ new JoinListenerRecorder(true, bc.listenerDescription, leftTable, result);
+ final JoinListenerRecorder rightRecorder =
+ new JoinListenerRecorder(false, bc.listenerDescription, rightTable, result);
- final ChunkedMergedJoinListener mergedJoinListener = new ChunkedMergedJoinListener(
- leftTable, rightTable, bucketingContext.leftSources, bucketingContext.rightSources,
- columnsToMatch,
- columnsToAdd, leftRecorder, rightRecorder, result, rowRedirection, jsm, exactMatch,
- bucketingContext.listenerDescription);
- leftRecorder.setMergedListener(mergedJoinListener);
- rightRecorder.setMergedListener(mergedJoinListener);
+ final ChunkedMergedJoinListener mergedJoinListener = new ChunkedMergedJoinListener(
+ leftTable, rightTable, bc.leftSources, bc.rightSources, columnsToMatch, columnsToAdd,
+ leftRecorder, rightRecorder, result, rowRedirection, jsm, exactMatch, bc.listenerDescription);
+ leftRecorder.setMergedListener(mergedJoinListener);
+ rightRecorder.setMergedListener(mergedJoinListener);
- leftTable.addUpdateListener(leftRecorder);
- rightTable.addUpdateListener(rightRecorder);
+ leftTable.addUpdateListener(leftRecorder);
+ rightTable.addUpdateListener(rightRecorder);
- result.addParentReference(mergedJoinListener);
+ result.addParentReference(mergedJoinListener);
- return result;
+ return result;
+ }
+
+ if (leftTable.isRefreshing()) {
+ Assert.eq(firstBuildFrom, "firstBuildFrom", RightInput);
+
+ final LongArraySource leftRedirections = new LongArraySource();
+ final StaticHashedNaturalJoinStateManager jsm = TypedHasherFactory.make(
+ StaticNaturalJoinStateManagerTypedBase.class, bc.leftSources, bc.originalLeftSources,
+ initialHashTableSize, control.getMaximumLoadFactor(),
+ control.getTargetLoadFactor());
+
+ jsm.buildFromRightSide(rightTable, bc.rightSources);
+ if (bc.leftDataIndexTable != null) {
+ jsm.decorateLeftSide(bc.leftDataIndexTable.getRowSet(), bc.leftDataIndexSources, leftRedirections);
+ rowRedirection = jsm.buildIndexedRowRedirectionFromRedirections(leftTable, exactMatch,
+ bc.leftDataIndexTable.getRowSet(), leftRedirections, bc.leftDataIndexRowSetSource,
+ control.getRedirectionType(leftTable));
} else {
- // right is live, left is static
- final RightIncrementalNaturalJoinStateManager jsm =
- TypedHasherFactory.make(RightIncrementalNaturalJoinStateManagerTypedBase.class,
- bucketingContext.leftSources, bucketingContext.originalLeftSources,
- control.tableSizeForLeftBuild(leftTable),
- control.getMaximumLoadFactor(), control.getTargetLoadFactor());
- RightIncrementalNaturalJoinStateManager.InitialBuildContext initialBuildContext =
- jsm.makeInitialBuildContext(leftTable);
-
- final ObjectArraySource rowSetSource;
- final MutableInt groupingSize = new MutableInt();
- if (bucketingContext.useLeftGrouping) {
- final Map, RowSet> grouping =
- bucketingContext.leftSources[0].getGroupToRange(leftTable.getRowSet());
-
- // noinspection unchecked,rawtypes
- final Pair, ObjectArraySource> flatResultColumnSources =
- GroupingUtils.groupingToFlatSources(
- (ColumnSource) bucketingContext.leftSources[0], grouping, leftTable.getRowSet(),
- groupingSize);
- final ArrayBackedColumnSource> groupSource = flatResultColumnSources.getFirst();
- rowSetSource = flatResultColumnSources.getSecond();
-
- final Table leftTableGrouped = new QueryTable(
- RowSetFactory.flat(groupingSize.intValue()).toTracking(),
- Collections.singletonMap(columnsToMatch[0].leftColumn(), groupSource));
-
- final ColumnSource>[] groupedSourceArray = {groupSource};
- jsm.buildFromLeftSide(leftTableGrouped, groupedSourceArray, initialBuildContext);
- jsm.convertLeftGroups(groupingSize.intValue(), initialBuildContext, rowSetSource);
- } else {
- jsm.buildFromLeftSide(leftTable, bucketingContext.leftSources, initialBuildContext);
- rowSetSource = null;
- }
+ jsm.decorateLeftSide(leftTable.getRowSet(), bc.leftSources, leftRedirections);
+ rowRedirection = jsm.buildRowRedirectionFromRedirections(leftTable, exactMatch, leftRedirections,
+ control.getRedirectionType(leftTable));
+ }
- jsm.addRightSide(rightTable.getRowSet(), bucketingContext.rightSources);
+ final QueryTable result = makeResult(leftTable, rightTable, columnsToAdd, rowRedirection, true);
- if (bucketingContext.useLeftGrouping) {
- rowRedirection = jsm.buildRowRedirectionFromHashSlotGrouped(leftTable, rowSetSource,
- groupingSize.intValue(), exactMatch, initialBuildContext,
- control.getRedirectionType(leftTable));
- } else {
- rowRedirection = jsm.buildRowRedirectionFromHashSlot(leftTable, exactMatch, initialBuildContext,
- control.getRedirectionType(leftTable));
- }
+ leftTable.addUpdateListener(
+ new LeftTickingListener(
+ bc.listenerDescription,
+ columnsToMatch,
+ columnsToAdd,
+ leftTable,
+ result,
+ rowRedirection,
+ jsm,
+ bc.leftSources));
+ return result;
+ }
- final QueryTable result = makeResult(leftTable, rightTable, columnsToAdd, rowRedirection, true);
-
- rightTable.addUpdateListener(
- new RightTickingListener(
- bucketingContext.listenerDescription,
- rightTable,
- columnsToMatch,
- columnsToAdd,
- result,
- rowRedirection,
- jsm,
- bucketingContext.rightSources,
- exactMatch));
- return result;
+ if (rightTable.isRefreshing()) {
+ Assert.assertion(firstBuildFrom == LeftInput || firstBuildFrom == LeftDataIndex,
+ "firstBuildFrom == LeftInput || firstBuildFrom == LeftDataIndex");
+
+ final RightIncrementalNaturalJoinStateManager jsm = TypedHasherFactory.make(
+ RightIncrementalNaturalJoinStateManagerTypedBase.class, bc.leftSources, bc.originalLeftSources,
+ initialHashTableSize, control.getMaximumLoadFactor(),
+ control.getTargetLoadFactor());
+ final RightIncrementalNaturalJoinStateManager.InitialBuildContext ibc =
+ jsm.makeInitialBuildContext(leftTable);
+
+ if (firstBuildFrom == LeftDataIndex) {
+ Assert.neqNull(bc.leftDataIndexTable, "leftDataIndexTable");
+ jsm.buildFromLeftSide(bc.leftDataIndexTable, bc.leftDataIndexSources, ibc);
+ jsm.convertLeftDataIndex(bc.leftDataIndexTable.intSize(), ibc, bc.leftDataIndexRowSetSource);
+ } else {
+ jsm.buildFromLeftSide(leftTable, bc.leftSources, ibc);
}
- } else {
- if (bucketingContext.useLeftGrouping) {
- if (leftTable.isRefreshing()) {
- throw new UnsupportedOperationException(
- "Grouping information is not supported when tables are refreshing!");
- }
- final Map, RowSet> grouping =
- bucketingContext.leftSources[0].getGroupToRange(leftTable.getRowSet());
-
- final MutableInt groupingSize = new MutableInt();
- // noinspection unchecked,rawtypes
- final Pair, ObjectArraySource> flatResultColumnSources =
- GroupingUtils.groupingToFlatSources((ColumnSource) bucketingContext.leftSources[0],
- grouping, leftTable.getRowSet(), groupingSize);
- final ArrayBackedColumnSource> groupSource = flatResultColumnSources.getFirst();
- final ObjectArraySource rowSetSource = flatResultColumnSources.getSecond();
-
- final Table leftTableGrouped = new QueryTable(
- RowSetFactory.flat(groupingSize.intValue()).toTracking(),
- Collections.singletonMap(columnsToMatch[0].leftColumn(), groupSource));
-
- final ColumnSource>[] groupedSourceArray = {groupSource};
- final StaticHashedNaturalJoinStateManager jsm =
- TypedHasherFactory.make(StaticNaturalJoinStateManagerTypedBase.class, groupedSourceArray,
- groupedSourceArray,
- control.tableSize(groupingSize.intValue()),
- control.getMaximumLoadFactor(), control.getTargetLoadFactor());
- final IntegerArraySource leftHashSlots = new IntegerArraySource();
- jsm.buildFromLeftSide(leftTableGrouped, groupedSourceArray, leftHashSlots);
- try {
- jsm.decorateWithRightSide(rightTable, bucketingContext.rightSources);
- } catch (DuplicateRightRowDecorationException e) {
- jsm.errorOnDuplicatesGrouped(leftHashSlots, leftTableGrouped.size(), rowSetSource);
- }
- rowRedirection = jsm.buildGroupedRowRedirection(leftTable, exactMatch, leftTableGrouped.size(),
- leftHashSlots, rowSetSource, control.getRedirectionType(leftTable));
- } else if (control.buildLeft(leftTable, rightTable)) {
- final StaticHashedNaturalJoinStateManager jsm =
- TypedHasherFactory.make(StaticNaturalJoinStateManagerTypedBase.class,
- bucketingContext.leftSources, bucketingContext.originalLeftSources,
- control.tableSizeForLeftBuild(leftTable),
- control.getMaximumLoadFactor(), control.getTargetLoadFactor());
- final IntegerArraySource leftHashSlots = new IntegerArraySource();
- jsm.buildFromLeftSide(leftTable, bucketingContext.leftSources, leftHashSlots);
- try {
- jsm.decorateWithRightSide(rightTable, bucketingContext.rightSources);
- } catch (DuplicateRightRowDecorationException e) {
- jsm.errorOnDuplicatesSingle(leftHashSlots, leftTable.size(), leftTable.getRowSet());
- }
- rowRedirection = jsm.buildRowRedirectionFromHashSlot(leftTable, exactMatch, leftHashSlots,
- control.getRedirectionType(leftTable));
+ jsm.addRightSide(rightTable.getRowSet(), bc.rightSources);
+
+ if (firstBuildFrom == LeftDataIndex) {
+ rowRedirection = jsm.buildRowRedirectionFromHashSlotIndexed(leftTable,
+ bc.leftDataIndexRowSetSource, bc.leftDataIndexTable.intSize(),
+ exactMatch, ibc, control.getRedirectionType(leftTable));
} else {
- final LongArraySource leftRedirections = new LongArraySource();
- final StaticHashedNaturalJoinStateManager jsm =
- TypedHasherFactory.make(StaticNaturalJoinStateManagerTypedBase.class,
- bucketingContext.leftSources, bucketingContext.originalLeftSources,
- control.tableSizeForRightBuild(rightTable),
- control.getMaximumLoadFactor(), control.getTargetLoadFactor());
- jsm.buildFromRightSide(rightTable, bucketingContext.rightSources);
- jsm.decorateLeftSide(leftTable.getRowSet(), bucketingContext.leftSources, leftRedirections);
- rowRedirection = jsm.buildRowRedirectionFromRedirections(leftTable, exactMatch, leftRedirections,
+ rowRedirection = jsm.buildRowRedirectionFromHashSlot(leftTable, exactMatch, ibc,
control.getRedirectionType(leftTable));
-
- final QueryTable result = makeResult(leftTable, rightTable, columnsToAdd, rowRedirection, true);
-
- leftTable.addUpdateListener(
- new LeftTickingListener(
- bucketingContext.listenerDescription,
- columnsToMatch,
- columnsToAdd,
- leftTable,
- result,
- rowRedirection,
- jsm,
- bucketingContext.leftSources));
- return result;
}
+
+ final QueryTable result = makeResult(leftTable, rightTable, columnsToAdd, rowRedirection, true);
+
+ rightTable.addUpdateListener(
+ new RightTickingListener(
+ bc.listenerDescription,
+ rightTable,
+ columnsToMatch,
+ columnsToAdd,
+ result,
+ rowRedirection,
+ jsm,
+ bc.rightSources,
+ exactMatch));
+ return result;
}
+ if (firstBuildFrom == LeftDataIndex) {
+ Assert.neqNull(bc.leftDataIndexTable, "leftDataIndexTable");
+ final StaticHashedNaturalJoinStateManager jsm = TypedHasherFactory.make(
+ StaticNaturalJoinStateManagerTypedBase.class, bc.leftDataIndexSources,
+ bc.originalLeftDataIndexSources, initialHashTableSize,
+ control.getMaximumLoadFactor(), control.getTargetLoadFactor());
+
+ final IntegerArraySource leftHashSlots = new IntegerArraySource();
+ jsm.buildFromLeftSide(bc.leftDataIndexTable, bc.leftDataIndexSources,
+ leftHashSlots);
+ jsm.decorateWithRightSide(rightTable, bc.rightSources);
+ rowRedirection = jsm.buildIndexedRowRedirectionFromHashSlots(leftTable, exactMatch,
+ bc.leftDataIndexTable.getRowSet(), leftHashSlots,
+ bc.leftDataIndexRowSetSource, control.getRedirectionType(leftTable));
+ } else if (firstBuildFrom == LeftInput) {
+ final StaticHashedNaturalJoinStateManager jsm = TypedHasherFactory.make(
+ StaticNaturalJoinStateManagerTypedBase.class, bc.leftSources, bc.originalLeftSources,
+ // The static state manager doesn't allow rehashing, so we must allocate a big enough hash
+ // table for the possibility that all left rows will have unique keys.
+ control.tableSize(leftTable.size()),
+ control.getMaximumLoadFactor(), control.getTargetLoadFactor());
+ final IntegerArraySource leftHashSlots = new IntegerArraySource();
+ jsm.buildFromLeftSide(leftTable, bc.leftSources, leftHashSlots);
+ try {
+ jsm.decorateWithRightSide(rightTable, bc.rightSources);
+ } catch (DuplicateRightRowDecorationException e) {
+ jsm.errorOnDuplicatesSingle(leftHashSlots, leftTable.size(), leftTable.getRowSet());
+ }
+ rowRedirection = jsm.buildRowRedirectionFromHashSlot(leftTable, exactMatch, leftHashSlots,
+ control.getRedirectionType(leftTable));
+ } else {
+ final LongArraySource leftRedirections = new LongArraySource();
+ final StaticHashedNaturalJoinStateManager jsm = TypedHasherFactory.make(
+ StaticNaturalJoinStateManagerTypedBase.class, bc.leftSources, bc.originalLeftSources,
+ initialHashTableSize, control.getMaximumLoadFactor(),
+ control.getTargetLoadFactor());
+
+ jsm.buildFromRightSide(rightTable, bc.rightSources);
+ jsm.decorateLeftSide(leftTable.getRowSet(), bc.leftSources, leftRedirections);
+ rowRedirection = jsm.buildRowRedirectionFromRedirections(leftTable, exactMatch, leftRedirections,
+ control.getRedirectionType(leftTable));
+ }
return makeResult(leftTable, rightTable, columnsToAdd, rowRedirection, false);
}
}
@@ -293,7 +271,7 @@ private static QueryTable zeroKeyColumnsJoin(QueryTable leftTable, QueryTable ri
final boolean rightRefreshing = rightTable.isRefreshing();
if (rightTable.size() > 1) {
- if (leftTable.size() > 0) {
+ if (!leftTable.isEmpty()) {
throw new RuntimeException(
"naturalJoin with zero key columns may not have more than one row in the right hand side table!");
}
@@ -302,7 +280,7 @@ private static QueryTable zeroKeyColumnsJoin(QueryTable leftTable, QueryTable ri
} else if (rightTable.size() == 1) {
rowRedirection = getSingleValueRowRedirection(rightRefreshing, rightTable.getRowSet().firstRowKey());
} else {
- if (exactMatch && leftTable.size() > 0) {
+ if (exactMatch && !leftTable.isEmpty()) {
throw new RuntimeException(
"exactJoin with zero key columns must have exactly one row in the right hand side table!");
}
@@ -385,7 +363,7 @@ public void onUpdate(final TableUpdate upstream) {
});
}
} else if (rightTable.isRefreshing()) {
- if (leftTable.size() > 0) {
+ if (!leftTable.isEmpty()) {
rightTable.addUpdateListener(
new BaseTable.ListenerImpl(listenerDescription, rightTable, result) {
@Override
@@ -415,7 +393,7 @@ private static SingleValueRowRedirection getSingleValueRowRedirection(boolean re
private static boolean updateRightRedirection(QueryTable rightTable, SingleValueRowRedirection rowRedirection) {
final boolean changed;
- if (rightTable.size() == 0) {
+ if (rightTable.isEmpty()) {
changed = rowRedirection.getValue() != RowSequence.NULL_ROW_KEY;
if (changed) {
rowRedirection.writableSingleValueCast().setValue(RowSequence.NULL_ROW_KEY);
@@ -431,11 +409,11 @@ private static boolean updateRightRedirection(QueryTable rightTable, SingleValue
}
private static void checkRightTableSizeZeroKeys(final Table leftTable, final Table rightTable, boolean exactMatch) {
- if (leftTable.size() != 0) {
+ if (!leftTable.isEmpty()) {
if (rightTable.size() > 1) {
throw new RuntimeException(
"naturalJoin with zero key columns may not have more than one row in the right hand side table!");
- } else if (rightTable.size() == 0 && exactMatch) {
+ } else if (rightTable.isEmpty() && exactMatch) {
throw new RuntimeException(
"exactJoin with zero key columns must have exactly one row in the right hand side table!");
}
@@ -582,8 +560,7 @@ public void onUpdate(final TableUpdate upstream) {
final boolean addedRightColumnsChanged;
- final int maxSize =
- UpdateSizeCalculator.chunkSize(upstream, JoinControl.CHUNK_SIZE);
+ final int maxSize = UpdateSizeCalculator.chunkSize(upstream, JoinControl.CHUNK_SIZE);
if (maxSize == 0) {
Assert.assertion(upstream.empty(), "upstream.empty()");
return;
@@ -601,23 +578,18 @@ public void onUpdate(final TableUpdate upstream) {
}
if (upstream.shifted().nonempty()) {
- final RowSet previousToShift;
-
- if (rightKeysChanged) {
- previousToShift =
- getParent().getRowSet().copyPrev().minus(modifiedPreShift)
- .minus(upstream.removed());
- } else {
- previousToShift = getParent().getRowSet().copyPrev().minus(upstream.removed());
- }
-
- final RowSetShiftData.Iterator sit = upstream.shifted().applyIterator();
- while (sit.hasNext()) {
- sit.next();
- final RowSet shiftedRowSet =
- previousToShift.subSetByKeyRange(sit.beginRange(), sit.endRange())
- .shift(sit.shiftDelta());
- jsm.applyRightShift(pc, rightSources, shiftedRowSet, sit.shiftDelta(), modifiedSlotTracker);
+ try (final WritableRowSet previousToShift =
+ getParent().getRowSet().prev().minus(upstream.removed())) {
+ if (rightKeysChanged) {
+ previousToShift.remove(modifiedPreShift);
+ }
+ upstream.shifted().apply((long beginRange, long endRange, long shiftDelta) -> {
+ try (final WritableRowSet shiftedRowSet =
+ previousToShift.subSetByKeyRange(beginRange, endRange)) {
+ shiftedRowSet.shiftInPlace(shiftDelta);
+ jsm.applyRightShift(pc, rightSources, shiftedRowSet, shiftDelta, modifiedSlotTracker);
+ }
+ });
}
}
@@ -827,22 +799,19 @@ protected void process() {
}
if (rightShifted.nonempty()) {
- final WritableRowSet previousToShift = rightRecorder.getParent().getRowSet().copyPrev();
- previousToShift.remove(rightRemoved);
-
- if (rightKeysModified) {
- previousToShift.remove(modifiedPreShift);
- }
-
- final RowSetShiftData.Iterator sit = rightShifted.applyIterator();
- while (sit.hasNext()) {
- sit.next();
- try (final WritableRowSet shiftedRowSet =
- previousToShift.subSetByKeyRange(sit.beginRange(), sit.endRange())) {
- shiftedRowSet.shiftInPlace(sit.shiftDelta());
- jsm.applyRightShift(pc, rightSources, shiftedRowSet, sit.shiftDelta(),
- modifiedSlotTracker);
+ try (final WritableRowSet previousToShift =
+ rightRecorder.getParent().getRowSet().prev().minus(rightRemoved)) {
+ if (rightKeysModified) {
+ previousToShift.remove(modifiedPreShift);
}
+ rightShifted.apply((long beginRange, long endRange, long shiftDelta) -> {
+ try (final WritableRowSet shiftedRowSet =
+ previousToShift.subSetByKeyRange(beginRange, endRange)) {
+ shiftedRowSet.shiftInPlace(shiftDelta);
+ jsm.applyRightShift(pc, rightSources, shiftedRowSet, shiftDelta,
+ modifiedSlotTracker);
+ }
+ });
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/NotificationStepReceiver.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/NotificationStepReceiver.java
index 1a4a3e50858..aa8895f0c17 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/NotificationStepReceiver.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/NotificationStepReceiver.java
@@ -3,6 +3,10 @@
//
package io.deephaven.engine.table.impl;
+import io.deephaven.engine.updategraph.LogicalClock;
+import io.deephaven.util.annotations.FinalDefault;
+import org.jetbrains.annotations.NotNull;
+
/**
* Used by {@link OperationSnapshotControl} to set the notification step of elements in our DAG.
*/
@@ -19,4 +23,16 @@ public interface NotificationStepReceiver {
* @param lastNotificationStep The last notification step to be delivered
*/
void setLastNotificationStep(long lastNotificationStep);
+
+ /**
+ * Deliver the appropriate last notification step to a receiver that isn't derived from a
+ * {@link NotificationStepSource}.
+ */
+ @FinalDefault
+ default void initializeLastNotificationStep(@NotNull final LogicalClock clock) {
+ final long currentClockValue = clock.currentValue();
+ setLastNotificationStep(LogicalClock.getState(currentClockValue) == LogicalClock.State.Updating
+ ? LogicalClock.getStep(currentClockValue) - 1
+ : LogicalClock.getStep(currentClockValue));
+ }
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/OperationSnapshotControl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/OperationSnapshotControl.java
index acece35da71..bccb06754c8 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/OperationSnapshotControl.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/OperationSnapshotControl.java
@@ -119,7 +119,7 @@ public synchronized boolean snapshotCompletedConsistently(
.append("} snapshotCompletedConsistently: afterClockValue=").append(afterClockValue)
.append(", usedPreviousValues=").append(usedPreviousValues)
.append(", snapshotConsistent=").append(snapshotConsistent)
- .append(", last=").append(lastNotificationStep)
+ .append(", lastNotificationStep=").append(lastNotificationStep)
.endl();
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/OperationSnapshotControlEx.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/OperationSnapshotControlEx.java
index 04f6c2c8160..875a13515c0 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/OperationSnapshotControlEx.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/OperationSnapshotControlEx.java
@@ -3,118 +3,119 @@
//
package io.deephaven.engine.table.impl;
-import io.deephaven.engine.context.ExecutionContext;
-import io.deephaven.engine.table.TableUpdateListener;
import io.deephaven.engine.updategraph.ClockInconsistencyException;
import io.deephaven.engine.updategraph.LogicalClock;
+import io.deephaven.engine.updategraph.NotificationQueue;
import io.deephaven.engine.updategraph.WaitNotification;
import io.deephaven.internal.log.LoggerFactory;
import io.deephaven.io.logger.Logger;
import org.jetbrains.annotations.NotNull;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static io.deephaven.engine.updategraph.LogicalClock.NULL_CLOCK_VALUE;
+
/**
- * Variant of {@link OperationSnapshotControl} that considers an "extra" {@link NotificationStepSource} in addition to
- * the source {@link BaseTable} when determining whether to use previous values during initialization or evaluating
- * success. This is useful anytime an operation needs to listen to and snapshot one data source while also snapshotting
- * another.
+ * Variant of {@link OperationSnapshotControl} that considers "extra" {@link NotificationQueue.Dependency dependencies}
+ * in addition to the source {@link BaseTable} when determining whether to use previous values during initialization or
+ * evaluating success. This is useful anytime an operation needs to listen to and snapshot one data source while also
+ * snapshotting others.
*/
public final class OperationSnapshotControlEx extends OperationSnapshotControl {
private static final Logger log = LoggerFactory.getLogger(OperationSnapshotControlEx.class);
- private final NotificationStepSource extra;
-
- private long extraLastNotificationStep;
+ private final NotificationQueue.Dependency[] extras;
public OperationSnapshotControlEx(
@NotNull final BaseTable> sourceTable,
- @NotNull final NotificationStepSource extra) {
+ @NotNull final NotificationQueue.Dependency... extras) {
super(sourceTable);
- this.extra = extra;
+ this.extras = extras;
}
@Override
@SuppressWarnings("AutoBoxing")
public synchronized Boolean usePreviousValues(final long beforeClockValue) {
lastNotificationStep = sourceTable.getLastNotificationStep();
- extraLastNotificationStep = extra.getLastNotificationStep();
final long beforeStep = LogicalClock.getStep(beforeClockValue);
final LogicalClock.State beforeState = LogicalClock.getState(beforeClockValue);
- final boolean idle = beforeState == LogicalClock.State.Idle;
- final boolean sourceUpdatedOnThisStep = lastNotificationStep == beforeStep;
- final boolean sourceSatisfied;
- final boolean extraUpdatedOnThisStep = extraLastNotificationStep == beforeStep;
- final boolean extraSatisfied;
+ if (beforeState == LogicalClock.State.Idle) {
+ if (DEBUG) {
+ log.info().append("OperationSnapshotControlEx {source=").append(System.identityHashCode(sourceTable))
+ .append(", extras=").append(Arrays.stream(extras)
+ .mapToInt(System::identityHashCode)
+ .mapToObj(Integer::toString)
+ .collect(Collectors.joining(", ", "[", "]")))
+ .append("} usePreviousValues: beforeStep=").append(beforeStep)
+ .append(", beforeState=").append(beforeState.name())
+ .append(", sourceLastNotificationStep=").append(lastNotificationStep)
+ .append(", usePrev=").append(false)
+ .endl();
+ }
+ return false;
+ }
+ final NotificationQueue.Dependency[] notYetSatisfied;
try {
- sourceSatisfied = idle || sourceUpdatedOnThisStep || sourceTable.satisfied(beforeStep);
- extraSatisfied = idle || extraUpdatedOnThisStep || extra.satisfied(beforeStep);
+ notYetSatisfied = Stream.concat(Stream.of(sourceTable), Arrays.stream(extras))
+ .sequential()
+ .filter(dependency -> !satisfied(dependency, beforeStep))
+ .toArray(NotificationQueue.Dependency[]::new);
} catch (ClockInconsistencyException e) {
return null;
}
+ final long postWaitStep;
final Boolean usePrev;
- if (sourceSatisfied == extraSatisfied) {
- usePrev = !sourceSatisfied;
- } else if (sourceSatisfied) {
- WaitNotification.waitForSatisfaction(beforeStep, extra);
- extraLastNotificationStep = extra.getLastNotificationStep();
- final long postWaitStep = ExecutionContext.getContext().getUpdateGraph().clock().currentStep();
- usePrev = postWaitStep == beforeStep ? false : null;
+ if (notYetSatisfied.length == extras.length + 1) {
+ // Nothing satisfied
+ postWaitStep = NULL_CLOCK_VALUE;
+ usePrev = true;
+ } else if (notYetSatisfied.length > 0) {
+ // Partially satisfied
+ if (WaitNotification.waitForSatisfaction(beforeStep, notYetSatisfied)) {
+ // Successful wait on beforeStep
+ postWaitStep = beforeStep;
+ usePrev = false;
+ } else {
+ // Updating phase finished before we could wait; use current if we're in the subsequent idle phase
+ postWaitStep = getUpdateGraph().clock().currentStep();
+ usePrev = postWaitStep == beforeStep ? false : null;
+ }
} else {
- WaitNotification.waitForSatisfaction(beforeStep, sourceTable);
- lastNotificationStep = sourceTable.getLastNotificationStep();
- final long postWaitStep = ExecutionContext.getContext().getUpdateGraph().clock().currentStep();
- usePrev = postWaitStep == beforeStep ? false : null;
+ // All satisfied
+ postWaitStep = NULL_CLOCK_VALUE;
+ usePrev = false;
}
if (DEBUG) {
log.info().append("OperationSnapshotControlEx {source=").append(System.identityHashCode(sourceTable))
- .append(", extra=").append(System.identityHashCode(extra))
+ .append(", extras=").append(Arrays.stream(extras)
+ .mapToInt(System::identityHashCode)
+ .mapToObj(Integer::toString)
+ .collect(Collectors.joining(", ", "[", "]")))
.append(", control=").append(System.identityHashCode(this))
.append("} usePreviousValues: beforeStep=").append(beforeStep)
.append(", beforeState=").append(beforeState.name())
.append(", sourceLastNotificationStep=").append(lastNotificationStep)
- .append(", sourceSatisfied=").append(sourceSatisfied)
- .append(", extraLastNotificationStep=").append(extraLastNotificationStep)
- .append(", extraSatisfied=").append(extraSatisfied)
+ .append(", notYetSatisfied=").append(Arrays.toString(notYetSatisfied))
+ .append(", postWaitStep=").append(postWaitStep)
.append(", usePrev=").append(usePrev)
.endl();
}
return usePrev;
}
- @Override
- public synchronized boolean snapshotCompletedConsistently(long afterClockValue, boolean usedPreviousValues) {
- if (DEBUG) {
- log.info().append("OperationSnapshotControlEx snapshotCompletedConsistently: control=")
- .append(System.identityHashCode(this))
- .append(", end={").append(LogicalClock.getStep(afterClockValue)).append(",")
- .append(LogicalClock.getState(afterClockValue).toString())
- .append("}, usedPreviousValues=").append(usedPreviousValues)
- .append(", last=").append(sourceTable.getLastNotificationStep())
- .append(", extraLast=").append(extra.getLastNotificationStep())
- .endl();
+ private static boolean satisfied(@NotNull final NotificationQueue.Dependency dependency, final long step) {
+ if (dependency instanceof NotificationStepSource
+ && ((NotificationStepSource) dependency).getLastNotificationStep() == step) {
+ return true;
}
- return extra.getLastNotificationStep() == extraLastNotificationStep
- && super.snapshotCompletedConsistently(afterClockValue, usedPreviousValues);
- }
-
- @Override
- public synchronized void setListenerAndResult(final TableUpdateListener listener,
- @NotNull final NotificationStepReceiver resultTable) {
- super.setListenerAndResult(listener, resultTable);
- if (DEBUG) {
- log.info().append("OperationSnapshotControlEx control=")
- .append(System.identityHashCode(OperationSnapshotControlEx.this))
- .append(", result=").append(System.identityHashCode(resultTable)).endl();
- }
- }
-
- @Override
- protected boolean isInInitialNotificationWindow() {
- return extra.getLastNotificationStep() == extraLastNotificationStep && super.isInInitialNotificationWindow();
+ return dependency.satisfied(step);
}
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java
index 544c3ccb25a..8409f97616c 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/PartitionAwareSourceTable.java
@@ -3,7 +3,6 @@
//
package io.deephaven.engine.table.impl;
-import io.deephaven.api.ColumnName;
import io.deephaven.api.Selectable;
import io.deephaven.api.filter.Filter;
import io.deephaven.base.verify.Assert;
@@ -12,7 +11,6 @@
import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder;
import io.deephaven.engine.util.TableTools;
import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey;
-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.select.*;
@@ -81,17 +79,17 @@ protected PartitionAwareSourceTable newInstance(@NotNull final TableDefinition t
}
private PartitionAwareSourceTable getFilteredTable(
- @NotNull final WhereFilter... additionalPartitioningColumnFilters) {
- WhereFilter[] resultPartitioningColumnFilters =
- new WhereFilter[partitioningColumnFilters.length + additionalPartitioningColumnFilters.length];
- System.arraycopy(partitioningColumnFilters, 0, resultPartitioningColumnFilters, 0,
- partitioningColumnFilters.length);
- System.arraycopy(additionalPartitioningColumnFilters, 0, resultPartitioningColumnFilters,
- partitioningColumnFilters.length, additionalPartitioningColumnFilters.length);
- return newInstance(definition,
- description + ".where(" + Arrays.deepToString(additionalPartitioningColumnFilters) + ')',
+ @NotNull final List additionalPartitioningColumnFilters) {
+ final WhereFilter[] resultPartitioningColumnFilters = Stream.concat(
+ Arrays.stream(partitioningColumnFilters),
+ additionalPartitioningColumnFilters.stream())
+ .toArray(WhereFilter[]::new);
+ final PartitionAwareSourceTable filtered = newInstance(definition,
+ description + ".where(" + additionalPartitioningColumnFilters + ')',
componentFactory, locationProvider, updateSourceRegistrar, partitioningColumnDefinitions,
resultPartitioningColumnFilters);
+ copyAttributes(filtered, CopyAttributeOperation.Filter);
+ return filtered;
}
private static Map> extractPartitioningColumnDefinitions(
@@ -110,42 +108,24 @@ private PartitionAwareQueryTableReference(PartitionAwareSourceTable table) {
@Override
protected TableAndRemainingFilters getWithWhere(WhereFilter... whereFilters) {
- ArrayList partitionFilters = new ArrayList<>();
- ArrayList groupFilters = new ArrayList<>();
- ArrayList otherFilters = new ArrayList<>();
-
- List> groupingColumns = table.getDefinition().getGroupingColumns();
- Set groupingColumnNames =
- groupingColumns.stream().map(ColumnDefinition::getName).collect(Collectors.toSet());
-
- for (WhereFilter filter : whereFilters) {
- // note: our filters are already initialized
- List columns = filter.getColumns();
- if (filter instanceof ReindexingFilter) {
- otherFilters.add(filter);
- } else if (((PartitionAwareSourceTable) table).isValidAgainstColumnPartitionTable(columns,
- filter.getColumnArrays())) {
- partitionFilters.add(filter);
- } else if (filter.isSimpleFilter() && (columns.size() == 1)
- && (groupingColumnNames.contains(columns.get(0)))) {
- groupFilters.add(filter);
+ final List partitionFilters = new ArrayList<>();
+ final List deferredFilters = new ArrayList<>();
+ for (WhereFilter whereFilter : whereFilters) {
+ if (!(whereFilter instanceof ReindexingFilter)
+ && ((PartitionAwareSourceTable) table).isValidAgainstColumnPartitionTable(
+ whereFilter.getColumns(), whereFilter.getColumnArrays())) {
+ partitionFilters.add(whereFilter);
} else {
- otherFilters.add(filter);
+ deferredFilters.add(whereFilter);
}
}
- final Table result = partitionFilters.isEmpty() ? table.coalesce()
+ final Table result = partitionFilters.isEmpty()
+ ? table.coalesce()
: table.where(Filter.and(partitionFilters));
- // put the other filters onto the end of the grouping filters, this means that the group filters should
- // go first, which should be preferable to having them second. This is basically the first query
- // optimization that we're doing for the user, so maybe it is a good thing but maybe not. The reason we do
- // it, is that we have deferred the filters for the users permissions, and they did not have the opportunity
- // to properly filter the data yet at this point.
- groupFilters.addAll(otherFilters);
-
return new TableAndRemainingFilters(result,
- groupFilters.toArray(WhereFilter.ZERO_LENGTH_SELECT_FILTER_ARRAY));
+ deferredFilters.toArray(WhereFilter.ZERO_LENGTH_SELECT_FILTER_ARRAY));
}
@Override
@@ -157,8 +137,8 @@ public Table selectDistinctInternal(Collection extends Selectable> columns) {
} catch (Exception e) {
return null;
}
- if (!((PartitionAwareSourceTable) table).isValidAgainstColumnPartitionTable(selectColumn.getColumns(),
- selectColumn.getColumnArrays())) {
+ if (!((PartitionAwareSourceTable) table).isValidAgainstColumnPartitionTable(
+ selectColumn.getColumns(), selectColumn.getColumnArrays())) {
return null;
}
}
@@ -176,7 +156,7 @@ protected PartitionAwareSourceTable copy() {
}
@Override
- protected final BaseTable redefine(@NotNull final TableDefinition newDefinition) {
+ protected final BaseTable> redefine(@NotNull final TableDefinition newDefinition) {
if (newDefinition.getColumnNames().equals(definition.getColumnNames())) {
// Nothing changed - we have the same columns in the same order.
return this;
@@ -270,60 +250,37 @@ public Table where(Filter filter) {
private Table whereImpl(final WhereFilter[] whereFilters) {
if (whereFilters.length == 0) {
- return QueryPerformanceRecorder.withNugget(description + ".coalesce()", this::coalesce);
+ return prepareReturnThis();
}
- ArrayList partitionFilters = new ArrayList<>();
- ArrayList groupFilters = new ArrayList<>();
- ArrayList otherFilters = new ArrayList<>();
-
- List> groupingColumns = definition.getGroupingColumns();
- Set groupingColumnNames =
- groupingColumns.stream().map(ColumnDefinition::getName).collect(Collectors.toSet());
+ final List partitionFilters = new ArrayList<>();
+ final List deferredFilters = new ArrayList<>();
for (WhereFilter whereFilter : whereFilters) {
whereFilter.init(definition);
- List columns = whereFilter.getColumns();
- if (whereFilter instanceof ReindexingFilter) {
- otherFilters.add(whereFilter);
- } else if (isValidAgainstColumnPartitionTable(columns, whereFilter.getColumnArrays())) {
+ if (!(whereFilter instanceof ReindexingFilter)
+ && isValidAgainstColumnPartitionTable(whereFilter.getColumns(), whereFilter.getColumnArrays())) {
partitionFilters.add(whereFilter);
- } else if (whereFilter.isSimpleFilter() && (columns.size() == 1)
- && (groupingColumnNames.contains(columns.get(0)))) {
- groupFilters.add(whereFilter);
} else {
- otherFilters.add(whereFilter);
+ deferredFilters.add(whereFilter);
}
}
- // if there was nothing that actually required the partition, defer the result.
- if (partitionFilters.isEmpty()) {
- return new DeferredViewTable(definition, description + "-withDeferredFilters",
- new PartitionAwareQueryTableReference(this), null, null, whereFilters);
- }
+ final PartitionAwareSourceTable withPartitionsFiltered = partitionFilters.isEmpty()
+ ? this
+ : QueryPerformanceRecorder.withNugget("getFilteredTable(" + partitionFilters + ")",
+ () -> getFilteredTable(partitionFilters));
- WhereFilter[] partitionFilterArray = partitionFilters.toArray(WhereFilter.ZERO_LENGTH_SELECT_FILTER_ARRAY);
- final String filteredTableDescription = "getFilteredTable(" + Arrays.toString(partitionFilterArray) + ")";
- SourceTable filteredTable = QueryPerformanceRecorder.withNugget(filteredTableDescription,
- () -> getFilteredTable(partitionFilterArray));
-
- copyAttributes(filteredTable, CopyAttributeOperation.Filter);
-
- // Apply the group filters before other filters.
- groupFilters.addAll(otherFilters);
-
- if (groupFilters.isEmpty()) {
- return QueryPerformanceRecorder.withNugget(description + filteredTableDescription + ".coalesce()",
- filteredTable::coalesce);
- }
-
- return QueryPerformanceRecorder.withNugget(description + ".coalesce().where(" + groupFilters + ")",
- () -> filteredTable.coalesce().where(Filter.and(groupFilters)));
+ return deferredFilters.isEmpty()
+ ? withPartitionsFiltered
+ : new DeferredViewTable(definition, withPartitionsFiltered.getDescription() + "-withDeferredFilters",
+ new PartitionAwareQueryTableReference(withPartitionsFiltered), null, null,
+ deferredFilters.toArray(WhereFilter.ZERO_LENGTH_SELECT_FILTER_ARRAY));
}
@Override
- public final Table selectDistinct(Collection extends Selectable> columns) {
+ public final Table selectDistinct(@NotNull final Collection extends Selectable> columns) {
final List selectColumns = Arrays.asList(SelectColumn.from(columns));
- for (SelectColumn selectColumn : selectColumns) {
+ for (final SelectColumn selectColumn : selectColumns) {
selectColumn.initDef(definition.getColumnNameMap());
if (!isValidAgainstColumnPartitionTable(selectColumn.getColumns(), selectColumn.getColumnArrays())) {
// Be sure to invoke the super-class version of this method, rather than the array-based one that
@@ -331,36 +288,20 @@ public final Table selectDistinct(Collection extends Selectable> columns) {
return super.selectDistinct(selectColumns);
}
}
- initializeAvailableLocations();
- final List existingLocationKeys =
- columnSourceManager.allLocations().stream().filter(tl -> {
- tl.refresh();
- final long size = tl.getSize();
- // noinspection ConditionCoveredByFurtherCondition
- return size != TableLocation.NULL_SIZE && size > 0;
- }).map(TableLocation::getKey).collect(Collectors.toList());
- final List partitionTableColumnNames = new ArrayList<>(partitioningColumnDefinitions.keySet());
- final List> partitionTableColumnSources = new ArrayList<>(partitioningColumnDefinitions.size());
- for (final ColumnDefinition> columnDefinition : partitioningColumnDefinitions.values()) {
- partitionTableColumnSources.add(makePartitionSource(columnDefinition, existingLocationKeys));
- }
- return TableTools
- .newTable(existingLocationKeys.size(), partitionTableColumnNames, partitionTableColumnSources)
- .selectDistinct(selectColumns);
- // TODO (https://github.com/deephaven/deephaven-core/issues/867): Refactor around a ticking partition table
- // TODO: Maybe just get rid of this implementation and coalesce? Partitioning columns are automatically grouped.
- // Needs lazy region allocation.
+
+ // Ensure that the location table is available and populated with non-null, non-empty locations.
+ initialize();
+
+ // Apply our selectDistinct() to the location table.
+ return columnSourceManager.locationTable().selectDistinct(selectColumns);
}
- private boolean isValidAgainstColumnPartitionTable(@NotNull final Collection columnNames,
+ private boolean isValidAgainstColumnPartitionTable(
+ @NotNull final Collection columnNames,
@NotNull final Collection columnArrayNames) {
- if (columnArrayNames.size() > 0) {
+ if (!columnArrayNames.isEmpty()) {
return false;
}
return columnNames.stream().allMatch(partitioningColumnDefinitions::containsKey);
}
-
- private boolean isValidAgainstColumnPartitionTable(Collection columns) {
- return columns.stream().map(ColumnName::name).allMatch(partitioningColumnDefinitions::containsKey);
- }
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java
index ad6115905c0..8e0a48fcd9f 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java
@@ -4,68 +4,72 @@
package io.deephaven.engine.table.impl;
import io.deephaven.api.*;
-import io.deephaven.api.agg.*;
+import io.deephaven.api.agg.Aggregation;
+import io.deephaven.api.agg.AggregationOptimizer;
+import io.deephaven.api.agg.AggregationOutputs;
+import io.deephaven.api.agg.Partition;
import io.deephaven.api.agg.spec.AggSpec;
import io.deephaven.api.agg.spec.AggSpecColumnReferences;
import io.deephaven.api.filter.Filter;
import io.deephaven.api.snapshot.SnapshotWhenOptions;
import io.deephaven.api.snapshot.SnapshotWhenOptions.Flag;
-import io.deephaven.api.updateby.UpdateByOperation;
import io.deephaven.api.updateby.UpdateByControl;
+import io.deephaven.api.updateby.UpdateByOperation;
import io.deephaven.base.verify.Assert;
import io.deephaven.base.verify.Require;
import io.deephaven.chunk.attributes.Values;
import io.deephaven.configuration.Configuration;
import io.deephaven.datastructures.util.CollectionUtil;
import io.deephaven.engine.context.ExecutionContext;
-import io.deephaven.engine.exceptions.TableInitializationException;
-import io.deephaven.engine.table.impl.util.*;
-import io.deephaven.engine.updategraph.UpdateGraph;
import io.deephaven.engine.exceptions.CancellationException;
+import io.deephaven.engine.exceptions.TableInitializationException;
+import io.deephaven.engine.liveness.Liveness;
import io.deephaven.engine.liveness.LivenessScope;
import io.deephaven.engine.primitive.iterator.*;
import io.deephaven.engine.rowset.*;
-import io.deephaven.engine.rowset.RowSetFactory;
import io.deephaven.engine.table.*;
import io.deephaven.engine.table.hierarchical.RollupTable;
import io.deephaven.engine.table.hierarchical.TreeTable;
+import io.deephaven.engine.table.impl.MemoizedOperationKey.SelectUpdateViewOrUpdateView.Flavor;
+import io.deephaven.engine.table.impl.by.*;
import io.deephaven.engine.table.impl.hierarchical.RollupTableImpl;
import io.deephaven.engine.table.impl.hierarchical.TreeTableImpl;
-import io.deephaven.engine.table.impl.indexer.RowSetIndexer;
+import io.deephaven.engine.table.impl.indexer.DataIndexer;
import io.deephaven.engine.table.impl.lang.QueryLanguageParser;
import io.deephaven.engine.table.impl.partitioned.PartitionedTableImpl;
import io.deephaven.engine.table.impl.perf.BasePerformanceEntry;
import io.deephaven.engine.table.impl.perf.QueryPerformanceNugget;
-import io.deephaven.engine.table.impl.rangejoin.RangeJoinOperation;
-import io.deephaven.engine.table.impl.updateby.UpdateBy;
-import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzerWrapper;
-import io.deephaven.engine.table.impl.sources.ring.RingTableTools;
-import io.deephaven.engine.table.iterators.*;
-import io.deephaven.engine.updategraph.DynamicNode;
-import io.deephaven.engine.util.*;
-import io.deephaven.engine.util.systemicmarking.SystemicObject;
-import io.deephaven.util.annotations.InternalUseOnly;
-import io.deephaven.util.annotations.ReferentialIntegrity;
-import io.deephaven.vector.Vector;
import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder;
-import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker;
-import io.deephaven.engine.liveness.Liveness;
-import io.deephaven.engine.table.impl.MemoizedOperationKey.SelectUpdateViewOrUpdateView.Flavor;
-import io.deephaven.engine.table.impl.by.*;
-import io.deephaven.engine.table.impl.locations.GroupingProvider;
+import io.deephaven.engine.table.impl.rangejoin.RangeJoinOperation;
import io.deephaven.engine.table.impl.remote.ConstructSnapshot;
import io.deephaven.engine.table.impl.select.*;
import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer;
+import io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzerWrapper;
import io.deephaven.engine.table.impl.snapshot.SnapshotIncrementalListener;
import io.deephaven.engine.table.impl.snapshot.SnapshotInternalListener;
import io.deephaven.engine.table.impl.snapshot.SnapshotUtils;
import io.deephaven.engine.table.impl.sources.*;
+import io.deephaven.engine.table.impl.sources.ring.RingTableTools;
import io.deephaven.engine.table.impl.sources.sparse.SparseConstants;
+import io.deephaven.engine.table.impl.updateby.UpdateBy;
+import io.deephaven.engine.table.impl.util.*;
+import io.deephaven.engine.table.iterators.*;
+import io.deephaven.engine.updategraph.DynamicNode;
+import io.deephaven.engine.updategraph.NotificationQueue;
+import io.deephaven.engine.updategraph.UpdateGraph;
+import io.deephaven.engine.util.IterableUtils;
+import io.deephaven.engine.util.TableTools;
+import io.deephaven.engine.util.systemicmarking.SystemicObject;
+import io.deephaven.engine.util.systemicmarking.SystemicObjectTracker;
import io.deephaven.internal.log.LoggerFactory;
import io.deephaven.io.logger.Logger;
import io.deephaven.util.SafeCloseable;
+import io.deephaven.util.SafeCloseableList;
+import io.deephaven.util.annotations.InternalUseOnly;
+import io.deephaven.util.annotations.ReferentialIntegrity;
import io.deephaven.util.annotations.TestUseOnly;
import io.deephaven.util.annotations.VisibleForTesting;
+import io.deephaven.vector.Vector;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.commons.lang3.mutable.MutableObject;
@@ -79,15 +83,16 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY;
import static io.deephaven.engine.table.impl.MatchPair.matchString;
import static io.deephaven.engine.table.impl.partitioned.PartitionedTableCreatorImpl.CONSTITUENT;
+import static java.lang.Boolean.TRUE;
/**
* Primary coalesced table implementation.
@@ -139,8 +144,20 @@ public Result(@NotNull final T resultNode,
*/
String getLogPrefix();
- default OperationSnapshotControl newSnapshotControl(final QueryTable queryTable) {
- return new OperationSnapshotControl(queryTable);
+ /**
+ * Perform pre-instantiation work.
+ *
+ * @param parent The parent table for the operation
+ * @return A {@link SafeCloseable} that will be {@link SafeCloseable#close() closed} when the operation is
+ * complete, whether successful or not
+ */
+ default SafeCloseable beginOperation(@NotNull final QueryTable parent) {
+ return () -> {
+ };
+ }
+
+ default OperationSnapshotControl newSnapshotControl(final QueryTable parent) {
+ return new OperationSnapshotControl(parent);
}
/**
@@ -823,8 +840,7 @@ public QueryTable aggNoMemo(
@NotNull final Collection extends ColumnName> groupByColumns) {
final UpdateGraph updateGraph = getUpdateGraph();
try (final SafeCloseable ignored = ExecutionContext.getContext().withUpdateGraph(updateGraph).open()) {
- final String description = "aggregation(" + aggregationContextFactory
- + ", " + groupByColumns + ")";
+ final String description = "aggregation(" + aggregationContextFactory + ", " + groupByColumns + ")";
return QueryPerformanceRecorder.withNugget(description, sizeForInstrumentation(),
() -> ChunkedOperatorAggregationHelper.aggregation(
aggregationContextFactory, this, preserveEmpty, initialGroups, groupByColumns));
@@ -1026,9 +1042,9 @@ void doRefilter(
// Remove upstream keys first, so that keys at rows that were removed and then added are propagated as such.
// Note that it is a failure to propagate these as modifies, since modifiedColumnSet may not mark that all
// columns have changed.
- update.removed = upstream == null ? RowSetFactory.empty()
- : upstream.removed().intersect(getRowSet());
- getRowSet().writableCast().remove(update.removed);
+ update.removed = upstream == null
+ ? RowSetFactory.empty()
+ : getRowSet().writableCast().extract(upstream.removed());
// Update our rowSet and compute removals due to splatting.
if (upstream != null && upstream.shifted().nonempty()) {
@@ -1147,17 +1163,62 @@ public Table where(Filter filter) {
}
}
+ private void initializeAndPrioritizeFilters(@NotNull final WhereFilter... filters) {
+ final DataIndexer dataIndexer = DataIndexer.existingOf(rowSet);
+ final int numFilters = filters.length;
+ final BitSet priorityFilterIndexes = new BitSet(numFilters);
+ for (int fi = 0; fi < numFilters; ++fi) {
+ final WhereFilter filter = filters[fi];
+
+ // Initialize our filters immediately so we can examine the columns they use. Note that filter
+ // initialization is safe to invoke repeatedly.
+ filter.init(getDefinition());
+
+ // Simple filters against indexed columns get priority
+ if (dataIndexer != null
+ && !(filter instanceof ReindexingFilter)
+ && filter.isSimpleFilter()
+ && DataIndexer.hasDataIndex(this, filter.getColumns().toArray(ZERO_LENGTH_STRING_ARRAY))) {
+ priorityFilterIndexes.set(fi);
+ }
+ }
+
+ if (priorityFilterIndexes.isEmpty()) {
+ return;
+ }
+
+ // Copy the priority filters to a temporary array
+ final int numPriorityFilters = priorityFilterIndexes.cardinality();
+ final WhereFilter[] priorityFilters = new WhereFilter[numPriorityFilters];
+ // @formatter:off
+ for (int pfi = 0, fi = priorityFilterIndexes.nextSetBit(0);
+ fi >= 0;
+ fi = priorityFilterIndexes.nextSetBit(fi + 1)) {
+ // @formatter:on
+ priorityFilters[pfi++] = filters[fi];
+ }
+ // Move the regular (non-priority) filters to the back of the array
+ // @formatter:off
+ for (int rfi = numFilters - 1, fi = priorityFilterIndexes.previousClearBit(numFilters - 1);
+ fi >= 0;
+ fi = priorityFilterIndexes.previousClearBit(fi - 1)) {
+ // @formatter:on
+ filters[rfi--] = filters[fi];
+ }
+ // Re-add the priority filters at the front of the array
+ System.arraycopy(priorityFilters, 0, filters, 0, numPriorityFilters);
+ }
+
private QueryTable whereInternal(final WhereFilter... filters) {
if (filters.length == 0) {
- if (isRefreshing()) {
- manageWithCurrentScope();
- }
- return this;
+ return (QueryTable) prepareReturnThis();
}
final String whereDescription = "where(" + Arrays.toString(filters) + ")";
return QueryPerformanceRecorder.withNugget(whereDescription, sizeForInstrumentation(),
() -> {
+ initializeAndPrioritizeFilters(filters);
+
for (int fi = 0; fi < filters.length; ++fi) {
if (!(filters[fi] instanceof ReindexingFilter)) {
continue;
@@ -1185,100 +1246,117 @@ private QueryTable whereInternal(final WhereFilter... filters) {
return result;
}
- List selectFilters = new LinkedList<>();
- List>>> shiftColPairs = new LinkedList<>();
- for (final WhereFilter filter : filters) {
- filter.init(getDefinition());
- if (filter instanceof AbstractConditionFilter
- && ((AbstractConditionFilter) filter).hasConstantArrayAccess()) {
- shiftColPairs.add(((AbstractConditionFilter) filter).getFormulaShiftColPair());
- } else {
- selectFilters.add(filter);
+ {
+ final List whereFilters = new LinkedList<>();
+ final List>>> shiftColPairs =
+ new LinkedList<>();
+ for (final WhereFilter filter : filters) {
+ if (filter instanceof AbstractConditionFilter
+ && ((AbstractConditionFilter) filter).hasConstantArrayAccess()) {
+ shiftColPairs.add(((AbstractConditionFilter) filter).getFormulaShiftColPair());
+ } else {
+ whereFilters.add(filter);
+ }
+ }
+ if (!shiftColPairs.isEmpty()) {
+ return (QueryTable) ShiftedColumnsFactory.where(this, shiftColPairs, whereFilters);
}
- }
-
- if (!shiftColPairs.isEmpty()) {
- return (QueryTable) ShiftedColumnsFactory.where(this, shiftColPairs, selectFilters);
}
return memoizeResult(MemoizedOperationKey.filter(filters), () -> {
- final OperationSnapshotControl snapshotControl =
- createSnapshotControlIfRefreshing(OperationSnapshotControl::new);
-
- final Mutable result = new MutableObject<>();
- initializeWithSnapshot("where", snapshotControl,
- (prevRequested, beforeClock) -> {
- final boolean usePrev = prevRequested && isRefreshing();
- final RowSet rowSetToUse = usePrev ? rowSet.prev() : rowSet;
-
- final CompletableFuture currentMappingFuture =
- new CompletableFuture<>();
-
- final InitialFilterExecution initialFilterExecution = new InitialFilterExecution(
- this, filters, rowSetToUse.copy(), usePrev);
- final TrackingWritableRowSet currentMapping;
- initialFilterExecution.scheduleCompletion((adds, mods) -> {
- currentMappingFuture.complete(adds.writableCast().toTracking());
- }, currentMappingFuture::completeExceptionally);
-
- try {
- currentMapping = currentMappingFuture.get();
- } catch (ExecutionException | InterruptedException e) {
- if (e instanceof InterruptedException) {
- throw new CancellationException("interrupted while filtering");
- }
- throw new TableInitializationException(whereDescription,
- "an exception occurred while performing the initial filter",
- e.getCause());
- } finally {
- // account for work done in alternative threads
- final BasePerformanceEntry basePerformanceEntry =
- initialFilterExecution.getBasePerformanceEntry();
- if (basePerformanceEntry != null) {
- QueryPerformanceRecorder.getInstance().getEnclosingNugget()
- .accumulate(basePerformanceEntry);
+ try (final SafeCloseable ignored = Arrays.stream(filters)
+ .map(filter -> filter.beginOperation(this)).collect(SafeCloseableList.COLLECTOR)) {
+ final OperationSnapshotControl snapshotControl = createSnapshotControlIfRefreshing(
+ (final BaseTable> parent) -> {
+ /*
+ * Note that the dependencies for instantiation may be different from the
+ * dependencies for the WhereListener. Do not refactor to share this array with
+ * the WhereListener unless you ensure that this no longer holds, i.e. if
+ * MatchFilter starts applying data indexes during update processing.
+ */
+ final NotificationQueue.Dependency[] filterDependencies =
+ WhereListener.extractDependencies(filters)
+ .toArray(NotificationQueue.Dependency[]::new);
+ getUpdateGraph(filterDependencies);
+ return filterDependencies.length > 0
+ ? new OperationSnapshotControlEx(parent, filterDependencies)
+ : new OperationSnapshotControl(parent);
+ });
+
+ final Mutable result = new MutableObject<>();
+ initializeWithSnapshot("where", snapshotControl,
+ (prevRequested, beforeClock) -> {
+ final boolean usePrev = prevRequested && isRefreshing();
+ final RowSet rowSetToUse = usePrev ? rowSet.prev() : rowSet;
+
+ final CompletableFuture currentMappingFuture =
+ new CompletableFuture<>();
+ final InitialFilterExecution initialFilterExecution =
+ new InitialFilterExecution(this, filters, rowSetToUse.copy(), usePrev);
+ final TrackingWritableRowSet currentMapping;
+ initialFilterExecution.scheduleCompletion((adds, mods) -> {
+ currentMappingFuture.complete(adds.writableCast().toTracking());
+ }, currentMappingFuture::completeExceptionally);
+
+ try {
+ currentMapping = currentMappingFuture.get();
+ } catch (ExecutionException | InterruptedException e) {
+ if (e instanceof InterruptedException) {
+ throw new CancellationException("interrupted while filtering");
+ }
+ throw new TableInitializationException(whereDescription,
+ "an exception occurred while performing the initial filter",
+ e.getCause());
+ } finally {
+ // account for work done in alternative threads
+ final BasePerformanceEntry basePerformanceEntry =
+ initialFilterExecution.getBasePerformanceEntry();
+ if (basePerformanceEntry != null) {
+ QueryPerformanceRecorder.getInstance().getEnclosingNugget()
+ .accumulate(basePerformanceEntry);
+ }
}
- }
- currentMapping.initializePreviousValue();
+ currentMapping.initializePreviousValue();
- final FilteredTable filteredTable = new FilteredTable(currentMapping, this);
+ final FilteredTable filteredTable = new FilteredTable(currentMapping, this);
- for (final WhereFilter filter : filters) {
- filter.setRecomputeListener(filteredTable);
- }
- final boolean refreshingFilters =
- Arrays.stream(filters).anyMatch(WhereFilter::isRefreshing);
- copyAttributes(filteredTable, CopyAttributeOperation.Filter);
- // as long as filters do not change, we can propagate add-only/append-only attrs
- if (!refreshingFilters) {
- if (isAddOnly()) {
- filteredTable.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE);
+ for (final WhereFilter filter : filters) {
+ filter.setRecomputeListener(filteredTable);
}
- if (isAppendOnly()) {
- filteredTable.setAttribute(Table.APPEND_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE);
+ final boolean refreshingFilters =
+ Arrays.stream(filters).anyMatch(WhereFilter::isRefreshing);
+ copyAttributes(filteredTable, CopyAttributeOperation.Filter);
+ // as long as filters do not change, we can propagate add-only/append-only attrs
+ if (!refreshingFilters) {
+ if (isAddOnly()) {
+ filteredTable.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, TRUE);
+ }
+ if (isAppendOnly()) {
+ filteredTable.setAttribute(Table.APPEND_ONLY_TABLE_ATTRIBUTE, TRUE);
+ }
}
- }
- if (snapshotControl != null) {
- final ListenerRecorder recorder = new ListenerRecorder(
- whereDescription, QueryTable.this,
- filteredTable);
- final WhereListener whereListener = new WhereListener(
- log, this, recorder, filteredTable, filters);
- filteredTable.setWhereListener(whereListener);
- recorder.setMergedListener(whereListener);
- snapshotControl.setListenerAndResult(recorder, filteredTable);
- filteredTable.addParentReference(whereListener);
- } else if (refreshingFilters) {
- final WhereListener whereListener = new WhereListener(
- log, this, null, filteredTable, filters);
- filteredTable.setWhereListener(whereListener);
- filteredTable.addParentReference(whereListener);
- }
- result.setValue(filteredTable);
- return true;
- });
- return result.getValue();
+ if (snapshotControl != null) {
+ final ListenerRecorder recorder = new ListenerRecorder(
+ whereDescription, QueryTable.this,
+ filteredTable);
+ final WhereListener whereListener = new WhereListener(
+ log, this, recorder, filteredTable, filters);
+ filteredTable.setWhereListener(whereListener);
+ recorder.setMergedListener(whereListener);
+ snapshotControl.setListenerAndResult(recorder, filteredTable);
+ filteredTable.addParentReference(whereListener);
+ } else if (refreshingFilters) {
+ final WhereListener whereListener = new WhereListener(
+ log, this, null, filteredTable, filters);
+ filteredTable.setWhereListener(whereListener);
+ filteredTable.addParentReference(whereListener);
+ }
+ result.setValue(filteredTable);
+ return true;
+ });
+ return result.getValue();
+ }
});
});
}
@@ -1308,7 +1386,13 @@ private Table whereInInternal(final Table rightTable, final boolean inclusion,
final Table distinctValues;
final boolean setRefreshing = rightTable.isRefreshing();
- if (setRefreshing) {
+
+ final String[] columnNames = MatchPair.getRightColumns(columnsToMatch);
+ final DataIndex rightIndex = DataIndexer.getDataIndex(rightTable, columnNames);
+ if (rightIndex != null) {
+ // We have a distinct index table, let's use it.
+ distinctValues = rightIndex.table();
+ } else if (setRefreshing) {
distinctValues = rightTable.selectDistinct(MatchPair.getRightColumns(columnsToMatch));
} else {
distinctValues = rightTable;
@@ -1497,9 +1581,7 @@ this, mode, columns, rowSet, getModifiedColumnSetForUpdates(), publishTheseSourc
if (resultTable.getRowSet().isFlat()) {
resultTable.setFlat();
}
- if (resultTable.getRowSet() == rowSet) {
- propagateGrouping(processedColumns, resultTable);
- }
+ propagateDataIndexes(processedColumns, resultTable);
for (final ColumnSource> columnSource : analyzer.getNewColumnSources().values()) {
if (columnSource instanceof PossiblyImmutableColumnSource) {
((PossiblyImmutableColumnSource) columnSource).setImmutable();
@@ -1521,8 +1603,39 @@ this, mode, columns, rowSet, getModifiedColumnSetForUpdates(), publishTheseSourc
}));
}
- private void propagateGrouping(SelectColumn[] selectColumns, QueryTable resultTable) {
- final Set usedOutputColumns = new HashSet<>();
+ /**
+ * Data indexes on select source columns may be valid in the result table. Add new mappings so the data indexes are
+ * retrievable from the result table column sources.
+ */
+ private void propagateDataIndexes(SelectColumn[] selectColumns, QueryTable resultTable) {
+ if (rowSet != resultTable.getRowSet()) {
+ return;
+ }
+
+ // Get a list of all the data indexes in the source table.
+ final DataIndexer dataIndexer = DataIndexer.existingOf(rowSet);
+ if (dataIndexer == null) {
+ return;
+ }
+ final List dataIndexes = dataIndexer.dataIndexes(false);
+ if (dataIndexes.isEmpty()) {
+ return;
+ }
+
+ // Make a set containing only the columns that were part of indexes.
+ Set> indexedColumns = new HashSet<>();
+ dataIndexes.forEach(di -> {
+ indexedColumns.addAll(di.keyColumnNamesByIndexedColumn().keySet());
+ });
+
+ final Set> usedOutputColumns = new HashSet<>();
+
+ // Maintain a list of unique old to new mappings.
+ // NOTE: this is complex because select() may return multiple aliases for the same source column. Until/unless
+ // we have a guarantee that only a single alias is created, we must create a new data index for each alias and
+ // this requires generating all unique mappings for these columns.
+ final List