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: + *

    + *
  1. Intersect the index {@link RowSet RowSets} with the supplied RowSet. Note that the result will always be a + * static snapshot.
  2. + *
  3. Invert the index {@link RowSet RowSets} with the supplied RowSet. Note that the result will always be a + * static snapshot.
  4. + *
  5. Sort the index table by the first row key within each {@link RowSet}. Be careful to use the correct + * dependency for satisfaction!
  6. + *
+ *

+ * + * @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 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 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 leftGrouping; - final Map 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: + *

    + *
  1. Keep the hash table small
  2. + *
  3. Avoid rehashing
  4. + *
+ * 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 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 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 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 columns) { + public final Table selectDistinct(@NotNull final Collection 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 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 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, ColumnSource>> oldToNewMaps = new ArrayList<>(); + for (SelectColumn selectColumn : selectColumns) { if (selectColumn instanceof SwitchColumn) { selectColumn = ((SwitchColumn) selectColumn).getRealColumn(); @@ -1532,39 +1645,67 @@ private void propagateGrouping(SelectColumn[] selectColumns, QueryTable resultTa if (selectColumn instanceof SourceColumn) { sourceColumn = (SourceColumn) selectColumn; } - if (sourceColumn != null && !usedOutputColumns.contains(sourceColumn.getSourceName())) { - final ColumnSource originalColumnSource = ReinterpretUtils.maybeConvertToPrimitive( - getColumnSource(sourceColumn.getSourceName())); - final ColumnSource selectedColumnSource = ReinterpretUtils.maybeConvertToPrimitive( - resultTable.getColumnSource(sourceColumn.getName())); - if (originalColumnSource != selectedColumnSource) { - if (originalColumnSource instanceof DeferredGroupingColumnSource) { - final DeferredGroupingColumnSource deferredGroupingSelectedSource = - (DeferredGroupingColumnSource) selectedColumnSource; - final GroupingProvider groupingProvider = - ((DeferredGroupingColumnSource) originalColumnSource).getGroupingProvider(); - if (groupingProvider != null) { - // noinspection unchecked,rawtypes - deferredGroupingSelectedSource.setGroupingProvider((GroupingProvider) groupingProvider); - } else if (originalColumnSource.getGroupToRange() != null) { - // noinspection unchecked,rawtypes - deferredGroupingSelectedSource - .setGroupToRange((Map) originalColumnSource.getGroupToRange()); - } - } else if (originalColumnSource.getGroupToRange() != null) { - final DeferredGroupingColumnSource deferredGroupingSelectedSource = - (DeferredGroupingColumnSource) selectedColumnSource; - // noinspection unchecked,rawtypes - deferredGroupingSelectedSource.setGroupToRange((Map) originalColumnSource.getGroupToRange()); - } else { - final RowSetIndexer indexer = RowSetIndexer.of(rowSet); - if (indexer.hasGrouping(originalColumnSource)) { - indexer.copyImmutableGroupings(originalColumnSource, selectedColumnSource); - } - } - } + + if (sourceColumn == null) { + continue; } - usedOutputColumns.add(selectColumn.getName()); + + final ColumnSource resultSource = resultTable.getColumnSource(sourceColumn.getName()); + if (usedOutputColumns.contains(resultSource)) { + // We already handled this result source. + continue; + } + usedOutputColumns.add(resultSource); + + final ColumnSource originalSource = getColumnSource(sourceColumn.getSourceName()); + if (!indexedColumns.contains(originalSource) || originalSource == resultSource) { + // Not part of an index, or not changed, ignore this column. + continue; + } + + if (oldToNewMaps.isEmpty()) { + oldToNewMaps.add(new HashMap<>()); + } + + // Map original source to the new source. + final Map, ColumnSource> firstMap = oldToNewMaps.get(0); + if (firstMap.containsKey(originalSource)) { + // We have a collision, which means our mapping won't be unique. We need to clone + // conflicting mappings with a new map containing our updated target. + final ColumnSource previousResultSource = firstMap.get(originalSource); + final List, ColumnSource>> newMaps = new ArrayList<>(); + oldToNewMaps.stream() + .filter(map -> map.get(originalSource) == previousResultSource) + .forEach(map -> { + final Map, ColumnSource> newMap = new HashMap<>(map); + newMap.put(originalSource, resultSource); + newMaps.add(newMap); + }); + oldToNewMaps.addAll(newMaps); + } else { + oldToNewMaps.forEach(map -> map.put(originalSource, resultSource)); + } + } + + if (oldToNewMaps.isEmpty()) { + return; + } + + // Add new DataIndex entries to the DataIndexer with the remapped column sources. + for (final DataIndex dataIndex : dataIndexes) { + // Create a new data index for each unique mapping. + oldToNewMaps.forEach(map -> { + if (Collections.disjoint(dataIndex.keyColumnNamesByIndexedColumn().keySet(), map.keySet())) { + // The index contains no remapped original sources, no work needed. + return; + } + + // Create a new DataIndex using the new column sources as keys. + final DataIndex remappedIndex = dataIndex.remapKeyColumns(map); + + // Add the new index to the DataIndexer. + dataIndexer.addDataIndex(remappedIndex); + }); } } @@ -2178,7 +2319,7 @@ private Table joinNoMemo( final Table naturalJoinResult = naturalJoinImpl(rightGrouped, columnsToMatch, columnsToAddAfterRename.toArray(MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY)); final QueryTable ungroupedResult = (QueryTable) naturalJoinResult - .ungroup(columnsToUngroupBy.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY)); + .ungroup(columnsToUngroupBy.toArray(ZERO_LENGTH_STRING_ARRAY)); maybeCopyColumnDescriptions(ungroupedResult, rightTable, columnsToMatch, realColumnsToAdd); @@ -3571,28 +3712,30 @@ private T getResultNoMemo(fin return QueryPerformanceRecorder.withNugget(operation.getDescription(), sizeForInstrumentation(), () -> { final Mutable resultTable = new MutableObject<>(); - final OperationSnapshotControl snapshotControl; - if (isRefreshing() && operation.snapshotNeeded()) { - snapshotControl = operation.newSnapshotControl(this); - } else { - snapshotControl = null; - } - - initializeWithSnapshot(operation.getLogPrefix(), snapshotControl, (usePrev, beforeClockValue) -> { - final Operation.Result result = operation.initialize(usePrev, beforeClockValue); - if (result == null) { - return false; + try (final SafeCloseable ignored = operation.beginOperation(this)) { + final OperationSnapshotControl snapshotControl; + if (isRefreshing() && operation.snapshotNeeded()) { + snapshotControl = operation.newSnapshotControl(this); + } else { + snapshotControl = null; } - resultTable.setValue(result.resultNode); - if (snapshotControl != null) { - snapshotControl.setListenerAndResult(result.resultListener, result.resultNode); - } + initializeWithSnapshot(operation.getLogPrefix(), snapshotControl, (usePrev, beforeClockValue) -> { + final Operation.Result result = operation.initialize(usePrev, beforeClockValue); + if (result == null) { + return false; + } - return true; - }); + resultTable.setValue(result.resultNode); + if (snapshotControl != null) { + snapshotControl.setListenerAndResult(result.resultListener, result.resultNode); + } - return resultTable.getValue(); + return true; + }); + + return resultTable.getValue(); + } }); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/RedefinableTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/RedefinableTable.java index 206121ab899..24c36fb5733 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/RedefinableTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/RedefinableTable.java @@ -50,7 +50,7 @@ private Table viewInternal(Collection selectables, boolean final Map> allColumns = new HashMap<>(definition.getColumnNameMap()); boolean simpleRetain = true; for (final SelectColumn selectColumn : columns) { - List usedColumnNames = selectColumn.initDef(allColumns); + final List usedColumnNames = new ArrayList<>(selectColumn.initDef(allColumns)); usedColumnNames.addAll(selectColumn.getColumnArrays()); resultColumnsInternal.addAll(usedColumnNames.stream() .filter(usedColumnName -> !resultColumnsExternal.containsKey(usedColumnName)) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/RightIncrementalNaturalJoinStateManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/RightIncrementalNaturalJoinStateManager.java index c1967df9ef6..98c5d43dc2a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/RightIncrementalNaturalJoinStateManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/RightIncrementalNaturalJoinStateManager.java @@ -5,17 +5,16 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Context; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; public abstract class RightIncrementalNaturalJoinStateManager extends StaticNaturalJoinStateManager implements IncrementalNaturalJoinStateManager { + protected RightIncrementalNaturalJoinStateManager(ColumnSource[] keySourcesForErrorMessages) { super(keySourcesForErrorMessages); } @@ -23,16 +22,16 @@ protected RightIncrementalNaturalJoinStateManager(ColumnSource[] keySourcesFo public abstract void buildFromLeftSide(final Table leftTable, ColumnSource[] leftSources, final InitialBuildContext initialBuildContext); - public abstract void convertLeftGroups(int groupingSize, InitialBuildContext initialBuildContext, - ObjectArraySource rowSetSource); + public abstract void convertLeftDataIndex(int groupingSize, InitialBuildContext initialBuildContext, + ColumnSource rowSetSource); public abstract void addRightSide(RowSequence rightIndex, ColumnSource[] rightSources); public abstract WritableRowRedirection buildRowRedirectionFromHashSlot(QueryTable leftTable, boolean exactMatch, InitialBuildContext initialBuildContext, JoinControl.RedirectionType redirectionType); - public abstract WritableRowRedirection buildRowRedirectionFromHashSlotGrouped(QueryTable leftTable, - ObjectArraySource rowSetSource, int groupingSize, boolean exactMatch, + public abstract WritableRowRedirection buildRowRedirectionFromHashSlotIndexed(QueryTable leftTable, + ColumnSource rowSetSource, int groupingSize, boolean exactMatch, InitialBuildContext initialBuildContext, JoinControl.RedirectionType redirectionType); // modification probes diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SimpleSourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SimpleSourceTable.java index 7b30e1c6760..f78050bca73 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SimpleSourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SimpleSourceTable.java @@ -47,7 +47,7 @@ protected SimpleSourceTable copy() { } @Override - protected final SourceTable redefine(TableDefinition newDefinition) { + protected final SourceTable redefine(TableDefinition newDefinition) { if (newDefinition.getColumnNames().equals(definition.getColumnNames())) { // Nothing changed - we have the same columns in the same order. return this; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortHelpers.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortHelpers.java index f7a87b9e952..34c90468406 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortHelpers.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortHelpers.java @@ -5,46 +5,45 @@ import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.configuration.Configuration; import io.deephaven.datastructures.util.CollectionUtil; +import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.rowset.RowSequenceFactory; -import io.deephaven.engine.table.WritableColumnSource; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.DataIndex; import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.sort.LongMegaMergeKernel; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.*; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.util.QueryConstants; -import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.engine.table.impl.sort.LongSortKernel; import io.deephaven.engine.table.impl.sort.findruns.FindRunsKernel; -import io.deephaven.engine.table.impl.sort.timsort.LongIntTimsortKernel; import io.deephaven.engine.table.impl.sort.permute.PermuteKernel; -import io.deephaven.engine.table.impl.sort.LongSortKernel; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.sources.LongArraySource; -import io.deephaven.engine.table.impl.sources.LongSparseArraySource; -import io.deephaven.chunk.*; +import io.deephaven.engine.table.impl.sort.timsort.LongIntTimsortKernel; +import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.sources.regioned.SymbolTableSource; +import io.deephaven.engine.table.impl.util.ContiguousWritableRowRedirection; +import io.deephaven.engine.table.impl.util.GroupedWritableRowRedirection; +import io.deephaven.engine.table.impl.util.LongColumnSourceWritableRowRedirection; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; +import io.deephaven.engine.table.iterators.ChunkedColumnIterator; +import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.VisibleForTesting; -import io.deephaven.util.type.TypeUtils; - +import io.deephaven.util.datastructures.LongSizedDataStructure; import org.apache.commons.lang3.mutable.MutableInt; import org.jetbrains.annotations.NotNull; -import java.lang.reflect.Array; -import java.util.Arrays; import java.util.Map; import java.util.function.LongPredicate; public class SortHelpers { + public static boolean sortBySymbolTable = Configuration.getInstance().getBooleanWithDefault("QueryTable.sortBySymbolTable", true); @@ -61,7 +60,7 @@ public class SortHelpers { * When the sort is greater than or equal to than megaSortSize, instead of sorting one large chunk, we will sort * individual chunks of sortChunkSize and then merge them into ArrayBackedColumnSources with the * LongMegaMergeKernel. - * + *

* There are some boundary conditions in the chunk sizing math that make Integer.MAX_VALUE fail; you could probably * back off to Integer.MAX_VALUE - 32 safely. We're being very conservative with 1 << 30 instead. */ @@ -106,12 +105,12 @@ final public long size() { } @NotNull - final public long[] getArrayMapping() { + public long[] getArrayMapping() { return Require.neqNull(mapping, "mapping"); } @Override - final public boolean forEachLong(LongPredicate consumer) { + public boolean forEachLong(LongPredicate consumer) { for (int ii = 0; ii < mapping.length; ++ii) { if (!consumer.test(mapping[ii])) { return false; @@ -143,17 +142,17 @@ private ColumnSourceSortMapping(LongArraySource columnSource, long size) { } @Override - final public long size() { + public long size() { return size; } @NotNull - final public long[] getArrayMapping() { + public long[] getArrayMapping() { throw new ArrayIndexOutOfBoundsException(); } @Override - final public boolean forEachLong(LongPredicate consumer) { + public boolean forEachLong(LongPredicate consumer) { assert null != columnSource; for (long ii = 0; ii < size; ++ii) { if (!consumer.test(columnSource.getLong(ii))) { @@ -169,24 +168,24 @@ public WritableRowRedirection makeHistoricalRowRedirection() { } } - final static class GroupedSortMapping implements SortMapping { + final static class IndexedSortMapping implements SortMapping { private final long size; private final long[] groupSize; private final RowSet[] groups; - private GroupedSortMapping(long size, long[] groupSize, RowSet[] groups) { + private IndexedSortMapping(long size, long[] groupSize, RowSet[] groups) { this.size = size; this.groupSize = groupSize; this.groups = groups; } @Override - final public long size() { + public long size() { return size; } @NotNull - final public long[] getArrayMapping() { + public long[] getArrayMapping() { if (size <= Integer.MAX_VALUE) { final long[] mapping = new long[(int) size]; final MutableInt pos = new MutableInt(0); @@ -201,7 +200,7 @@ final public long[] getArrayMapping() { } @Override - final public boolean forEachLong(LongPredicate consumer) { + public boolean forEachLong(LongPredicate consumer) { for (int ii = 0; ii < groups.length; ++ii) { if (!groups[ii].forEachRowKey(consumer::test)) { return false; @@ -220,42 +219,52 @@ public WritableRowRedirection makeHistoricalRowRedirection() { /** * Note that if usePrev is true, then rowSetToSort is the previous RowSet; not the current RowSet, and we should not - * need to call copyPrev. + * need to call prev(). */ - static SortMapping getSortedKeys(SortingOrder[] order, ColumnSource>[] columnsToSortBy, - RowSet rowSetToSort, boolean usePrev) { - return getSortedKeys(order, columnsToSortBy, rowSetToSort, usePrev, sortBySymbolTable); + static SortMapping getSortedKeys( + final SortingOrder[] order, + final ColumnSource>[] originalColumnsToSortBy, + final ColumnSource>[] columnsToSortBy, + final DataIndex dataIndex, + final RowSet rowSetToSort, + final boolean usePrev) { + return getSortedKeys(order, originalColumnsToSortBy, columnsToSortBy, dataIndex, rowSetToSort, usePrev, + sortBySymbolTable); } /** * Note that if usePrev is true, then rowSetToSort is the previous RowSet; not the current RowSet, and we should not - * need to call copyPrev. + * need to call prev(). */ - static SortMapping getSortedKeys(SortingOrder[] order, ColumnSource>[] columnsToSortBy, - RowSet rowSetToSort, boolean usePrev, boolean allowSymbolTable) { + static SortMapping getSortedKeys( + final SortingOrder[] order, + final ColumnSource>[] originalColumnsToSortBy, + final ColumnSource>[] columnsToSortBy, + final DataIndex dataIndex, + final RowSet rowSetToSort, + final boolean usePrev, + final boolean allowSymbolTable) { if (rowSetToSort.isEmpty()) { return EMPTY_SORT_MAPPING; } + // Don't use a full index if it is too large. + if (dataIndex != null + && dataIndex.keyColumnNames().size() == columnsToSortBy.length + && rowSetToSort.size() > dataIndex.table().size() * 2L) { + return getSortMappingIndexed(order, originalColumnsToSortBy, dataIndex, rowSetToSort, usePrev); + } + if (columnsToSortBy.length == 1) { - if (rowSetToSort.isTracking() - && RowSetIndexer.of(rowSetToSort.trackingCast()).hasGrouping(columnsToSortBy[0])) { - if (!usePrev || columnsToSortBy[0].isImmutable()) { - return getSortMappingGrouped(order[0], columnsToSortBy[0], rowSetToSort.trackingCast()); - } else { - return getSortMappingOne(order[0], columnsToSortBy[0], rowSetToSort, usePrev); - } + if (allowSymbolTable && columnsToSortBy[0] instanceof SymbolTableSource + && ((SymbolTableSource>) columnsToSortBy[0]).hasSymbolTable(rowSetToSort)) { + return doSymbolTableMapping(order[0], columnsToSortBy[0], rowSetToSort, usePrev); } else { - if (allowSymbolTable && columnsToSortBy[0] instanceof SymbolTableSource - && ((SymbolTableSource>) columnsToSortBy[0]).hasSymbolTable(rowSetToSort)) { - return doSymbolTableMapping(order[0], columnsToSortBy[0], rowSetToSort, usePrev); - } else { - return getSortMappingOne(order[0], columnsToSortBy[0], rowSetToSort, usePrev); - } + return getSortMappingOne(order[0], columnsToSortBy[0], rowSetToSort, usePrev); } } - return getSortMappingMulti(order, columnsToSortBy, rowSetToSort, usePrev); + return getSortMappingMulti(order, originalColumnsToSortBy, columnsToSortBy, dataIndex, rowSetToSort, usePrev); } private static class SparseSymbolMapping { @@ -515,49 +524,67 @@ private static long[] doChunkSortingOne(SortingOrder order, ColumnSource> columnSource, - TrackingRowSet rowSet) { - final Map groupToRange = RowSetIndexer.of(rowSet).getGrouping(columnSource); - final Object[] keys = groupToRange.keySet().toArray((Object[]) Array.newInstance( - io.deephaven.util.type.TypeUtils.getBoxedType(columnSource.getType()), groupToRange.size())); - - Arrays.sort((Comparable[]) keys, order.getComparator()); + private static SortMapping getSortMappingIndexed(SortingOrder[] order, ColumnSource>[] columnSources, + DataIndex dataIndex, RowSet rowSet, boolean usePrev) { + Assert.neqNull(dataIndex, "dataIndex"); + + final Table indexTable = dataIndex.table(); + final RowSet indexRowSet = usePrev ? indexTable.getRowSet().prev() : indexTable.getRowSet(); + // noinspection unchecked + final ColumnSource>[] originalIndexKeyColumns = + (ColumnSource>[]) dataIndex.keyColumns(columnSources); + // noinspection unchecked + final ColumnSource>[] indexKeyColumns = + (ColumnSource>[]) ReinterpretUtils.maybeConvertToPrimitive(originalIndexKeyColumns); + final SortMapping indexMapping = getSortedKeys(order, originalIndexKeyColumns, indexKeyColumns, null, + indexRowSet, usePrev); + + final String rowSetColumnName = dataIndex.rowSetColumnName(); + final ColumnSource rawRowSetColumn = dataIndex.table().getColumnSource(rowSetColumnName, RowSet.class); + final ColumnSource redirectedRowSetColumn = RedirectedColumnSource.alwaysRedirect( + indexMapping.makeHistoricalRowRedirection(), + usePrev ? rawRowSetColumn.getPrevSource() : rawRowSetColumn); final boolean fitsIntoArray = rowSet.size() < (long) Integer.MAX_VALUE; - final long elementsPerGroup = rowSet.size() / (long) groupToRange.size(); - + final long elementsPerGroup = rowSet.size() / indexRowSet.size(); if (fitsIntoArray && (elementsPerGroup < groupedRedirectionThreshold)) { final long[] rowKeysArray = new long[rowSet.intSize()]; - final MutableInt outputIdx = new MutableInt(0); - for (final Object key : keys) { - final RowSet group = groupToRange.get(key).intersect(rowSet); - group.forAllRowKeys(rowKey -> { - rowKeysArray[outputIdx.intValue()] = rowKey; - outputIdx.increment(); - }); + + final MutableInt nextOutputOffset = new MutableInt(0); + try (final RowSequence flat = RowSequenceFactory.forRange(0, indexRowSet.size() - 1); + final CloseableIterator groups = ChunkedColumnIterator.make(redirectedRowSetColumn, flat)) { + groups.forEachRemaining((final RowSet group) -> group + .forAllRowKeys(rowKey -> rowKeysArray[nextOutputOffset.getAndIncrement()] = rowKey)); } return new ArraySortMapping(rowKeysArray); } else { - // create a grouped row redirection - final long[] groupSize = new long[groupToRange.size()]; - final RowSet[] groupRowSet = new RowSet[groupToRange.size()]; - - long outputSize = 0; - int ii = 0; - for (final Object key : keys) { - final RowSet group = groupToRange.get(key).intersect(rowSet); - outputSize += group.size(); - groupSize[ii] = outputSize; - groupRowSet[ii++] = group; + final int indexSize = indexRowSet.intSize(); + final long[] cumulativeSize = new long[indexSize]; + final RowSet[] groupRowSet = new RowSet[indexSize]; + + final MutableInt nextGroupOffset = new MutableInt(0); + try (final RowSequence flat = RowSequenceFactory.forRange(0, indexRowSet.size() - 1); + final CloseableIterator groups = ChunkedColumnIterator.make(redirectedRowSetColumn, flat)) { + groups.forEachRemaining((final RowSet group) -> { + final int go = nextGroupOffset.getAndIncrement(); + cumulativeSize[go] = go == 0 ? group.size() : cumulativeSize[go - 1] + group.size(); + groupRowSet[go] = group; // No need to copy(): either static, or never used after instantiation + }); } - return new GroupedSortMapping(outputSize, groupSize, groupRowSet); + Assert.eq(cumulativeSize[indexSize - 1], "cumulativeSize[indexSize - 1]", rowSet.size(), "rowSet.size()"); + return new IndexedSortMapping(rowSet.size(), cumulativeSize, groupRowSet); } } - private static SortMapping getSortMappingMulti(SortingOrder[] order, ColumnSource>[] columnSources, - RowSet rowSet, boolean usePrev) { + private static SortMapping getSortMappingMulti( + final SortingOrder[] order, + final ColumnSource>[] originalColumnSources, + final ColumnSource>[] columnSources, + final DataIndex dataIndex, + final RowSet rowSet, + boolean usePrev) { Assert.gt(columnSources.length, "columnSources.length", 1); final int sortSize = rowSet.intSize(); @@ -569,27 +596,41 @@ private static SortMapping getSortMappingMulti(SortingOrder[] order, ColumnSourc ColumnSource> columnSource = columnSources[0]; - if (rowSet.isTracking() && RowSetIndexer.of(rowSet.trackingCast()).hasGrouping(columnSources[0])) { - final Map, RowSet> groupToRange = columnSource.getGroupToRange(); - final Object[] keys = groupToRange.keySet().toArray( - (Object[]) Array.newInstance(TypeUtils.getBoxedType(columnSource.getType()), groupToRange.size())); - - Arrays.sort((Comparable[]) keys, order[0].getComparator()); - + // Can we utilize an existing index on the first column? + if (dataIndex != null + && dataIndex.keyColumnNames().size() == 1 + && dataIndex.keyColumnNamesByIndexedColumn().containsKey(originalColumnSources[0])) { + final Table indexTable = dataIndex.table(); + final RowSet indexRowSet = usePrev ? indexTable.getRowSet().prev() : indexTable.getRowSet(); + final ColumnSource> originalIndexKeyColumn = indexTable.getColumnSource( + dataIndex.keyColumnNamesByIndexedColumn().get(originalColumnSources[0]), + originalColumnSources[0].getType()); + // noinspection unchecked + final ColumnSource> indexColumn = + (ColumnSource>) ReinterpretUtils.maybeConvertToPrimitive(originalIndexKeyColumn); + + final SortMapping indexMapping = getSortMappingOne(order[0], indexColumn, indexRowSet, usePrev); + + final String rowSetColumnName = dataIndex.rowSetColumnName(); + final ColumnSource rawRowSetColumn = + dataIndex.table().getColumnSource(rowSetColumnName, RowSet.class); + final ColumnSource redirectedRowSetColumn = RedirectedColumnSource.alwaysRedirect( + indexMapping.makeHistoricalRowRedirection(), + usePrev ? rawRowSetColumn.getPrevSource() : rawRowSetColumn); + + rowKeys.setSize(0); offsetsOut.setSize(0); lengthsOut.setSize(0); - - final MutableInt outputIdx = new MutableInt(0); - for (final Object key : keys) { - // noinspection SuspiciousMethodCalls - final RowSet group = groupToRange.get(key).intersect(rowSet); - if (group.size() > 1) { - offsetsOut.add(outputIdx.intValue()); - lengthsOut.add(group.intSize()); - } - group.forAllRowKeys(rowKey -> { - rowKeysArray[outputIdx.intValue()] = rowKey; - outputIdx.increment(); + final WritableIntChunk fOffsetsOut = offsetsOut; + final WritableIntChunk fLengthsOut = lengthsOut; + try (final RowSequence flat = RowSequenceFactory.forRange(0, indexRowSet.size() - 1); + final CloseableIterator groups = ChunkedColumnIterator.make(redirectedRowSetColumn, flat)) { + groups.forEachRemaining((final RowSet group) -> { + if (group.size() > 1) { + fOffsetsOut.add(rowKeys.size()); + fLengthsOut.add(group.intSize()); + } + group.forAllRowKeys(rowKeys::add); }); } } else { @@ -684,7 +725,7 @@ private static SortMapping getSortMappingMulti(SortingOrder[] order, ColumnSourc return new ArraySortMapping(rowKeysArray); } - private static WritableChunk fetchSecondaryValues(boolean usePrev, ColumnSource columnSource, + private static WritableChunk fetchSecondaryValues(boolean usePrev, ColumnSource columnSource, WritableLongChunk indicesToFetch, WritableIntChunk originalPositions, LongIntTimsortKernel.LongIntSortKernelContext sortIndexContext, int maximumSecondarySize) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java index 3a35285ddba..4451e2d4856 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortListener.java @@ -45,6 +45,7 @@ public class SortListener extends BaseTable.ListenerImpl { private final Table parent; private final QueryTable result; private final HashMapK4V4 reverseLookup; + private final ColumnSource>[] originalColumnsToSortBy; private final ColumnSource>[] columnsToSortBy; private final WritableRowSet resultRowSet; private final SortingOrder[] order; @@ -57,14 +58,22 @@ public class SortListener extends BaseTable.ListenerImpl { private final ModifiedColumnSet.Transformer mcsTransformer; private final ModifiedColumnSet sortColumnSet; - public SortListener(Table parent, QueryTable result, HashMapK4V4 reverseLookup, - ColumnSource>[] columnsToSortBy, SortingOrder[] order, - WritableRowRedirection sortMapping, ColumnSource>[] sortedColumnsToSortBy, - ModifiedColumnSet.Transformer mcsTransformer, ModifiedColumnSet sortColumnSet) { + public SortListener( + final Table parent, + final QueryTable result, + final HashMapK4V4 reverseLookup, + final ColumnSource>[] originalColumnsToSortBy, + final ColumnSource>[] columnsToSortBy, + final SortingOrder[] order, + final WritableRowRedirection sortMapping, + final ColumnSource>[] sortedColumnsToSortBy, + final ModifiedColumnSet.Transformer mcsTransformer, + final ModifiedColumnSet sortColumnSet) { super("sortInternal", parent, result); this.parent = parent; this.result = result; this.reverseLookup = reverseLookup; + this.originalColumnsToSortBy = originalColumnsToSortBy; this.columnsToSortBy = columnsToSortBy; this.resultRowSet = result.getRowSet().writableCast(); this.order = order; @@ -223,8 +232,8 @@ public void onUpdate(final TableUpdate upstream) { int numPropagatedModdedKeys = 0; final RowSet addedAndModified = modifiedNeedsSorting ? closer.add(upstream.added().union(upstream.modified())) : upstream.added(); - final long[] addedInputKeys = - SortHelpers.getSortedKeys(order, columnsToSortBy, addedAndModified, false, false).getArrayMapping(); + final long[] addedInputKeys = SortHelpers.getSortedKeys(order, originalColumnsToSortBy, columnsToSortBy, + null, addedAndModified, false, false).getArrayMapping(); final long[] addedOutputKeys = new long[addedInputKeys.length]; final long[] propagatedModOutputKeys = modifiedNeedsSorting ? new long[upstream.modified().intSize()] : CollectionUtil.ZERO_LENGTH_LONG_ARRAY; @@ -363,9 +372,8 @@ private RowSet sortedArrayToIndex(long[] arr, int offset, int length) { * @param qs Queue state -- containing the view on the various keys arrays, directions, etc. */ private void performUpdatesInDirection(final RowSetBuilderSequential added, final RowSetShiftData.Builder shifted, - final long start, - final QueueState qs, final SortMappingAggregator mappingChanges) { - final long numRequestedAdds = (qs.addedEnd - qs.addedCurrent) * qs.direction; + final long start, final QueueState qs, final SortMappingAggregator mappingChanges) { + final long numRequestedAdds = (qs.addedEnd - qs.addedCurrent) * (long) qs.direction; if (numRequestedAdds == 0) { return; @@ -428,7 +436,7 @@ private void performUpdatesInDirection(final RowSetBuilderSequential added, fina } // determine if we must be in spreading mode - final long maxRunKey = desiredOutputKey + maximumRunLength * qs.direction; + final long maxRunKey = desiredOutputKey + maximumRunLength * (long) qs.direction; // note: this is an (over) approximation of cardinality since binarySearch will give any index if exists long addedMaxIdx; @@ -535,7 +543,7 @@ private long insertAGap(final long destinationSlot, final QueueState qs, final DirectionalResettableBuilderSequential modRemoved, final SortMappingAggregator mappingChanges, final RowSet.SearchIterator gapEvictionIter) { - final long gapEnd = destinationSlot + REBALANCE_GAP_SIZE * qs.direction; // exclusive + final long gapEnd = destinationSlot + REBALANCE_GAP_SIZE * (long) qs.direction; // exclusive checkDestinationSlotOk(gapEnd); modRemoved.appendRange(destinationSlot, gapEnd - qs.direction); @@ -603,9 +611,11 @@ private class SortMappingAggregator implements SafeCloseable { private final int chunkSize; private final ExposedTLongArrayList keys; private final ExposedTLongArrayList values; + @SuppressWarnings("rawtypes") private final WritableLongChunk valuesChunk; private final WritableLongChunk keysChunk; private final ChunkSink.FillFromContext fillFromContext; + @SuppressWarnings("rawtypes") private final LongSortKernel sortKernel; SortMappingAggregator() { @@ -627,7 +637,7 @@ public void close() { } public void flush() { - if (keys.size() == 0) { + if (keys.isEmpty()) { return; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortOperation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortOperation.java index 3d66d2071fb..34f65b86f7a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SortOperation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SortOperation.java @@ -5,24 +5,26 @@ import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.table.impl.sources.RedirectedColumnSource; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.engine.table.impl.sources.SwitchColumnSource; +import io.deephaven.engine.table.impl.sources.chunkcolumnsource.LongChunkColumnSource; +import io.deephaven.engine.table.impl.util.LongColumnSourceRowRedirection; +import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.engine.table.iterators.ChunkedLongColumnIterator; import io.deephaven.engine.table.iterators.LongColumnIterator; +import io.deephaven.util.SafeCloseableList; import io.deephaven.util.datastructures.hash.HashMapK4V4; import io.deephaven.util.datastructures.hash.HashMapLockFreeK4V4; -import io.deephaven.engine.table.impl.sources.RedirectedColumnSource; -import io.deephaven.engine.table.impl.sources.SwitchColumnSource; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.engine.table.impl.sources.chunkcolumnsource.LongChunkColumnSource; -import io.deephaven.engine.table.impl.util.*; -import io.deephaven.util.SafeCloseableList; - import org.apache.commons.lang3.mutable.MutableObject; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.Arrays; import java.util.LinkedHashMap; @@ -40,21 +42,29 @@ public class SortOperation implements QueryTable.MemoizableOperation private final SortPair[] sortPairs; private final SortingOrder[] sortOrder; private final String[] sortColumnNames; + /** Stores original column sources. */ + private final ColumnSource>[] originalSortColumns; + /** Stores reinterpreted column sources. */ private final ColumnSource>[] sortColumns; + private final DataIndex dataIndex; + public SortOperation(QueryTable parent, SortPair[] sortPairs) { this.parent = parent; this.sortPairs = sortPairs; this.sortOrder = Arrays.stream(sortPairs).map(SortPair::getOrder).toArray(SortingOrder[]::new); this.sortColumnNames = Arrays.stream(sortPairs).map(SortPair::getColumn).toArray(String[]::new); + // noinspection unchecked + originalSortColumns = new ColumnSource[sortColumnNames.length]; // noinspection unchecked sortColumns = new ColumnSource[sortColumnNames.length]; for (int ii = 0; ii < sortColumnNames.length; ++ii) { + originalSortColumns[ii] = parent.getColumnSource(sortColumnNames[ii]); // noinspection unchecked - sortColumns[ii] = (ColumnSource>) ReinterpretUtils - .maybeConvertToPrimitive(parent.getColumnSource(sortColumnNames[ii])); + sortColumns[ii] = + (ColumnSource>) ReinterpretUtils.maybeConvertToPrimitive(originalSortColumns[ii]); Require.requirement( Comparable.class.isAssignableFrom(sortColumns[ii].getType()) @@ -64,6 +74,9 @@ public SortOperation(QueryTable parent, SortPair[] sortPairs) { } parent.assertSortable(sortColumnNames); + + // This sort operation might leverage a data index. + dataIndex = optimalIndex(parent); } @Override @@ -83,29 +96,33 @@ public MemoizedOperationKey getMemoizedOperationKey() { @Override public OperationSnapshotControl newSnapshotControl(QueryTable queryTable) { - return new OperationSnapshotControl(queryTable) { - @Override - public synchronized boolean snapshotCompletedConsistently( - final long afterClockValue, - final boolean usedPreviousValues) { - final boolean success = super.snapshotCompletedConsistently(afterClockValue, usedPreviousValues); - if (success) { - QueryTable.startTrackingPrev(resultTable.getColumnSources()); - if (sortMapping.isWritable()) { - sortMapping.writableCast().startTrackingPrevValues(); - } - } - return success; - } - }; + return dataIndex != null + ? new OperationSnapshotControlEx(queryTable, dataIndex.table()) + : new OperationSnapshotControl(queryTable); + } + + /** + * Returns the optimal data index for the supplied table, or null if no index is available. The ideal index would + * contain all key columns but matching the first column is still useful. + */ + @Nullable + private DataIndex optimalIndex(final Table inputTable) { + final DataIndex full = DataIndexer.getDataIndex(inputTable, sortColumnNames); + if (full != null) { + // We have an index for all sort columns. + return full; + } + // Return an index for the first column (if one exists) or null. + return DataIndexer.getDataIndex(inputTable, sortColumnNames[0]); } private static boolean alreadySorted(final QueryTable parent, @NotNull final SortHelpers.SortMapping sortedKeys) { if (sortedKeys.size() == 0) { return true; } - final RowSet.Iterator it = parent.getRowSet().iterator(); - return sortedKeys.forEachLong(currentKey -> currentKey == it.nextLong()); + try (RowSet.Iterator it = parent.getRowSet().iterator()) { + return sortedKeys.forEachLong(currentKey -> currentKey == it.nextLong()); + } } @NotNull @@ -160,6 +177,11 @@ private Result blinkTableSort(@NotNull final SortHelpers.SortMapping resultTable.setFlat(); setSorted(resultTable); + QueryTable.startTrackingPrev(resultTable.getColumnSources()); + if (sortMapping.isWritable()) { + sortMapping.writableCast().startTrackingPrevValues(); + } + final TableUpdateListener resultListener = new BaseTable.ListenerImpl("Stream sort listener", parent, resultTable) { @Override @@ -173,7 +195,8 @@ public void onUpdate(@NotNull final TableUpdate upstream) { } final SortHelpers.SortMapping updateSortedKeys = - SortHelpers.getSortedKeys(sortOrder, sortColumns, upstream.added(), false, false); + SortHelpers.getSortedKeys(sortOrder, originalSortColumns, sortColumns, null, + upstream.added(), false, false); final LongChunkColumnSource recycled = recycledInnerRedirectionSource.getValue(); recycledInnerRedirectionSource.setValue(null); final LongChunkColumnSource updateInnerRedirectSource = @@ -212,31 +235,30 @@ private QueryTable withSorted(QueryTable table) { public Result initialize(boolean usePrev, long beforeClock) { if (!parent.isRefreshing()) { final SortHelpers.SortMapping sortedKeys = - SortHelpers.getSortedKeys(sortOrder, sortColumns, parent.getRowSet(), false); + SortHelpers.getSortedKeys(sortOrder, originalSortColumns, sortColumns, dataIndex, + parent.getRowSet(), false); return new Result<>(historicalSort(sortedKeys)); } if (parent.isBlink()) { - try (final RowSet prevIndex = usePrev ? parent.getRowSet().copyPrev() : null) { - final RowSet indexToUse = usePrev ? prevIndex : parent.getRowSet(); - final SortHelpers.SortMapping sortedKeys = - SortHelpers.getSortedKeys(sortOrder, sortColumns, indexToUse, usePrev); - return blinkTableSort(sortedKeys); - } + final RowSet rowSetToUse = usePrev ? parent.getRowSet().prev() : parent.getRowSet(); + final SortHelpers.SortMapping sortedKeys = SortHelpers.getSortedKeys( + sortOrder, originalSortColumns, sortColumns, dataIndex, rowSetToUse, usePrev); + return blinkTableSort(sortedKeys); } try (final SafeCloseableList closer = new SafeCloseableList()) { // reset the sort data structures that we share between invocations final Map> resultMap = new LinkedHashMap<>(); - final RowSet rowSetToSort = usePrev ? closer.add(parent.getRowSet().copyPrev()) : parent.getRowSet(); + final RowSet rowSetToSort = usePrev ? parent.getRowSet().prev() : parent.getRowSet(); if (rowSetToSort.size() >= Integer.MAX_VALUE) { throw new UnsupportedOperationException("Can not perform ticking sort for table larger than " + Integer.MAX_VALUE + " rows, table is" + rowSetToSort.size()); } - final long[] sortedKeys = - SortHelpers.getSortedKeys(sortOrder, sortColumns, rowSetToSort, usePrev).getArrayMapping(); + final long[] sortedKeys = SortHelpers.getSortedKeys( + sortOrder, originalSortColumns, sortColumns, dataIndex, rowSetToSort, usePrev).getArrayMapping(); final HashMapK4V4 reverseLookup = new HashMapLockFreeK4V4(sortedKeys.length, .75f, -3); sortMapping = SortHelpers.createSortRowRedirection(); @@ -281,13 +303,19 @@ public Result initialize(boolean usePrev, long beforeClock) { return outerRowKey == reverseLookup.getNoEntryValue() ? RowSequence.NULL_ROW_KEY : outerRowKey; }); - final SortListener listener = new SortListener(parent, resultTable, reverseLookup, sortColumns, sortOrder, + final SortListener listener = new SortListener(parent, resultTable, reverseLookup, + originalSortColumns, sortColumns, sortOrder, sortMapping.writableCast(), sortedColumnsToSortBy, parent.newModifiedColumnSetIdentityTransformer(resultTable), parent.newModifiedColumnSet(sortColumnNames)); setSorted(resultTable); + QueryTable.startTrackingPrev(resultTable.getColumnSources()); + if (sortMapping.isWritable()) { + sortMapping.writableCast().startTrackingPrevValues(); + } + return new Result<>(resultTable, listener); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java index 98f922b0f19..ade5aba24f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/SourceTable.java @@ -5,19 +5,22 @@ import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; +import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.TrackingWritableRowSet; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.TableUpdateListener; -import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.TableLocationProvider; +import io.deephaven.engine.table.impl.locations.TableLocationRemovedException; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; import io.deephaven.engine.table.impl.locations.impl.TableLocationSubscriptionBuffer; -import io.deephaven.engine.updategraph.LogicalClock; -import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.util.QueryConstants; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.TestUseOnly; import org.apache.commons.lang3.mutable.Mutable; import org.apache.commons.lang3.mutable.MutableObject; @@ -92,13 +95,15 @@ public abstract class SourceTable> exte this.updateSourceRegistrar = updateSourceRegistrar; final boolean isRefreshing = updateSourceRegistrar != null; - columnSourceManager = componentFactory.createColumnSourceManager(isRefreshing, ColumnToCodecMappings.EMPTY, - definition.getColumns() // NB: this is the *re-written* definition passed to the super-class - // constructor. - ); - if (isRefreshing) { - // NB: There's no reason to start out trying to group, if this is a refreshing table. - columnSourceManager.disableGrouping(); + try (final SafeCloseable ignored = isRefreshing ? LivenessScopeStack.open() : null) { + columnSourceManager = componentFactory.createColumnSourceManager( + isRefreshing, + ColumnToCodecMappings.EMPTY, + definition.getColumns() // This is the *re-written* definition passed to the super-class constructor + ); + if (isRefreshing) { + manage(columnSourceManager); + } } setRefreshing(isRefreshing); @@ -108,7 +113,7 @@ public abstract class SourceTable> exte /** * Force this table to determine its initial state (available locations, size, RowSet) if it hasn't already done so. */ - private void initialize() { + protected final void initialize() { initializeAvailableLocations(); initializeLocationSizes(); } @@ -128,8 +133,7 @@ public final void refresh() { } } - @SuppressWarnings("WeakerAccess") - protected final void initializeAvailableLocations() { + private void initializeAvailableLocations() { if (locationsInitialized) { return; } @@ -187,29 +191,22 @@ private void initializeLocationSizes() { QueryPerformanceRecorder.withNugget(description + ".initializeLocationSizes()", sizeForInstrumentation(), () -> { Assert.eqNull(rowSet, "rowSet"); - rowSet = refreshLocationSizes().toTracking(); + try { + rowSet = columnSourceManager.initialize(); + } catch (Exception e) { + throw new TableDataException("Error initializing location sizes", e); + } if (!isRefreshing()) { return; } - rowSet.initializePreviousValue(); - final long currentClockValue = getUpdateGraph().clock().currentValue(); - setLastNotificationStep(LogicalClock.getState(currentClockValue) == LogicalClock.State.Updating - ? LogicalClock.getStep(currentClockValue) - 1 - : LogicalClock.getStep(currentClockValue)); + initializeLastNotificationStep(getUpdateGraph().clock()); }); locationSizesInitialized = true; } } - private WritableRowSet refreshLocationSizes() { - try { - return columnSourceManager.refresh(); - } catch (Exception e) { - throw new TableDataException("Error refreshing location sizes", e); - } - } - private class LocationChangePoller extends InstrumentedTableUpdateSource { + private final TableLocationSubscriptionBuffer locationBuffer; private LocationChangePoller(@NotNull final TableLocationSubscriptionBuffer locationBuffer) { @@ -228,14 +225,14 @@ protected void instrumentedRefresh() { } maybeAddLocations(locationUpdate.getPendingAddedLocationKeys()); - // NB: This class previously had functionality to notify "location listeners", but it was never used. + // This class previously had functionality to notify "location listeners", but it was never used. // Resurrect from git history if needed. if (!locationSizesInitialized) { // We don't want to start polling size changes until the initial RowSet has been computed. return; } - final RowSet added = refreshLocationSizes(); + final RowSet added = columnSourceManager.refresh(); if (added.isEmpty()) { return; } @@ -244,6 +241,12 @@ protected void instrumentedRefresh() { notifyListeners(added, RowSetFactory.empty(), RowSetFactory.empty()); } + @Override + protected void onRefreshError(@NotNull final Exception error) { + super.onRefreshError(error); + // Be sure that the ColumnSourceManager is aware + columnSourceManager.deliverError(error, entry); + } } /** @@ -268,6 +271,8 @@ protected final QueryTable doCoalesce() { @Override public boolean subscribeForUpdates(@NotNull final TableUpdateListener listener) { + // This impl cannot call super.subscribeForUpdates(), because we must subscribe to the actual + // (uncoalesced) SourceTable. return addUpdateListenerUncoalesced(listener, lastNotificationStep); } }); @@ -286,8 +291,8 @@ public boolean subscribeForUpdates(@NotNull final TableUpdateListener listener) @Override protected void destroy() { - // NB: This implementation cannot call super.destroy() for the same reason as the swap - // listener + // This impl cannot call super.destroy() because we must unsubscribe from the actual + // (uncoalesced) SourceTable. removeUpdateListenerUncoalesced(this); } }; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java index 2688647f509..7ea30ae8394 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TableDefaults.java @@ -52,7 +52,6 @@ default Table meta() { List columnDataTypes = new ArrayList<>(); List columnTypes = new ArrayList<>(); List columnPartitioning = new ArrayList<>(); - List columnGrouping = new ArrayList<>(); for (ColumnDefinition cDef : getDefinition().getColumns()) { columnNames.add(cDef.getName()); final Class dataType = cDef.getDataType(); @@ -60,15 +59,13 @@ default Table meta() { columnDataTypes.add(dataTypeName == null ? dataType.getName() : dataTypeName); columnTypes.add(cDef.getColumnType().name()); columnPartitioning.add(cDef.isPartitioning()); - columnGrouping.add(cDef.isGrouping()); } - final String[] resultColumnNames = {"Name", "DataType", "ColumnType", "IsPartitioning", "IsGrouping"}; + final String[] resultColumnNames = {"Name", "DataType", "ColumnType", "IsPartitioning"}; final Object[] resultValues = { columnNames.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY), columnDataTypes.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY), columnTypes.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY), columnPartitioning.toArray(new Boolean[0]), - columnGrouping.toArray(new Boolean[0]), }; return new InMemoryTable(resultColumnNames, resultValues); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/TimeTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/TimeTable.java index aa7c53c8bde..c2dbedf33ee 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/TimeTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/TimeTable.java @@ -17,6 +17,7 @@ import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.DataIndex; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.sources.FillUnordered; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; @@ -29,7 +30,6 @@ import java.time.Duration; import java.time.Instant; import java.util.Collections; -import java.util.LinkedHashMap; import java.util.Map; import java.util.Objects; @@ -251,6 +251,7 @@ public WritableRowSet match( final boolean invertMatch, final boolean usePrev, final boolean caseInsensitive, + @Nullable final DataIndex dataIndex, @NotNull final RowSet selection, final Object... keys) { if (startTime == null) { @@ -284,14 +285,6 @@ public WritableRowSet match( return matching; } - @Override - public Map getValuesMapping(RowSet subRange) { - final Map result = new LinkedHashMap<>(); - subRange.forAllRowKeys( - ii -> result.put(computeInstant(ii), RowSetFactory.fromKeys(ii))); - return result; - } - @Override public boolean allowsReinterpret( @NotNull final Class alternateDataType) { @@ -401,6 +394,7 @@ public WritableRowSet match( final boolean invertMatch, final boolean usePrev, final boolean caseInsensitive, + @Nullable final DataIndex dataIndex, @NotNull final RowSet selection, final Object... keys) { if (startTime == null) { @@ -434,14 +428,6 @@ public WritableRowSet match( return matching; } - @Override - public Map getValuesMapping(RowSet subRange) { - final Map result = new LinkedHashMap<>(); - subRange.forAllRowKeys( - ii -> result.put(box(computeNanos(ii)), RowSetFactory.fromKeys(ii))); - return result; - } - @Override public boolean allowsReinterpret( @NotNull final Class alternateDataType) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/WhereListener.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/WhereListener.java index 1c92d020e66..8bf2354f70b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/WhereListener.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/WhereListener.java @@ -95,7 +95,7 @@ class WhereListener extends MergedListener { } @NotNull - private static List extractDependencies(@NotNull final WhereFilter[] filters) { + static List extractDependencies(@NotNull final WhereFilter[] filters) { return Stream.concat( Stream.of(filters) .filter(f -> f instanceof NotificationQueue.Dependency) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/WouldMatchOperation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/WouldMatchOperation.java index 49766528990..13c3c2d7bad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/WouldMatchOperation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/WouldMatchOperation.java @@ -7,6 +7,7 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.exceptions.UncheckedTableException; +import io.deephaven.engine.liveness.LivenessReferent; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; @@ -15,6 +16,7 @@ import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.SafeCloseableList; import org.apache.commons.lang3.mutable.MutableBoolean; import org.jetbrains.annotations.NotNull; @@ -29,43 +31,50 @@ * cell values if any of the underlying filters are dynamic, and change. */ public class WouldMatchOperation implements QueryTable.MemoizableOperation { + private static final RowSet EMPTY_INDEX = RowSetFactory.empty(); - private final List matchColumns; + private final QueryTable parent; + private final List matchColumns; + private final WhereFilter[] whereFilters; + private QueryTable resultTable; private ModifiedColumnSet.Transformer transformer; /** * Just a little helper to keep column stuff together. */ - private class ColumnHolder { - final WouldMatchPair wouldMatchPair; - IndexWrapperColumnSource column; + private static class ColumnHolder { + + private final WouldMatchPair wouldMatchPair; + private final WhereFilter filter; + + private IndexWrapperColumnSource column; - ColumnHolder(WouldMatchPair pair) { + private ColumnHolder(WouldMatchPair pair) { this.wouldMatchPair = pair; + filter = WhereFilter.of(wouldMatchPair.getFilter()); } - String getColumnName() { + private String getColumnName() { return wouldMatchPair.getColumnName(); } - WhereFilter getFilter() { - return WhereFilter.of(wouldMatchPair.getFilter()); + private WhereFilter getFilter() { + return filter; } } WouldMatchOperation(QueryTable parent, WouldMatchPair... filters) { this.parent = parent; matchColumns = Arrays.stream(filters).map(ColumnHolder::new).collect(Collectors.toList()); + whereFilters = matchColumns.stream().map(ColumnHolder::getFilter).toArray(WhereFilter[]::new); final List parentColumns = parent.getDefinition().getColumnNames(); - final List collidingColumns = matchColumns.stream() .map(ColumnHolder::getColumnName) .filter(parentColumns::contains) .collect(Collectors.toList()); - if (!collidingColumns.isEmpty()) { throw new UncheckedTableException( "The table already contains the following columns: " + String.join(", ", collidingColumns)); @@ -82,18 +91,35 @@ public String getLogPrefix() { return "match"; } + @Override + public SafeCloseable beginOperation(@NotNull final QueryTable parent) { + return Arrays.stream(whereFilters) + .map((final WhereFilter filter) -> { + filter.init(parent.getDefinition()); + // Ensure we gather the correct dependencies when building a snapshot control. + return filter.beginOperation(parent); + }).collect(SafeCloseableList.COLLECTOR); + } + + @Override + public OperationSnapshotControl newSnapshotControl(@NotNull final QueryTable queryTable) { + final List dependencies = WhereListener.extractDependencies(whereFilters); + if (dependencies.isEmpty()) { + return QueryTable.MemoizableOperation.super.newSnapshotControl(queryTable); + } + return new OperationSnapshotControlEx(queryTable, dependencies.toArray(NotificationQueue.Dependency[]::new)); + } + @Override public Result initialize(boolean usePrev, long beforeClock) { MutableBoolean anyRefreshing = new MutableBoolean(false); try (final SafeCloseableList closer = new SafeCloseableList()) { final RowSet fullRowSet = usePrev ? closer.add(parent.getRowSet().copyPrev()) : parent.getRowSet(); - - final List dependencies = new ArrayList<>(); final Map> newColumns = new LinkedHashMap<>(parent.getColumnSourceMap()); + matchColumns.forEach(holder -> { final WhereFilter filter = holder.getFilter(); - filter.init(parent.getDefinition()); final WritableRowSet result = filter.filter(fullRowSet, fullRowSet, parent, usePrev); holder.column = new IndexWrapperColumnSource( holder.getColumnName(), parent, result.toTracking(), filter); @@ -104,26 +130,18 @@ public Result initialize(boolean usePrev, long beforeClock) { "In match(), column " + holder.getColumnName() + " already exists in the table."); } - // Accumulate dependencies - if (filter instanceof NotificationQueue.Dependency) { - dependencies.add((NotificationQueue.Dependency) filter); - } else if (filter instanceof DependencyStreamProvider) { - ((DependencyStreamProvider) filter).getDependencyStream().forEach(dependencies::add); - } - if (filter.isRefreshing()) { anyRefreshing.setTrue(); } }); - this.resultTable = new QueryTable(parent.getRowSet(), newColumns); - + resultTable = new QueryTable(parent.getRowSet(), newColumns); transformer = parent.newModifiedColumnSetTransformer(resultTable, parent.getDefinition().getColumnNamesArray()); // Set up the column to be a listener for recomputes matchColumns.forEach(mc -> { - if (mc.getFilter() instanceof LivenessArtifact) { + if (mc.getFilter() instanceof LivenessReferent) { resultTable.manage((LivenessArtifact) mc.getFilter()); } mc.column.setResultTable(resultTable); @@ -136,7 +154,7 @@ public Result initialize(boolean usePrev, long beforeClock) { // If we're refreshing, our final listener needs to handle upstream updates from a recorder. final ListenerRecorder recorder = new ListenerRecorder("where(" + makeDescription() + ")", parent, resultTable); - final Listener listener = new Listener(recorder, dependencies); + final Listener listener = new Listener(recorder, WhereListener.extractDependencies(whereFilters)); recorder.setMergedListener(listener); eventualMergedListener = listener; @@ -145,7 +163,7 @@ public Result initialize(boolean usePrev, long beforeClock) { // If not, then we still need to update if any of our filters request updates. We'll use the // merge listener to handle that. Note that the filters themselves should set the table to // refreshing. - eventualMergedListener = new StaticListener(dependencies); + eventualMergedListener = new StaticListener(WhereListener.extractDependencies(whereFilters)); } if (eventualMergedListener != null) { @@ -317,11 +335,11 @@ public WritableRowSet match( final boolean invertMatch, final boolean usePrev, final boolean caseInsensitive, + @Nullable final DataIndex dataIndex, @NotNull final RowSet mapper, final Object... keys) { boolean hasFalse = false; boolean hasTrue = false; - boolean hasOther = false; for (Object key : keys) { if (key instanceof Boolean) { @@ -330,8 +348,6 @@ public WritableRowSet match( } else { hasFalse = true; } - } else { - hasOther = true; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/BucketedChunkedAjMergedListener.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/BucketedChunkedAjMergedListener.java index 1e04a421ae2..a01dd2442b9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/BucketedChunkedAjMergedListener.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/BucketedChunkedAjMergedListener.java @@ -193,7 +193,7 @@ public void process() { rowRedirection.removeAll(leftRemoved); - final SegmentedSortedArray leftSsa = asOfJoinStateManager.getLeftSsaOrIndex(slot, leftIndexOutput); + final SegmentedSortedArray leftSsa = asOfJoinStateManager.getLeftSsaOrRowSet(slot, leftIndexOutput); if (leftSsa == null) { leftIndexOutput.getValue().remove(leftRemoved); leftIndexOutput.setValue(null); @@ -239,7 +239,7 @@ public void process() { sequentialBuilders.ensureCapacity(relevantShift.size()); slots.ensureCapacity(relevantShift.size()); - final int shiftedSlotCount = asOfJoinStateManager.gatherShiftIndex(relevantShift, + final int shiftedSlotCount = asOfJoinStateManager.gatherShiftRowSet(relevantShift, leftKeySources, slots, sequentialBuilders); for (int slotIndex = 0; slotIndex < shiftedSlotCount; ++slotIndex) { @@ -252,7 +252,7 @@ public void process() { if ((state & ENTRY_RIGHT_MASK) == ENTRY_RIGHT_IS_EMPTY) { // if the left is empty, we should be a RowSet entry rather than an SSA, and we can // not be empty, because we are responsive - final WritableRowSet leftRowSet = asOfJoinStateManager.getLeftIndex(slot); + final WritableRowSet leftRowSet = asOfJoinStateManager.getLeftRowSet(slot); shiftDataForSlot.apply(leftRowSet); shiftedRowSet.close(); leftRowSet.compact(); @@ -322,7 +322,7 @@ public void process() { try (final RowSet rightRemoved = indexFromBuilder(slotIndex)) { final SegmentedSortedArray rightSsa = - asOfJoinStateManager.getRightSsaOrIndex(slot, rowSetOutput); + asOfJoinStateManager.getRightSsaOrRowSet(slot, rowSetOutput); if (rightSsa == null) { rowSetOutput.getValue().remove(rightRemoved); continue; @@ -372,7 +372,7 @@ public void process() { sequentialBuilders.ensureCapacity(relevantShift.size()); slots.ensureCapacity(relevantShift.size()); - final int shiftedSlotCount = asOfJoinStateManager.gatherShiftIndex(relevantShift, + final int shiftedSlotCount = asOfJoinStateManager.gatherShiftRowSet(relevantShift, rightKeySources, slots, sequentialBuilders); for (int slotIndex = 0; slotIndex < shiftedSlotCount; ++slotIndex) { @@ -392,7 +392,7 @@ public void process() { if (leftSsa == null) { // if the left is empty, we should be a RowSet entry rather than an SSA, and we can // not be empty, because we are responsive - final WritableRowSet rightRowSet = asOfJoinStateManager.getRightIndex(slot); + final WritableRowSet rightRowSet = asOfJoinStateManager.getRightRowSet(slot); shiftDataForSlot.apply(rightRowSet); shiftedRowSet.close(); rightRowSet.compact(); @@ -478,26 +478,26 @@ public void process() { makeRightIndex = true; break; - case ENTRY_LEFT_IS_EMPTY | ENTRY_RIGHT_IS_INDEX: + case ENTRY_LEFT_IS_EMPTY | ENTRY_RIGHT_IS_ROWSET: case ENTRY_LEFT_IS_EMPTY | ENTRY_RIGHT_IS_BUILDER: updateRightIndex = true; break; case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_EMPTY: - case ENTRY_LEFT_IS_INDEX | ENTRY_RIGHT_IS_EMPTY: + case ENTRY_LEFT_IS_ROWSET | ENTRY_RIGHT_IS_EMPTY: case ENTRY_LEFT_IS_SSA | ENTRY_RIGHT_IS_EMPTY: processInitial = true; break; - case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_INDEX: - case ENTRY_LEFT_IS_INDEX | ENTRY_RIGHT_IS_INDEX: - case ENTRY_LEFT_IS_SSA | ENTRY_RIGHT_IS_INDEX: + case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_ROWSET: + case ENTRY_LEFT_IS_ROWSET | ENTRY_RIGHT_IS_ROWSET: + case ENTRY_LEFT_IS_SSA | ENTRY_RIGHT_IS_ROWSET: case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_BUILDER: - case ENTRY_LEFT_IS_INDEX | ENTRY_RIGHT_IS_BUILDER: + case ENTRY_LEFT_IS_ROWSET | ENTRY_RIGHT_IS_BUILDER: case ENTRY_LEFT_IS_SSA | ENTRY_RIGHT_IS_BUILDER: case ENTRY_LEFT_IS_EMPTY | ENTRY_RIGHT_IS_SSA: case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_SSA: - case ENTRY_LEFT_IS_INDEX | ENTRY_RIGHT_IS_SSA: + case ENTRY_LEFT_IS_ROWSET | ENTRY_RIGHT_IS_SSA: throw new IllegalStateException(); case ENTRY_LEFT_IS_SSA | ENTRY_RIGHT_IS_SSA: @@ -505,11 +505,11 @@ public void process() { } if (makeRightIndex) { - asOfJoinStateManager.setRightIndex(slot, rightAdded); + asOfJoinStateManager.setRightRowSet(slot, rightAdded); continue; } if (updateRightIndex) { - asOfJoinStateManager.getRightIndex(slot).insert(rightAdded); + asOfJoinStateManager.getRightRowSet(slot).insert(rightAdded); rightAdded.close(); continue; } @@ -645,26 +645,26 @@ public void process() { break; case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_EMPTY: - case ENTRY_LEFT_IS_INDEX | ENTRY_RIGHT_IS_EMPTY: + case ENTRY_LEFT_IS_ROWSET | ENTRY_RIGHT_IS_EMPTY: updateLeftIndex = true; break; case ENTRY_LEFT_IS_SSA | ENTRY_RIGHT_IS_EMPTY: throw new IllegalStateException(); case ENTRY_LEFT_IS_EMPTY | ENTRY_RIGHT_IS_BUILDER: - case ENTRY_LEFT_IS_EMPTY | ENTRY_RIGHT_IS_INDEX: + case ENTRY_LEFT_IS_EMPTY | ENTRY_RIGHT_IS_ROWSET: processInitial = true; break; case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_BUILDER: - case ENTRY_LEFT_IS_INDEX | ENTRY_RIGHT_IS_BUILDER: + case ENTRY_LEFT_IS_ROWSET | ENTRY_RIGHT_IS_BUILDER: case ENTRY_LEFT_IS_SSA | ENTRY_RIGHT_IS_BUILDER: - case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_INDEX: - case ENTRY_LEFT_IS_INDEX | ENTRY_RIGHT_IS_INDEX: - case ENTRY_LEFT_IS_SSA | ENTRY_RIGHT_IS_INDEX: + case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_ROWSET: + case ENTRY_LEFT_IS_ROWSET | ENTRY_RIGHT_IS_ROWSET: + case ENTRY_LEFT_IS_SSA | ENTRY_RIGHT_IS_ROWSET: case ENTRY_LEFT_IS_EMPTY | ENTRY_RIGHT_IS_SSA: case ENTRY_LEFT_IS_BUILDER | ENTRY_RIGHT_IS_SSA: - case ENTRY_LEFT_IS_INDEX | ENTRY_RIGHT_IS_SSA: + case ENTRY_LEFT_IS_ROWSET | ENTRY_RIGHT_IS_SSA: throw new IllegalStateException( "Bad state: " + state + ", slot=" + slot + ", slotIndex=" + slotIndex); @@ -675,11 +675,11 @@ public void process() { final RowSet leftAdded = indexFromBuilder(slotIndex); if (makeLeftIndex) { - asOfJoinStateManager.setLeftIndex(slot, leftAdded); + asOfJoinStateManager.setLeftRowSet(slot, leftAdded); continue; } if (updateLeftIndex) { - final WritableRowSet leftRowSet = asOfJoinStateManager.getLeftIndex(slot); + final WritableRowSet leftRowSet = asOfJoinStateManager.getLeftRowSet(slot); leftRowSet.insert(leftAdded); leftAdded.close(); leftRowSet.compact(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/RightIncrementalAsOfJoinStateManagerTypedBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/RightIncrementalAsOfJoinStateManagerTypedBase.java index 99bc7ce06f7..eec82972695 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/RightIncrementalAsOfJoinStateManagerTypedBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/RightIncrementalAsOfJoinStateManagerTypedBase.java @@ -207,7 +207,7 @@ protected static void addToBuilder(ImmutableObjectArraySource[] rightSo @Override public int markForRemoval(RowSet restampRemovals, ColumnSource[] sources, IntegerArraySource slots, ObjectArraySource sequentialBuilders) { - return accumulateIndices(restampRemovals, sources, slots, sequentialBuilders, true); + return accumulateRowSets(restampRemovals, sources, slots, sequentialBuilders, true); } @Override public int probeAdditions(RowSet restampAdditions, ColumnSource[] sources, IntegerArraySource slots, ObjectArraySource sequentialBuilders) { - return accumulateIndices(restampAdditions, sources, slots, sequentialBuilders, false); + return accumulateRowSets(restampAdditions, sources, slots, sequentialBuilders, false); } @Override - public int gatherShiftIndex(RowSet restampAdditions, ColumnSource[] sources, IntegerArraySource slots, + public int gatherShiftRowSet(RowSet restampAdditions, ColumnSource[] sources, IntegerArraySource slots, ObjectArraySource sequentialBuilders) { - return accumulateIndices(restampAdditions, sources, slots, sequentialBuilders, true); + return accumulateRowSets(restampAdditions, sources, slots, sequentialBuilders, true); } public int gatherModifications(RowSet restampAdditions, ColumnSource[] sources, IntegerArraySource slots, ObjectArraySource sequentialBuilders) { - return accumulateIndices(restampAdditions, sources, slots, sequentialBuilders, false); + return accumulateRowSets(restampAdditions, sources, slots, sequentialBuilders, false); } protected void probeTable( @@ -425,16 +425,16 @@ public void doProbe(RowSequence chunkOk, Chunk[] sourceKeyChunks) { } @Override - public void probeRightInitial(RowSequence rightIndex, ColumnSource[] rightSources) { - if (rightIndex.isEmpty()) { + public void probeRightInitial(RowSequence rowsToProbe, ColumnSource[] rightSources) { + if (rowsToProbe.isEmpty()) { return; } - try (final ProbeContext pc = makeProbeContext(rightSources, rightIndex.size())) { - probeTable(pc, rightIndex, false, rightSources, new RightProbeHandler()); + try (final ProbeContext pc = makeProbeContext(rightSources, rowsToProbe.size())) { + probeTable(pc, rowsToProbe, false, rightSources, new RightProbeHandler()); } } - private int accumulateIndices(RowSet rowSet, ColumnSource[] sources, IntegerArraySource slots, + private int accumulateRowSets(RowSet rowSet, ColumnSource[] sources, IntegerArraySource slots, ObjectArraySource sequentialBuilders, boolean usePrev) { resetCookie(); @@ -479,13 +479,20 @@ public int getTableSize() { } @Override - public WritableRowSet getAndClearLeftIndex(int slot) { - final RowSetBuilderSequential builder = (RowSetBuilderSequential) leftRowSetSource.getUnsafe(slot); - leftRowSetSource.set(slot, null); - if (builder == null) { + public WritableRowSet getAndClearLeftRowSet(int slot) { + final Object o = leftRowSetSource.getUnsafe(slot); + if (o == null) { return null; } - return builder.build(); + leftRowSetSource.set(slot, null); + + // This might already be a row set loaded from an index table. If so, just return it. + if (o instanceof WritableRowSet) { + return (WritableRowSet) o; + } else { + final RowSetBuilderSequential builder = (RowSetBuilderSequential) o; + return builder.build(); + } } @Override @@ -542,8 +549,8 @@ public SegmentedSortedArray getRightSsa(int slot, Function rowSetSource; if ((slot & ALTERNATE_SWITCH_MASK) == mainInsertMask) { @@ -569,19 +576,19 @@ public WritableRowSet getRightIndex(int slot) { slot = slot & ALTERNATE_INNER_MASK; final byte entryType = source.getUnsafe(slot); - if ((entryType & ENTRY_RIGHT_MASK) == ENTRY_RIGHT_IS_INDEX) { + if ((entryType & ENTRY_RIGHT_MASK) == ENTRY_RIGHT_IS_ROWSET) { return (WritableRowSet) rowSetSource.getUnsafe(slot); } else if ((entryType & ENTRY_RIGHT_MASK) == ENTRY_RIGHT_IS_BUILDER) { final WritableRowSet rowSet = ((RowSetBuilderSequential) rowSetSource.getUnsafe(slot)).build(); rowSetSource.set(slot, rowSet); - source.set(slot, (byte) ((entryType & ENTRY_LEFT_MASK) | ENTRY_RIGHT_IS_INDEX)); + source.set(slot, (byte) ((entryType & ENTRY_LEFT_MASK) | ENTRY_RIGHT_IS_ROWSET)); return rowSet; } throw new IllegalStateException(); } @Override - public WritableRowSet getLeftIndex(int slot) { + public WritableRowSet getLeftRowSet(int slot) { final ImmutableByteArraySource source; final ImmutableObjectArraySource rowSetSource; if ((slot & ALTERNATE_SWITCH_MASK) == mainInsertMask) { @@ -595,19 +602,19 @@ public WritableRowSet getLeftIndex(int slot) { slot = slot & ALTERNATE_INNER_MASK; final byte entryType = source.getUnsafe(slot); - if ((entryType & ENTRY_LEFT_MASK) == ENTRY_LEFT_IS_INDEX) { + if ((entryType & ENTRY_LEFT_MASK) == ENTRY_LEFT_IS_ROWSET) { return (WritableRowSet) rowSetSource.getUnsafe(slot); } else if ((entryType & ENTRY_LEFT_MASK) == ENTRY_LEFT_IS_BUILDER) { final WritableRowSet rowSet = ((RowSetBuilderSequential) rowSetSource.getUnsafe(slot)).build(); rowSetSource.set(slot, rowSet); - source.set(slot, (byte) ((entryType & ENTRY_RIGHT_MASK) | ENTRY_LEFT_IS_INDEX)); + source.set(slot, (byte) ((entryType & ENTRY_RIGHT_MASK) | ENTRY_LEFT_IS_ROWSET)); return rowSet; } throw new IllegalStateException(); } @Override - public void setLeftIndex(int slot, RowSet rowSet) { + public void setLeftRowSet(int slot, RowSet rowSet) { final ImmutableByteArraySource source; final ImmutableObjectArraySource rowSetSource; if ((slot & ALTERNATE_SWITCH_MASK) == mainInsertMask) { @@ -622,7 +629,7 @@ public void setLeftIndex(int slot, RowSet rowSet) { final byte entryType = source.getUnsafe(slot); if ((entryType & ENTRY_LEFT_MASK) == ENTRY_LEFT_IS_EMPTY) { - source.set(slot, (byte) ((entryType & ENTRY_RIGHT_MASK) | ENTRY_LEFT_IS_INDEX)); + source.set(slot, (byte) ((entryType & ENTRY_RIGHT_MASK) | ENTRY_LEFT_IS_ROWSET)); rowSetSource.set(slot, rowSet); return; } @@ -630,7 +637,7 @@ public void setLeftIndex(int slot, RowSet rowSet) { } @Override - public void setRightIndex(int slot, RowSet rowSet) { + public void setRightRowSet(int slot, RowSet rowSet) { final ImmutableByteArraySource source; final ImmutableObjectArraySource rowSetSource; if ((slot & ALTERNATE_SWITCH_MASK) == mainInsertMask) { @@ -645,13 +652,67 @@ public void setRightIndex(int slot, RowSet rowSet) { final byte entryType = source.getUnsafe(slot); if ((entryType & ENTRY_RIGHT_MASK) == ENTRY_RIGHT_IS_EMPTY) { - source.set(slot, (byte) ((entryType & ENTRY_LEFT_MASK) | ENTRY_RIGHT_IS_INDEX)); + source.set(slot, (byte) ((entryType & ENTRY_LEFT_MASK) | ENTRY_RIGHT_IS_ROWSET)); rowSetSource.set(slot, rowSet); return; } throw new IllegalStateException(); } + @Override + public void populateRightRowSetsFromIndexTable( + @NotNull final IntegerArraySource slots, + final int slotCount, + @NotNull final ColumnSource rowSetSource) { + for (int slotIndex = 0; slotIndex < slotCount; ++slotIndex) { + final int slot = slots.getInt(slotIndex); + + final RowSetBuilderSequential sequentialBuilder = + (RowSetBuilderSequential) rightRowSetSource.getUnsafe(slot); + if (sequentialBuilder == null) { + continue; + } + final WritableRowSet rs = sequentialBuilder.build(); + final byte entryType = stateSource.getUnsafe(slot); + if (rs.isEmpty()) { + stateSource.set(slot, (byte) ((entryType & ENTRY_LEFT_MASK) | ENTRY_RIGHT_IS_EMPTY)); + rs.close(); + } else if (rs.size() == 1) { + // Set a copy of the RowSet into the row set source because the original is owned by the index. + rightRowSetSource.set(slot, rowSetSource.get(rs.firstRowKey()).copy()); + stateSource.set(slot, (byte) ((entryType & ENTRY_LEFT_MASK) | ENTRY_RIGHT_IS_ROWSET)); + } else { + throw new IllegalStateException("Index-built row set should have exactly one value: " + rs); + } + } + } + + @Override + public void populateLeftRowSetsFromIndexTable(IntegerArraySource slots, int slotCount, + ColumnSource rowSetSource) { + for (int slotIndex = 0; slotIndex < slotCount; ++slotIndex) { + final int slot = slots.getInt(slotIndex); + + final RowSetBuilderSequential sequentialBuilder = + (RowSetBuilderSequential) leftRowSetSource.getUnsafe(slot); + if (sequentialBuilder == null) { + continue; + } + final WritableRowSet rs = sequentialBuilder.build(); + final byte entryType = stateSource.getUnsafe(slot); + if (rs.isEmpty()) { + stateSource.set(slot, (byte) ((entryType & ENTRY_RIGHT_MASK) | ENTRY_LEFT_IS_EMPTY)); + rs.close(); + } else if (rs.size() == 1) { + // Set a copy of the RowSet into the row set source because the original is owned by the index. + leftRowSetSource.set(slot, rowSetSource.get(rs.firstRowKey()).copy()); + stateSource.set(slot, (byte) ((entryType & ENTRY_RIGHT_MASK) | ENTRY_LEFT_IS_ROWSET)); + } else { + throw new IllegalStateException("Index-built row set should have exactly one value: " + rs); + } + } + } + @Override public SegmentedSortedArray getLeftSsa(int slot) { final ImmutableByteArraySource source; @@ -695,8 +756,8 @@ public SegmentedSortedArray getLeftSsa(int slot, Function indexOutput) { + public SegmentedSortedArray getLeftSsaOrRowSet(int slot, MutableObject indexOutput) { final ImmutableByteArraySource source; final ImmutableObjectArraySource rowSetSource; if ((slot & ALTERNATE_SWITCH_MASK) == mainInsertMask) { @@ -719,13 +780,12 @@ public SegmentedSortedArray getLeftSsaOrIndex(int slot, MutableObject indexOutput) { + public SegmentedSortedArray getRightSsaOrRowSet(int slot, MutableObject indexOutput) { final ImmutableByteArraySource source; final ImmutableObjectArraySource rowSetSource; if ((slot & ALTERNATE_SWITCH_MASK) == mainInsertMask) { @@ -739,32 +799,31 @@ public SegmentedSortedArray getRightSsaOrIndex(int slot, MutableObject indexOutput, long location, + private static SegmentedSortedArray getSsaOrRowSet(MutableObject indexOutput, long location, byte entryType, ImmutableObjectArraySource sideSource, ImmutableByteArraySource stateSource, - byte stateValueForIndex) { + byte state) { switch (entryType) { case ENTRY_RIGHT_IS_SSA: return (SegmentedSortedArray) sideSource.getUnsafe(location); - case ENTRY_RIGHT_IS_INDEX: + case ENTRY_RIGHT_IS_ROWSET: indexOutput.setValue((WritableRowSet) sideSource.getUnsafe(location)); return null; case ENTRY_RIGHT_IS_EMPTY: { final WritableRowSet emptyRowSet = RowSetFactory.empty(); sideSource.set(location, emptyRowSet); - stateSource.set(location, stateValueForIndex); + stateSource.set(location, state); indexOutput.setValue(emptyRowSet); return null; } case ENTRY_RIGHT_IS_BUILDER: { final WritableRowSet rowSet = ((RowSetBuilderSequential) sideSource.getUnsafe(location)).build(); sideSource.set(location, rowSet); - stateSource.set(location, stateValueForIndex); + stateSource.set(location, state); indexOutput.setValue(rowSet); return null; } @@ -783,22 +842,23 @@ private SegmentedSortedArray makeSsaFromBuilder(int slot, Function ssaFactory, ImmutableObjectArraySource ssaSource, ImmutableByteArraySource stateSource, byte newState) { - return makeSsaFromIndex(slot, ssaFactory, ssaSource, stateSource, newState, RowSetFactory.empty()); + return makeSsaFromRowSet(slot, ssaFactory, ssaSource, stateSource, newState, RowSetFactory.empty()); } @Nullable - private SegmentedSortedArray makeSsaFromIndex(int slot, Function ssaFactory, + private SegmentedSortedArray makeSsaFromRowSet(int slot, Function ssaFactory, ImmutableObjectArraySource ssaSource, ImmutableByteArraySource stateSource, byte newState) { - return makeSsaFromIndex(slot, ssaFactory, ssaSource, stateSource, newState, (RowSet) ssaSource.getUnsafe(slot)); + return makeSsaFromRowSet(slot, ssaFactory, ssaSource, stateSource, newState, + (RowSet) ssaSource.getUnsafe(slot)); } - private SegmentedSortedArray makeSsaFromIndex(int slot, Function ssaFactory, + private SegmentedSortedArray makeSsaFromRowSet(int slot, Function ssaFactory, ImmutableObjectArraySource ssaSource, ImmutableByteArraySource stateSource, byte newState, RowSet rowSet) { stateSource.set(slot, newState); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/RightIncrementalHashedAsOfJoinStateManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/RightIncrementalHashedAsOfJoinStateManager.java index 5044867097d..ee00c0676dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/RightIncrementalHashedAsOfJoinStateManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/RightIncrementalHashedAsOfJoinStateManager.java @@ -6,7 +6,8 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.RightIncrementalAsOfJoinStateManager; -import io.deephaven.engine.table.impl.sources.*; +import io.deephaven.engine.table.impl.sources.IntegerArraySource; +import io.deephaven.engine.table.impl.sources.ObjectArraySource; import io.deephaven.engine.table.impl.ssa.SegmentedSortedArray; import org.apache.commons.lang3.mutable.MutableObject; import org.jetbrains.annotations.NotNull; @@ -22,13 +23,13 @@ protected RightIncrementalHashedAsOfJoinStateManager(ColumnSource[] keySource public static final byte ENTRY_RIGHT_IS_EMPTY = 0x0; public static final byte ENTRY_RIGHT_IS_BUILDER = 0x1; public static final byte ENTRY_RIGHT_IS_SSA = 0x2; - public static final byte ENTRY_RIGHT_IS_INDEX = 0x3; + public static final byte ENTRY_RIGHT_IS_ROWSET = 0x3; public static final byte ENTRY_LEFT_MASK = 0x30; public static final byte ENTRY_LEFT_IS_EMPTY = 0x00; public static final byte ENTRY_LEFT_IS_BUILDER = 0x10; public static final byte ENTRY_LEFT_IS_SSA = 0x20; - public static final byte ENTRY_LEFT_IS_INDEX = 0x30; + public static final byte ENTRY_LEFT_IS_ROWSET = 0x30; protected void addToSequentialBuilder(long slot, @NotNull ObjectArraySource sequentialBuilders, long indexKey) { @@ -54,7 +55,7 @@ public abstract int buildFromLeftSide(RowSequence leftRowSet, ColumnSource[] public abstract int buildFromRightSide(RowSequence rightRowSet, ColumnSource[] rightSources, @NotNull IntegerArraySource addedSlots, int usedSlots); - public abstract void probeRightInitial(RowSequence rightIndex, ColumnSource[] rightSources); + public abstract void probeRightInitial(RowSequence rowsToProbe, ColumnSource[] rightSources); public abstract int probeAdditions(RowSet restampAdditions, ColumnSource[] sources, IntegerArraySource slots, ObjectArraySource sequentialBuilders); @@ -70,24 +71,30 @@ public abstract int buildAdditions(boolean isLeftSide, RowSet additions, ColumnS public abstract SegmentedSortedArray getLeftSsa(int slot, Function ssaFactory); - public abstract SegmentedSortedArray getLeftSsaOrIndex(int slot, MutableObject indexOutput); + public abstract SegmentedSortedArray getLeftSsaOrRowSet(int slot, MutableObject indexOutput); - public abstract SegmentedSortedArray getRightSsaOrIndex(int slot, MutableObject indexOutput); + public abstract SegmentedSortedArray getRightSsaOrRowSet(int slot, MutableObject indexOutput); - public abstract void setRightIndex(int slot, RowSet rowSet); + public abstract void setRightRowSet(int slot, RowSet rowSet); - public abstract void setLeftIndex(int slot, RowSet rowSet); + public abstract void setLeftRowSet(int slot, RowSet rowSet); - public abstract WritableRowSet getLeftIndex(int slot); + public abstract WritableRowSet getLeftRowSet(int slot); - public abstract WritableRowSet getRightIndex(int slot); + public abstract WritableRowSet getRightRowSet(int slot); - public abstract WritableRowSet getAndClearLeftIndex(int slot); + public abstract void populateRightRowSetsFromIndexTable(IntegerArraySource slots, int slotCount, + ColumnSource rowSetSource); + + public abstract void populateLeftRowSetsFromIndexTable(IntegerArraySource slots, int slotCount, + ColumnSource rowSetSource); + + public abstract WritableRowSet getAndClearLeftRowSet(int slot); public abstract int markForRemoval(RowSet restampRemovals, ColumnSource[] sources, IntegerArraySource slots, ObjectArraySource sequentialBuilders); - public abstract int gatherShiftIndex(RowSet restampAdditions, ColumnSource[] sources, IntegerArraySource slots, + public abstract int gatherShiftRowSet(RowSet restampAdditions, ColumnSource[] sources, IntegerArraySource slots, ObjectArraySource sequentialBuilders); public abstract int gatherModifications(RowSet restampAdditions, ColumnSource[] sources, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/StaticAsOfJoinStateManagerTypedBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/StaticAsOfJoinStateManagerTypedBase.java index aba60c6caf0..50806b6827c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/StaticAsOfJoinStateManagerTypedBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/StaticAsOfJoinStateManagerTypedBase.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.asofjoin; +import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ChunkType; @@ -12,7 +13,6 @@ import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.sources.IntegerArraySource; import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; import io.deephaven.engine.table.impl.sources.immutable.ImmutableObjectArraySource; import io.deephaven.engine.table.impl.util.TypedHasherUtil; import io.deephaven.engine.table.impl.util.TypedHasherUtil.BuildOrProbeContext.BuildContext; @@ -101,7 +101,8 @@ ProbeContext makeProbeContext(ColumnSource[] buildSources, long maxSize) { return new ProbeContext(buildSources, (int) Math.min(CHUNK_SIZE, maxSize)); } - static boolean addIndex(ImmutableObjectArraySource source, int location, long keyToAdd) { + static boolean addKeyToSlot(ImmutableObjectArraySource source, int location, + long keyToAdd) { boolean addedSlot = false; RowSetBuilderSequential builder = source.getUnsafe(location); if (builder == null) { @@ -115,13 +116,13 @@ static boolean addIndex(ImmutableObjectArraySource sour /** * Returns true if this is the first left row key added to this slot. */ - protected boolean addLeftIndex(int tableLocation, long keyToAdd) { - return addIndex(leftRowSetSource, tableLocation, keyToAdd); + protected boolean addLeftKey(int tableLocation, long keyToAdd) { + return addKeyToSlot(leftRowSetSource, tableLocation, keyToAdd); } - protected void addRightIndex(int tableLocation, long keyToAdd) { + protected void addRightKey(int tableLocation, long keyToAdd) { // noinspection unchecked - addIndex((ImmutableObjectArraySource) rightRowSetSource, tableLocation, keyToAdd); + addKeyToSlot((ImmutableObjectArraySource) rightRowSetSource, tableLocation, keyToAdd); } protected void buildTable( @@ -302,7 +303,7 @@ public int getTableSize() { * @return the RowSet for this slot */ @Override - public RowSet getLeftIndex(int slot) { + public RowSet getLeftRowSet(int slot) { RowSetBuilderSequential builder = (RowSetBuilderSequential) leftRowSetSource.getAndSetUnsafe(slot, null); if (builder == null) { return null; @@ -311,60 +312,64 @@ public RowSet getLeftIndex(int slot) { } @Override - public RowSet getRightIndex(int slot) { + public RowSet getRightRowset(int slot) { if (rightBuildersConverted) { return (RowSet) rightRowSetSource.getUnsafe(slot); } throw new IllegalStateException( - "getRightIndex() may not be called before convertRightBuildersToIndex() or convertRightGrouping()"); + "getRightRowset() may not be called before convertRightBuildersToRowSet() or populateRightRowSetsFromIndex()"); } @Override - public void convertRightBuildersToIndex(IntegerArraySource slots, int slotCount) { + public void convertRightBuildersToRowSet(@NotNull final IntegerArraySource slots, final int slotCount) { + Assert.eqFalse(rightBuildersConverted, "rightBuildersConverted"); + rightBuildersConverted = true; + for (int slotIndex = 0; slotIndex < slotCount; ++slotIndex) { final int slot = slots.getInt(slotIndex); - // this might be empty, if so then set null final RowSetBuilderSequential sequentialBuilder = (RowSetBuilderSequential) rightRowSetSource.getUnsafe(slot); - if (sequentialBuilder != null) { - WritableRowSet rs = sequentialBuilder.build(); - if (rs.isEmpty()) { - rightRowSetSource.set(slot, EMPTY_RIGHT_STATE); - rs.close(); - } else { - rightRowSetSource.set(slot, rs); - } + if (sequentialBuilder == null) { + continue; + } + final WritableRowSet rs = sequentialBuilder.build(); + if (rs.isEmpty()) { + rightRowSetSource.set(slot, EMPTY_RIGHT_STATE); + rs.close(); + } else { + rightRowSetSource.set(slot, rs); } } - rightBuildersConverted = true; } @Override - public void convertRightGrouping(IntegerArraySource slots, int slotCount, ObjectArraySource rowSetSource) { + public void populateRightRowSetsFromIndexTable( + @NotNull final IntegerArraySource slots, + final int slotCount, + @NotNull final ColumnSource rowSetSource) { + Assert.eqFalse(rightBuildersConverted, "rightBuildersConverted"); + rightBuildersConverted = true; + for (int slotIndex = 0; slotIndex < slotCount; ++slotIndex) { final int slot = slots.getInt(slotIndex); final RowSetBuilderSequential sequentialBuilder = (RowSetBuilderSequential) rightRowSetSource.getUnsafe(slot); - if (sequentialBuilder != null) { - WritableRowSet rs = sequentialBuilder.build(); + if (sequentialBuilder == null) { + continue; + } + try (final RowSet rs = sequentialBuilder.build()) { if (rs.isEmpty()) { rightRowSetSource.set(slot, EMPTY_RIGHT_STATE); - rs.close(); + } else if (rs.size() == 1) { + // The index cannot be modified, since the right table must be static, but make a defensive copy + // anyway in case the index is cleaned up aggressively in the future. + rightRowSetSource.set(slot, rowSetSource.get(rs.firstRowKey()).copy()); } else { - rightRowSetSource.set(slot, getGroupedIndex(rowSetSource, sequentialBuilder)); + throw new IllegalStateException("Index-built row set should have exactly one value: " + rs); } } } - rightBuildersConverted = true; - } - - private RowSet getGroupedIndex(ObjectArraySource rowSetSource, RowSetBuilderSequential sequentialBuilder) { - final RowSet groupedRowSet = sequentialBuilder.build(); - if (groupedRowSet.size() != 1) { - throw new IllegalStateException("Grouped rowSet should have exactly one value: " + groupedRowSet); - } - return rowSetSource.getUnsafe(groupedRowSet.get(0)); } abstract protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/StaticHashedAsOfJoinStateManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/StaticHashedAsOfJoinStateManager.java index b1341a5ec3a..d754ff64414 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/StaticHashedAsOfJoinStateManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/StaticHashedAsOfJoinStateManager.java @@ -9,10 +9,10 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.StaticAsOfJoinStateManager; import io.deephaven.engine.table.impl.sources.IntegerArraySource; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; import org.jetbrains.annotations.NotNull; public abstract class StaticHashedAsOfJoinStateManager extends StaticAsOfJoinStateManager { + protected StaticHashedAsOfJoinStateManager(ColumnSource[] keySourcesForErrorMessages) { super(keySourcesForErrorMessages); } @@ -32,12 +32,12 @@ public abstract int probeLeft(RowSequence leftRowSet, ColumnSource[] leftSour public abstract int getTableSize(); - public abstract RowSet getLeftIndex(int slot); + public abstract RowSet getLeftRowSet(int slot); - public abstract RowSet getRightIndex(int slot); + public abstract RowSet getRightRowset(int slot); - public abstract void convertRightBuildersToIndex(IntegerArraySource slots, int slotCount); + public abstract void convertRightBuildersToRowSet(IntegerArraySource slots, int slotCount); - public abstract void convertRightGrouping(IntegerArraySource slots, int slotCount, - ObjectArraySource rowSetSource); + public abstract void populateRightRowSetsFromIndexTable(IntegerArraySource slots, int slotCount, + ColumnSource rowSetSource); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/TypedAsOfJoinFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/TypedAsOfJoinFactory.java index 128e7526dea..3a8f5724fc7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/TypedAsOfJoinFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/TypedAsOfJoinFactory.java @@ -4,7 +4,6 @@ package io.deephaven.engine.table.impl.asofjoin; import com.squareup.javapoet.CodeBlock; -import io.deephaven.base.verify.Assert; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.impl.by.typed.HasherConfig; import io.deephaven.util.QueryConstants; @@ -12,27 +11,27 @@ public class TypedAsOfJoinFactory { public static void staticBuildLeftFound(HasherConfig hasherConfig, boolean alternate, CodeBlock.Builder builder) { - builder.addStatement("addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition))"); + builder.addStatement("addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition))"); } public static void staticBuildLeftInsert(HasherConfig hasherConfig, CodeBlock.Builder builder) { - builder.addStatement("addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition))"); + builder.addStatement("addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition))"); builder.addStatement("rightRowSetSource.set(tableLocation, $T.builderSequential())", RowSetFactory.class); } public static void staticBuildRightFound(HasherConfig hasherConfig, boolean alternate, CodeBlock.Builder builder) { - builder.addStatement("addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition))"); + builder.addStatement("addRightKey(tableLocation, rowKeyChunk.get(chunkPosition))"); } public static void staticBuildRightInsert(HasherConfig hasherConfig, CodeBlock.Builder builder) { - builder.addStatement("addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition))"); + builder.addStatement("addRightKey(tableLocation, rowKeyChunk.get(chunkPosition))"); } public static void staticProbeDecorateLeftFound(HasherConfig hasherConfig, boolean alternate, CodeBlock.Builder builder) { builder.addStatement("final long indexKey = rowKeyChunk.get(chunkPosition)"); - builder.beginControlFlow("if (addLeftIndex(tableLocation, indexKey) && hashSlots != null)"); + builder.beginControlFlow("if (addLeftKey(tableLocation, indexKey) && hashSlots != null)"); builder.addStatement("hashSlots.set(hashSlotOffset.getAndIncrement(), tableLocation)"); builder.addStatement("foundBuilder.addKey(indexKey)"); builder.endControlFlow(); @@ -40,7 +39,7 @@ public static void staticProbeDecorateLeftFound(HasherConfig hasherConfig, bo public static void staticProbeDecorateRightFound(HasherConfig hasherConfig, boolean alternate, CodeBlock.Builder builder) { - builder.addStatement("addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition))"); + builder.addStatement("addRightKey(tableLocation, rowKeyChunk.get(chunkPosition))"); } public static void staticRehashSetup(CodeBlock.Builder builder) { @@ -97,7 +96,7 @@ public static void rightIncrementalBuildLeftFound(HasherConfig hasherConfig, builder.beginControlFlow("if (sequentialBuilders != null)"); builder.addStatement("addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition))"); builder.nextControlFlow("else"); - builder.addStatement("addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState)"); + builder.addStatement("addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState)"); builder.endControlFlow(); } else { builder.addStatement("final long cookie = getCookieAlternate(alternateTableLocation)"); @@ -107,7 +106,7 @@ public static void rightIncrementalBuildLeftFound(HasherConfig hasherConfig, builder.addStatement("addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition))"); builder.nextControlFlow("else"); builder.addStatement( - "addAlternateLeftIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState)"); + "addAlternateLeftKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState)"); builder.endControlFlow(); } } @@ -120,7 +119,7 @@ public static void rightIncrementalBuildLeftInsert(HasherConfig hasherConfig, builder.addStatement("addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition))"); builder.addStatement("stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY))"); builder.nextControlFlow("else"); - builder.addStatement("addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0)"); + builder.addStatement("addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0)"); builder.endControlFlow(); } @@ -133,7 +132,7 @@ public static void rightIncrementalRightFound(HasherConfig hasherConfig, bool builder.beginControlFlow("if (sequentialBuilders != null)"); builder.addStatement("addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition))"); builder.nextControlFlow("else"); - builder.addStatement("addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState)"); + builder.addStatement("addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState)"); builder.endControlFlow(); } else { builder.addStatement("final long cookie = getCookieAlternate(alternateTableLocation)"); @@ -143,7 +142,7 @@ public static void rightIncrementalRightFound(HasherConfig hasherConfig, bool builder.addStatement("addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition))"); builder.nextControlFlow("else"); builder.addStatement( - "addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState)"); + "addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState)"); builder.endControlFlow(); } } @@ -156,7 +155,7 @@ public static void rightIncrementalRightInsert(HasherConfig hasherConfig, Cod builder.addStatement("addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition))"); builder.addStatement("stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY))"); builder.nextControlFlow("else"); - builder.addStatement("addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0)"); + builder.addStatement("addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0)"); builder.endControlFlow(); } @@ -168,7 +167,7 @@ public static void rightIncrementalProbeDecorateRightFound(HasherConfig hashe builder.addStatement("hashSlots.set(cookie, tableLocation | mainInsertMask)"); builder.addStatement("addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition))"); builder.nextControlFlow("else"); - builder.addStatement("addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState)"); + builder.addStatement("addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState)"); builder.endControlFlow(); } else { builder.beginControlFlow("if (sequentialBuilders != null)"); @@ -177,7 +176,7 @@ public static void rightIncrementalProbeDecorateRightFound(HasherConfig hashe builder.addStatement("addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition))"); builder.nextControlFlow("else"); builder.addStatement( - "addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState)"); + "addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState)"); builder.endControlFlow(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherByte.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherByte.java index 20097339969..04ddeebbee9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherByte.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherByte.java @@ -71,7 +71,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateLeftIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateLeftKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -87,7 +87,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -97,7 +97,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -133,7 +133,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -149,7 +149,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -158,7 +158,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -188,7 +188,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, tableLocation | mainInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } found = true; break; @@ -207,7 +207,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, alternateTableLocation | alternateInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherChar.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherChar.java index ced1596876d..b420de67043 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherChar.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherChar.java @@ -71,7 +71,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateLeftIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateLeftKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -87,7 +87,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -97,7 +97,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -133,7 +133,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -149,7 +149,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -158,7 +158,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -188,7 +188,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, tableLocation | mainInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } found = true; break; @@ -207,7 +207,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, alternateTableLocation | alternateInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherDouble.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherDouble.java index f181e591739..b50d881023a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherDouble.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherDouble.java @@ -71,7 +71,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateLeftIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateLeftKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -87,7 +87,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -97,7 +97,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -133,7 +133,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -149,7 +149,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -158,7 +158,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -188,7 +188,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, tableLocation | mainInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } found = true; break; @@ -207,7 +207,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, alternateTableLocation | alternateInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherFloat.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherFloat.java index 3201d846b4a..3358a812158 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherFloat.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherFloat.java @@ -71,7 +71,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateLeftIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateLeftKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -87,7 +87,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -97,7 +97,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -133,7 +133,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -149,7 +149,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -158,7 +158,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -188,7 +188,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, tableLocation | mainInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } found = true; break; @@ -207,7 +207,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, alternateTableLocation | alternateInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherInt.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherInt.java index 881690c33cb..f478df8250b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherInt.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherInt.java @@ -71,7 +71,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateLeftIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateLeftKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -87,7 +87,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -97,7 +97,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -133,7 +133,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -149,7 +149,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -158,7 +158,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -188,7 +188,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, tableLocation | mainInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } found = true; break; @@ -207,7 +207,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, alternateTableLocation | alternateInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherLong.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherLong.java index 57762bf26fd..6ffb8cf79c8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherLong.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherLong.java @@ -70,7 +70,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateLeftIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateLeftKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -86,7 +86,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -96,7 +96,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -132,7 +132,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -148,7 +148,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -157,7 +157,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -187,7 +187,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, tableLocation | mainInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } found = true; break; @@ -206,7 +206,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, alternateTableLocation | alternateInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherObject.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherObject.java index 77cce00ba00..1ccc5ff49f5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherObject.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherObject.java @@ -72,7 +72,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateLeftIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateLeftKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -88,7 +88,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -98,7 +98,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -134,7 +134,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -150,7 +150,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -159,7 +159,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -189,7 +189,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, tableLocation | mainInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } found = true; break; @@ -208,7 +208,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, alternateTableLocation | alternateInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherShort.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherShort.java index 7cbff0c9b9f..e577c0c46af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherShort.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/rightincopen/gen/RightIncrementalAsOfJoinHasherShort.java @@ -71,7 +71,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateLeftIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateLeftKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -87,7 +87,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -97,7 +97,7 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -133,7 +133,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break MAIN_SEARCH; } else { @@ -149,7 +149,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); stateSource.set(tableLocation, (byte)(ENTRY_RIGHT_IS_EMPTY | ENTRY_LEFT_IS_EMPTY)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), (byte) 0); } break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { @@ -158,7 +158,7 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (sequentialBuilders != null) { addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } else { @@ -188,7 +188,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, tableLocation | mainInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition), rowState); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition), rowState); } found = true; break; @@ -207,7 +207,7 @@ protected void probeRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks, hashSlots.set(cookie, alternateTableLocation | alternateInsertMask); addToSequentialBuilder(cookie, sequentialBuilders, rowKeyChunk.get(chunkPosition)); } else { - addAlternateRightIndex(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); + addAlternateRightKey(alternateTableLocation, rowKeyChunk.get(chunkPosition), rowState); } break; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherByte.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherByte.java index 7df3a47e880..adbfec587ad 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherByte.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherByte.java @@ -57,11 +57,11 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); rightRowSetSource.set(tableLocation, RowSetFactory.builderSequential()); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -85,10 +85,10 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -112,7 +112,7 @@ protected void decorateLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { final long indexKey = rowKeyChunk.get(chunkPosition); - if (addLeftIndex(tableLocation, indexKey) && hashSlots != null) { + if (addLeftKey(tableLocation, indexKey) && hashSlots != null) { hashSlots.set(hashSlotOffset.getAndIncrement(), tableLocation); foundBuilder.addKey(indexKey); } @@ -135,7 +135,7 @@ protected void decorateWithRightSide(RowSequence rowSequence, Chunk[] sourceKeyC int tableLocation = firstTableLocation; while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } tableLocation = nextTableLocation(tableLocation); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherChar.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherChar.java index e25af6c6b40..25f5ddc4b91 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherChar.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherChar.java @@ -57,11 +57,11 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); rightRowSetSource.set(tableLocation, RowSetFactory.builderSequential()); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -85,10 +85,10 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -112,7 +112,7 @@ protected void decorateLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { final long indexKey = rowKeyChunk.get(chunkPosition); - if (addLeftIndex(tableLocation, indexKey) && hashSlots != null) { + if (addLeftKey(tableLocation, indexKey) && hashSlots != null) { hashSlots.set(hashSlotOffset.getAndIncrement(), tableLocation); foundBuilder.addKey(indexKey); } @@ -135,7 +135,7 @@ protected void decorateWithRightSide(RowSequence rowSequence, Chunk[] sourceKeyC int tableLocation = firstTableLocation; while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } tableLocation = nextTableLocation(tableLocation); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherDouble.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherDouble.java index 21ad9b5c1b9..a692ab71a2f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherDouble.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherDouble.java @@ -57,11 +57,11 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); rightRowSetSource.set(tableLocation, RowSetFactory.builderSequential()); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -85,10 +85,10 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -112,7 +112,7 @@ protected void decorateLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { final long indexKey = rowKeyChunk.get(chunkPosition); - if (addLeftIndex(tableLocation, indexKey) && hashSlots != null) { + if (addLeftKey(tableLocation, indexKey) && hashSlots != null) { hashSlots.set(hashSlotOffset.getAndIncrement(), tableLocation); foundBuilder.addKey(indexKey); } @@ -135,7 +135,7 @@ protected void decorateWithRightSide(RowSequence rowSequence, Chunk[] sourceKeyC int tableLocation = firstTableLocation; while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } tableLocation = nextTableLocation(tableLocation); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherFloat.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherFloat.java index 902811a14af..6309f693958 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherFloat.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherFloat.java @@ -57,11 +57,11 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); rightRowSetSource.set(tableLocation, RowSetFactory.builderSequential()); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -85,10 +85,10 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -112,7 +112,7 @@ protected void decorateLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { final long indexKey = rowKeyChunk.get(chunkPosition); - if (addLeftIndex(tableLocation, indexKey) && hashSlots != null) { + if (addLeftKey(tableLocation, indexKey) && hashSlots != null) { hashSlots.set(hashSlotOffset.getAndIncrement(), tableLocation); foundBuilder.addKey(indexKey); } @@ -135,7 +135,7 @@ protected void decorateWithRightSide(RowSequence rowSequence, Chunk[] sourceKeyC int tableLocation = firstTableLocation; while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } tableLocation = nextTableLocation(tableLocation); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherInt.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherInt.java index c0f2b508293..d46b708148e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherInt.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherInt.java @@ -57,11 +57,11 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); rightRowSetSource.set(tableLocation, RowSetFactory.builderSequential()); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -85,10 +85,10 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -112,7 +112,7 @@ protected void decorateLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { final long indexKey = rowKeyChunk.get(chunkPosition); - if (addLeftIndex(tableLocation, indexKey) && hashSlots != null) { + if (addLeftKey(tableLocation, indexKey) && hashSlots != null) { hashSlots.set(hashSlotOffset.getAndIncrement(), tableLocation); foundBuilder.addKey(indexKey); } @@ -135,7 +135,7 @@ protected void decorateWithRightSide(RowSequence rowSequence, Chunk[] sourceKeyC int tableLocation = firstTableLocation; while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } tableLocation = nextTableLocation(tableLocation); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherLong.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherLong.java index 81ce2ab9083..24bf8b01a6c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherLong.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherLong.java @@ -56,11 +56,11 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); rightRowSetSource.set(tableLocation, RowSetFactory.builderSequential()); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -84,10 +84,10 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -111,7 +111,7 @@ protected void decorateLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { final long indexKey = rowKeyChunk.get(chunkPosition); - if (addLeftIndex(tableLocation, indexKey) && hashSlots != null) { + if (addLeftKey(tableLocation, indexKey) && hashSlots != null) { hashSlots.set(hashSlotOffset.getAndIncrement(), tableLocation); foundBuilder.addKey(indexKey); } @@ -134,7 +134,7 @@ protected void decorateWithRightSide(RowSequence rowSequence, Chunk[] sourceKeyC int tableLocation = firstTableLocation; while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } tableLocation = nextTableLocation(tableLocation); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherObject.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherObject.java index e42163696d0..262c0793de7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherObject.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherObject.java @@ -57,11 +57,11 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); rightRowSetSource.set(tableLocation, RowSetFactory.builderSequential()); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -85,10 +85,10 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -112,7 +112,7 @@ protected void decorateLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { final long indexKey = rowKeyChunk.get(chunkPosition); - if (addLeftIndex(tableLocation, indexKey) && hashSlots != null) { + if (addLeftKey(tableLocation, indexKey) && hashSlots != null) { hashSlots.set(hashSlotOffset.getAndIncrement(), tableLocation); foundBuilder.addKey(indexKey); } @@ -135,7 +135,7 @@ protected void decorateWithRightSide(RowSequence rowSequence, Chunk[] sourceKeyC int tableLocation = firstTableLocation; while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } tableLocation = nextTableLocation(tableLocation); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherShort.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherShort.java index 17b65b74819..14ce7ef0f99 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherShort.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/asofjoin/typed/staticopen/gen/StaticAsOfJoinHasherShort.java @@ -57,11 +57,11 @@ protected void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunk if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); rightRowSetSource.set(tableLocation, RowSetFactory.builderSequential()); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addLeftIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addLeftKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -85,10 +85,10 @@ protected void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChun if (rightSideSentinel == EMPTY_RIGHT_STATE) { numEntries++; mainKeySource0.set(tableLocation, k0); - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } else { tableLocation = nextTableLocation(tableLocation); @@ -112,7 +112,7 @@ protected void decorateLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { final long indexKey = rowKeyChunk.get(chunkPosition); - if (addLeftIndex(tableLocation, indexKey) && hashSlots != null) { + if (addLeftKey(tableLocation, indexKey) && hashSlots != null) { hashSlots.set(hashSlotOffset.getAndIncrement(), tableLocation); foundBuilder.addKey(indexKey); } @@ -135,7 +135,7 @@ protected void decorateWithRightSide(RowSequence rowSequence, Chunk[] sourceKeyC int tableLocation = firstTableLocation; while (rightRowSetSource.getUnsafe(tableLocation) != EMPTY_RIGHT_STATE) { if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { - addRightIndex(tableLocation, rowKeyChunk.get(chunkPosition)); + addRightKey(tableLocation, rowKeyChunk.get(chunkPosition)); break; } tableLocation = nextTableLocation(tableLocation); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationControl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationControl.java index bcb9f035b65..86107107da2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationControl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationControl.java @@ -3,11 +3,13 @@ // package io.deephaven.engine.table.impl.by; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.ColumnSource; +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.sources.regioned.SymbolTableSource; import io.deephaven.util.annotations.VisibleForTesting; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * Stateless "control" class for giving external code (e.g. unit tests) knobs to turn w.r.t. to how aggregations should @@ -22,10 +24,11 @@ public class AggregationControl { private static final double DEFAULT_TARGET_LOAD_FACTOR = 0.70; public static final AggregationControl DEFAULT = new AggregationControl(); - public static final AggregationControl DEFAULT_FOR_OPERATOR = new AggregationControl() { + + public static final AggregationControl IGNORE_INDEXING = new AggregationControl() { @Override - public boolean considerGrouping(@NotNull Table table, @NotNull ColumnSource[] sources) { - return sources.length == 1; + public DataIndex dataIndexToUse(@NotNull final Table table, @NotNull final String... keyColumnNames) { + return null; } }; @@ -42,17 +45,41 @@ public double getMaximumLoadFactor() { return DEFAULT_MAX_LOAD_FACTOR; } - public boolean considerGrouping(@NotNull final Table inputTable, @NotNull final ColumnSource[] sources) { - return !inputTable.isRefreshing() && sources.length == 1; - } - - public boolean shouldProbeShift(final long shiftSize, final int numStates) { - return shiftSize <= numStates * 2; + /** + * Get a {@link BasicDataIndex} to use for aggregating {@code table} by {@code keyColumnNames}. + *

+ * This call should be enclosed within a {@link io.deephaven.engine.liveness.LivenessScope} to ensure liveness is + * not unintentionally leaked for any new {@link BasicDataIndex indexes} or {@link Table tables} created. If a + * non-{@code null} {@link DataIndex#isRefreshing()} is returned, it will have been managed by the enclosing + * {@link io.deephaven.engine.liveness.LivenessScope}. + *

+ * If a non-{@code null} result is returned, it will have transformed as needed to ensure that the + * {@link BasicDataIndex#table()} is sorted by first row key. + * + * @param table The {@link Table} to aggregate + * @param keyColumnNames The column names to aggregate by + * @return The {@link DataIndex} to use, or {@code null} if no index should be used + */ + @Nullable + public BasicDataIndex dataIndexToUse(@NotNull final Table table, @NotNull final String... keyColumnNames) { + final DataIndex preTransformDataIndex = DataIndexer.getDataIndex(table, keyColumnNames); + if (preTransformDataIndex == null) { + return null; + } + Assert.eq(table.isRefreshing(), "table.isRefreshing()", + preTransformDataIndex.isRefreshing(), "preTransformDataIndex.isRefreshing()"); + // Note that this transformation uses only concurrent table operations. + final BasicDataIndex transformedDataIndex = preTransformDataIndex.transform(DataIndexTransformer.builder() + .sortByFirstRowKey(true) + .build()); + Assert.eq(table.isRefreshing(), "table.isRefreshing()", + transformedDataIndex.isRefreshing(), "transformedDataIndex.isRefreshing()"); + return transformedDataIndex; } - boolean considerSymbolTables(@NotNull final Table inputTable, final boolean useGrouping, + boolean considerSymbolTables(@NotNull final Table inputTable, final boolean indexed, @NotNull final ColumnSource[] sources) { - return !inputTable.isRefreshing() && !useGrouping && sources.length == 1 + return !inputTable.isRefreshing() && !indexed && sources.length == 1 && SymbolTableSource.hasSymbolTable(sources[0], inputTable.getRowSet()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 80464ec1751..ad7dfa44d3f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -1430,7 +1430,8 @@ private static AggregationContext makeExposedGroupRowSetAggregationContext( new CountAggregationOperator(null) }, new String[][] {ZERO_LENGTH_STRING_ARRAY, ZERO_LENGTH_STRING_ARRAY}, - new ChunkSource.WithPrev[] {null, null}); + new ChunkSource.WithPrev[] {null, null}, + new AggregationContextTransformer[] {new RowLookupAttributeSetter()}); } // noinspection unchecked return new AggregationContext( @@ -1438,7 +1439,8 @@ private static AggregationContext makeExposedGroupRowSetAggregationContext( new GroupByChunkedOperator(inputQueryTable, true, EXPOSED_GROUP_ROW_SETS.name()) }, new String[][] {ZERO_LENGTH_STRING_ARRAY}, - new ChunkSource.WithPrev[] {null}); + new ChunkSource.WithPrev[] {null}, + new AggregationContextTransformer[] {new RowLookupAttributeSetter()}); } private static ColumnSource maybeReinterpretInstantAsLong(@NotNull final ColumnSource inputSource) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java index 68bfce9fa33..2bca8acc7d3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java @@ -3,11 +3,7 @@ // package io.deephaven.engine.table.impl.by; -import gnu.trove.impl.Constants; -import gnu.trove.map.TObjectIntMap; -import gnu.trove.map.hash.TObjectIntHashMap; import io.deephaven.api.ColumnName; -import io.deephaven.base.Pair; import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; import io.deephaven.chunk.*; @@ -16,6 +12,8 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.configuration.Configuration; import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -23,22 +21,21 @@ import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.NoSuchColumnException.Type; import io.deephaven.engine.table.impl.by.typed.TypedHasherFactory; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; +import io.deephaven.engine.table.impl.dataindex.DataIndexUtils; import io.deephaven.engine.table.impl.remote.ConstructSnapshot; import io.deephaven.engine.table.impl.sort.findruns.IntFindRunsKernel; import io.deephaven.engine.table.impl.sort.permute.LongPermuteKernel; import io.deephaven.engine.table.impl.sort.permute.PermuteKernel; import io.deephaven.engine.table.impl.sort.timsort.IntIntTimsortKernel; import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.sources.regioned.SymbolTableSource; import io.deephaven.engine.table.impl.util.ChunkUtils; import io.deephaven.engine.table.impl.util.UpdateSizeCalculator; +import io.deephaven.engine.table.iterators.ChunkedColumnIterator; import io.deephaven.util.SafeCloseable; import io.deephaven.util.SafeCloseableArray; import io.deephaven.util.SafeCloseableList; -import io.deephaven.util.annotations.ReferentialIntegrity; import io.deephaven.util.annotations.VisibleForTesting; import org.apache.commons.lang3.mutable.Mutable; import org.apache.commons.lang3.mutable.MutableInt; @@ -47,8 +44,8 @@ import org.jetbrains.annotations.Nullable; import java.util.*; -import java.util.function.LongFunction; import java.util.function.Supplier; +import java.util.function.ToLongFunction; import java.util.function.UnaryOperator; import static io.deephaven.engine.table.impl.by.AggregationRowLookup.DEFAULT_UNKNOWN_ROW; @@ -74,7 +71,7 @@ public static QueryTable aggregation( final boolean preserveEmpty, @Nullable final Table initialKeys, @NotNull final Collection groupByColumns) { - return aggregation(AggregationControl.DEFAULT_FOR_OPERATOR, + return aggregation(AggregationControl.DEFAULT, aggregationContextFactory, input, preserveEmpty, initialKeys, groupByColumns); } @@ -118,17 +115,28 @@ public static QueryTable aggregation( } } } - final Mutable resultHolder = new MutableObject<>(); - final OperationSnapshotControl snapshotControl = - input.createSnapshotControlIfRefreshing(OperationSnapshotControl::new); - BaseTable.initializeWithSnapshot( - "by(" + aggregationContextFactory + ", " + groupByColumns + ")", snapshotControl, - (usePrev, beforeClockValue) -> { - resultHolder.setValue(aggregation(control, snapshotControl, aggregationContextFactory, - input, preserveEmpty, initialKeys, keyNames, usePrev)); - return true; - }); - return resultHolder.getValue(); + + return LivenessScopeStack.computeEnclosed(() -> { + final Mutable resultHolder = new MutableObject<>(); + final OperationSnapshotControl snapshotControl; + // If the table is refreshing and using an index, include the index table in the snapshot control. + final BasicDataIndex dataIndex = control.dataIndexToUse(input, keyNames); + if (dataIndex != null && dataIndex.isRefreshing()) { + snapshotControl = input.createSnapshotControlIfRefreshing( + it -> new OperationSnapshotControlEx(it, dataIndex.table())); + } else { + snapshotControl = input.createSnapshotControlIfRefreshing(OperationSnapshotControl::new); + } + + BaseTable.initializeWithSnapshot( + "by(" + aggregationContextFactory + ", " + groupByColumns + ")", snapshotControl, + (usePrev, beforeClockValue) -> { + resultHolder.setValue(aggregation(control, snapshotControl, aggregationContextFactory, + input, dataIndex, preserveEmpty, initialKeys, keyNames, usePrev)); + return true; + }); + return resultHolder.getValue(); + }, input::isRefreshing, Table::isRefreshing); } private static QueryTable aggregation( @@ -136,6 +144,7 @@ private static QueryTable aggregation( @Nullable final OperationSnapshotControl snapshotControl, @NotNull final AggregationContextFactory aggregationContextFactory, @NotNull final QueryTable input, + @Nullable final BasicDataIndex dataIndex, final boolean preserveEmpty, @Nullable final Table initialKeys, @NotNull final String[] keyNames, @@ -157,23 +166,13 @@ private static QueryTable aggregation( final PermuteKernel[] permuteKernels = ac.makePermuteKernels(); - final boolean useGrouping; - if (control.considerGrouping(input, keySources)) { - Assert.eq(keySources.length, "keySources.length", 1); - - final boolean hasGrouping = RowSetIndexer.of(input.getRowSet()).hasGrouping(keySources[0]); - if (!input.isRefreshing() && hasGrouping && initialKeys == null) { - return staticGroupedAggregation(input, keyNames[0], keySources[0], ac); - } - // we have no hasPrevGrouping method - useGrouping = !usePrev && hasGrouping && Arrays.equals(reinterpretedKeySources, keySources); - } else { - useGrouping = false; + if (dataIndex != null && initialKeys == null && !input.isRefreshing()) { + return staticIndexedAggregation(dataIndex, keyNames, ac); } final Table symbolTable; final boolean useSymbolTable; - if (!input.isRefreshing() && control.considerSymbolTables(input, useGrouping, keySources)) { + if (!input.isRefreshing() && control.considerSymbolTables(input, dataIndex != null, keySources)) { Assert.eq(keySources.length, "keySources.length", 1); symbolTable = ((SymbolTableSource) keySources[0]).getStaticSymbolTable(input.getRowSet(), @@ -198,9 +197,9 @@ private static QueryTable aggregation( final RowSetBuilderRandom initialRowsBuilder = initialKeys != null && !preserveEmpty ? new BitmapRandomBuilder(stateManager.maxTableSize() - 1) : null; - if (useGrouping) { - initialGroupedKeyAddition(input, reinterpretedKeySources, ac, stateManager, outputPosition, - initialRowsBuilder, usePrev); + if (dataIndex != null) { + initialIndexedKeyAddition(dataIndex, keySources, ac, stateManager, outputPosition, initialRowsBuilder, + usePrev); } else { initialBucketedKeyAddition(input, reinterpretedKeySources, ac, permuteKernels, stateManager, outputPosition, initialRowsBuilder, usePrev); @@ -231,6 +230,7 @@ private static QueryTable aggregation( } ac.getResultColumns(resultColumnSourceMap); + @SuppressWarnings("resource") final TrackingWritableRowSet resultRowSet = (initialRowsBuilder == null ? RowSetFactory.flat(outputPosition.intValue()) : initialRowsBuilder.build()).toTracking(); @@ -242,7 +242,7 @@ private static QueryTable aggregation( final QueryTable result = new QueryTable(resultRowSet, resultColumnSourceMap); ac.propagateInitialStateToOperators(result, outputPosition.intValue()); - if (input.isRefreshing()) { + if (snapshotControl != null) { assert keyColumnsCopied != null; ac.startTrackingPrevValues(); @@ -251,54 +251,51 @@ private static QueryTable aggregation( incrementalStateManager.startTrackingPrevValues(); final boolean isBlink = input.isBlink(); - final TableUpdateListener listener = - new BaseTable.ListenerImpl("by(" + aggregationContextFactory + ")", input, result) { - @ReferentialIntegrity - final OperationSnapshotControl swapListenerHardReference = snapshotControl; - - final ModifiedColumnSet keysUpstreamModifiedColumnSet = input.newModifiedColumnSet(keyNames); - final ModifiedColumnSet[] operatorInputModifiedColumnSets = - ac.getInputModifiedColumnSets(input); - final UnaryOperator[] resultModifiedColumnSetFactories = - ac.initializeRefreshing(result, this); - - final StateChangeRecorder stateChangeRecorder = - preserveEmpty ? null : ac.getStateChangeRecorder(); - - @Override - public void onUpdate(@NotNull final TableUpdate upstream) { - incrementalStateManager.beginUpdateCycle(); - - final TableUpdate upstreamToUse = isBlink ? adjustForBlinkTable(upstream) : upstream; - if (upstreamToUse.empty()) { - return; - } - final TableUpdate downstream; - try (final KeyedUpdateContext kuc = new KeyedUpdateContext(ac, incrementalStateManager, - reinterpretedKeySources, permuteKernels, keysUpstreamModifiedColumnSet, - operatorInputModifiedColumnSets, stateChangeRecorder, upstreamToUse, - outputPosition)) { - downstream = kuc.computeDownstreamIndicesAndCopyKeys(input.getRowSet(), - keyColumnsRaw, - keyColumnsCopied, - result.getModifiedColumnSetForUpdates(), resultModifiedColumnSetFactories); - } + final TableUpdateListener listener = new BaseTable.ListenerImpl( + "by(" + aggregationContextFactory + ", " + Arrays.toString(keyNames) + ")", input, result) { + final ModifiedColumnSet keysUpstreamModifiedColumnSet = input.newModifiedColumnSet(keyNames); + final ModifiedColumnSet[] operatorInputModifiedColumnSets = + ac.getInputModifiedColumnSets(input); + final UnaryOperator[] resultModifiedColumnSetFactories = + ac.initializeRefreshing(result, this); + + final StateChangeRecorder stateChangeRecorder = + preserveEmpty ? null : ac.getStateChangeRecorder(); + + @Override + public void onUpdate(@NotNull final TableUpdate upstream) { + incrementalStateManager.beginUpdateCycle(); + + final TableUpdate upstreamToUse = isBlink ? adjustForBlinkTable(upstream) : upstream; + if (upstreamToUse.empty()) { + return; + } + final TableUpdate downstream; + try (final KeyedUpdateContext kuc = new KeyedUpdateContext(ac, incrementalStateManager, + reinterpretedKeySources, permuteKernels, keysUpstreamModifiedColumnSet, + operatorInputModifiedColumnSets, stateChangeRecorder, upstreamToUse, + outputPosition)) { + downstream = kuc.computeDownstreamIndicesAndCopyKeys(input.getRowSet(), + keyColumnsRaw, + keyColumnsCopied, + result.getModifiedColumnSetForUpdates(), resultModifiedColumnSetFactories); + } - if (downstream.empty()) { - downstream.release(); - return; - } + if (downstream.empty()) { + downstream.release(); + return; + } - result.getRowSet().writableCast().update(downstream.added(), downstream.removed()); - result.notifyListeners(downstream); - } + result.getRowSet().writableCast().update(downstream.added(), downstream.removed()); + result.notifyListeners(downstream); + } - @Override - public void onFailureInternal(@NotNull final Throwable originalException, Entry sourceEntry) { - ac.propagateFailureToOperators(originalException, sourceEntry); - super.onFailureInternal(originalException, sourceEntry); - } - }; + @Override + public void onFailureInternal(@NotNull final Throwable originalException, Entry sourceEntry) { + ac.propagateFailureToOperators(originalException, sourceEntry); + super.onFailureInternal(originalException, sourceEntry); + } + }; snapshotControl.setListenerAndResult(listener, result); } @@ -491,7 +488,9 @@ private KeyedUpdateContext(@NotNull final AggregationContext ac, ac.initializeGetContexts(sharedContext, getContexts, chunkSize); // noinspection unchecked workingChunks = toClose.addArray(new WritableChunk[ac.size()]); + // noinspection unchecked valueChunks = new Chunk[ac.size()]; + // noinspection unchecked postValueChunks = new Chunk[ac.size()]; ac.initializeWorkingChunks(workingChunks, chunkSize); permutedKeyIndices = @@ -894,8 +893,8 @@ private void doProcessShiftBucketed(@NotNull final WritableLongChunk keySource, - AggregationContext ac) { - final Pair> groupKeyIndexTable; - final Map grouping = RowSetIndexer.of(withView.getRowSet()).getGrouping(keySource); - // noinspection unchecked - groupKeyIndexTable = GroupingUtils.groupingToFlatSources((ColumnSource) keySource, grouping); - final int responsiveGroups = grouping.size(); + private static QueryTable staticIndexedAggregation( + final BasicDataIndex dataIndex, + final String[] keyNames, + final AggregationContext ac) { + // Ensure that we can re-use the key column sources in the result table + final Table indexTable = dataIndex.table(); + final int groupCount = indexTable.intSize(); final Map> resultColumnSourceMap = new LinkedHashMap<>(); - resultColumnSourceMap.put(keyName, groupKeyIndexTable.first); + + // Add the index key columns directly to the result table. + final Table indexKeyTable = indexTable.flatten().select(keyNames); + for (final String keyName : keyNames) { + resultColumnSourceMap.put(keyName, indexKeyTable.getColumnSource(keyName)); + } ac.getResultColumns(resultColumnSourceMap); - doGroupedAddition(ac, groupKeyIndexTable.second::get, responsiveGroups, CHUNK_SIZE); + // Add the index row sets to the internal grouping. Note that we have not flattened the RowSet column, but + // doIndexedAddition operations based on row position. + try (final CloseableIterator groupRowSetIterator = + ChunkedColumnIterator.make(dataIndex.rowSetColumn(), indexTable.getRowSet())) { + doIndexedAddition(ac, groupRowSetIterator::next, groupCount, CHUNK_SIZE); + } - final QueryTable result = new QueryTable(RowSetFactory.flat(responsiveGroups).toTracking(), - resultColumnSourceMap); - ac.propagateInitialStateToOperators(result, responsiveGroups); + final QueryTable result = new QueryTable(RowSetFactory.flat(groupCount).toTracking(), resultColumnSourceMap); + ac.propagateInitialStateToOperators(result, groupCount); + // Create a lookup function from the index table ac.supplyRowLookup(() -> { - final TObjectIntMap keyToSlot = new TObjectIntHashMap<>( - responsiveGroups, Constants.DEFAULT_LOAD_FACTOR, DEFAULT_UNKNOWN_ROW); - final MutableInt slotNumber = new MutableInt(0); - grouping.keySet().forEach(k -> keyToSlot.put(k, slotNumber.getAndIncrement())); - return keyToSlot::get; + final ToLongFunction lookupKeyToRowKey = + DataIndexUtils.buildRowKeyMappingFunction(indexKeyTable, keyNames); + return key -> (int) lookupKeyToRowKey.applyAsLong(key); }); final QueryTable finalResult = ac.transformResult(result); @@ -1668,10 +1675,10 @@ private static QueryTable staticGroupedAggregation(QueryTable withView, String k return finalResult; } - private static void doGroupedAddition( + private static void doIndexedAddition( @NotNull final AggregationContext ac, - @NotNull final LongFunction groupIndexToRowSet, - final int responsiveGroups, + @NotNull final Supplier groupRowSetSupplier, + final int indexEntryCount, final int chunkSize) { final boolean indicesRequired = ac.requiresIndices(); @@ -1681,14 +1688,14 @@ private static void doGroupedAddition( try (final SafeCloseableArray ignored = new SafeCloseableArray<>(getContexts); final SafeCloseable ignored2 = new SafeCloseableArray<>(operatorContexts); final SharedContext sharedContext = SharedContext.makeSharedContext()) { - ac.ensureCapacity(responsiveGroups); + ac.ensureCapacity(indexEntryCount); ac.initializeGetContexts(sharedContext, getContexts, chunkSize); ac.initializeSingletonContexts(operatorContexts, chunkSize); final boolean unchunked = !ac.requiresInputs() && ac.unchunkedIndices(); if (unchunked) { - for (int ii = 0; ii < responsiveGroups; ++ii) { - final RowSet rowSet = groupIndexToRowSet.apply(ii); + for (int ii = 0; ii < indexEntryCount; ++ii) { + final RowSet rowSet = groupRowSetSupplier.get(); for (int oi = 0; oi < ac.size(); ++oi) { ac.operators[oi].addRowSet(operatorContexts[oi], rowSet, ii); } @@ -1696,8 +1703,8 @@ private static void doGroupedAddition( } else { // noinspection unchecked final Chunk[] workingChunks = new Chunk[ac.size()]; - for (int ii = 0; ii < responsiveGroups; ++ii) { - final RowSet rowSet = groupIndexToRowSet.apply(ii); + for (int ii = 0; ii < indexEntryCount; ++ii) { + final RowSet rowSet = groupRowSetSupplier.get(); try (final RowSequence.Iterator rsIt = rowSet.getRowSequenceIterator()) { do { final RowSequence chunkRows = rsIt.getNextRowSequenceWithLength(chunkSize); @@ -1738,7 +1745,7 @@ private static void doGroupedAddition( private static OperatorAggregationStateManager initialKeyTableAddition( @NotNull final AggregationControl control, - @NotNull final Table initialKeys, + @NotNull final Table inputInitialKeys, @NotNull final String[] keyColumnNames, @NotNull final AggregationContext ac, @NotNull final MutableInt outputPosition, @@ -1747,50 +1754,54 @@ private static OperatorAggregationStateManager initialKeyTableAddition( // This logic is duplicative of the logic in the main aggregation function, but it's hard to consolidate // further. A better strategy might be to do a selectDistinct first, but that would result in more hash table // inserts. - final ColumnSource[] keySources = Arrays.stream(keyColumnNames) - .map(initialKeys::getColumnSource) - .toArray(ColumnSource[]::new); - final ColumnSource[] reinterpretedKeySources = Arrays.stream(keyColumnNames) - .map(initialKeys::getColumnSource) - .map(ReinterpretUtils::maybeConvertToPrimitive) + final ColumnSource[] initialKeysSources = Arrays.stream(keyColumnNames) + .map(inputInitialKeys::getColumnSource) .toArray(ColumnSource[]::new); - final boolean useGroupingAllowed = control.considerGrouping(initialKeys, keySources) - && keySources.length == 1 - && reinterpretedKeySources[0] == keySources[0]; - final OperatorAggregationStateManager stateManager; - if (initialKeys.isRefreshing()) { - try (final SafeCloseable ignored = ExecutionContext.getContext().withUpdateGraph( - initialKeys.getUpdateGraph()).open()) { + try (final SafeCloseable ignored1 = inputInitialKeys.isRefreshing() ? LivenessScopeStack.open() : null; + final SafeCloseable ignored2 = + ExecutionContext.getContext().withUpdateGraph(inputInitialKeys.getUpdateGraph()).open()) { + + final Table initialKeys; + final ColumnSource[] keySources; + final BasicDataIndex dataIndex = control.dataIndexToUse(inputInitialKeys, keyColumnNames); + if (dataIndex != null) { + initialKeys = dataIndex.table(); + keySources = dataIndex.keyColumns(initialKeysSources); + } else { + initialKeys = inputInitialKeys; + keySources = initialKeysSources; + } + + final OperatorAggregationStateManager stateManager; + if (initialKeys.isRefreshing()) { final MutableObject stateManagerHolder = new MutableObject<>(); ConstructSnapshot.callDataSnapshotFunction( "InitialKeyTableSnapshot-" + System.identityHashCode(initialKeys) + ": ", ConstructSnapshot.makeSnapshotControl(false, true, (NotificationStepSource) initialKeys), (final boolean usePrev, final long beforeClockValue) -> { stateManagerHolder.setValue(makeInitializedStateManager( - initialKeys, reinterpretedKeySources, ac, outputPosition, stateManagerSupplier, - useGroupingAllowed, usePrev)); + initialKeys, keySources, outputPosition, stateManagerSupplier, usePrev)); return true; }); stateManager = stateManagerHolder.getValue(); + } else { + stateManager = makeInitializedStateManager( + initialKeys, keySources, outputPosition, stateManagerSupplier, false); } - } else { - stateManager = makeInitializedStateManager(initialKeys, reinterpretedKeySources, - ac, outputPosition, stateManagerSupplier, useGroupingAllowed, false); - } - try (final RowSet empty = RowSetFactory.empty()) { - doGroupedAddition(ac, gi -> empty, outputPosition.intValue(), 0); + + try (final RowSet empty = RowSetFactory.empty()) { + doIndexedAddition(ac, () -> empty, outputPosition.intValue(), 0); + } + return stateManager; } - return stateManager; } private static OperatorAggregationStateManager makeInitializedStateManager( @NotNull final Table initialKeys, - @NotNull ColumnSource[] reinterpretedKeySources, - @NotNull final AggregationContext ac, + @NotNull final ColumnSource[] keySources, @NotNull final MutableInt outputPosition, @NotNull final Supplier stateManagerSupplier, - final boolean useGroupingAllowed, final boolean usePrev) { outputPosition.setValue(0); final OperatorAggregationStateManager stateManager = stateManagerSupplier.get(); @@ -1799,30 +1810,14 @@ private static OperatorAggregationStateManager makeInitializedStateManager( return stateManager; } - final ColumnSource[] keyColumnsToInsert; - final boolean closeRowsToInsert; - final RowSequence rowsToInsert; - final RowSetIndexer groupingIndexer = useGroupingAllowed && (!initialKeys.isRefreshing() || !usePrev) - ? RowSetIndexer.of(initialKeys.getRowSet()) - : null; - if (groupingIndexer != null && groupingIndexer.hasGrouping(reinterpretedKeySources[0])) { - final ColumnSource groupedSource = reinterpretedKeySources[0]; - final Map grouping = groupingIndexer.getGrouping(groupedSource); - // noinspection unchecked - keyColumnsToInsert = new ColumnSource[] { - GroupingUtils.groupingKeysToImmutableFlatSource(groupedSource, grouping)}; - closeRowsToInsert = true; - // noinspection resource - rowsToInsert = RowSequenceFactory.forRange(0, grouping.size() - 1); - } else { - keyColumnsToInsert = reinterpretedKeySources; - closeRowsToInsert = usePrev; - rowsToInsert = usePrev ? initialKeys.getRowSet().copyPrev() : initialKeys.getRowSet(); - } + final RowSequence rowsToInsert = usePrev ? initialKeys.getRowSet().prev() : initialKeys.getRowSet();; + final ColumnSource[] keyColumnsToInsert = Arrays.stream(keySources) + .map(ReinterpretUtils::maybeConvertToPrimitive) + .map(cs -> usePrev && !cs.isImmutable() ? cs.getPrevSource() : cs) + .toArray(ColumnSource[]::new); final int chunkSize = chunkSize(rowsToInsert.size()); - try (final SafeCloseable ignored = closeRowsToInsert ? rowsToInsert : null; - final SafeCloseable bc = stateManager.makeAggregationStateBuildContext(keyColumnsToInsert, chunkSize); + try (final SafeCloseable bc = stateManager.makeAggregationStateBuildContext(keyColumnsToInsert, chunkSize); final RowSequence.Iterator rowsIterator = rowsToInsert.getRowSequenceIterator(); final WritableIntChunk outputPositions = WritableIntChunk.makeWritableChunk(chunkSize)) { while (rowsIterator.hasMore()) { @@ -1846,6 +1841,7 @@ private static void initialBucketedKeyAddition(QueryTable input, final ChunkSource.GetContext[] getContexts = new ChunkSource.GetContext[ac.size()]; // noinspection unchecked final WritableChunk[] workingChunks = findRuns ? new WritableChunk[ac.size()] : null; + // noinspection unchecked final Chunk[] valueChunks = new Chunk[ac.size()]; final IterativeChunkedAggregationOperator.BucketedContext[] bucketedContexts = new IterativeChunkedAggregationOperator.BucketedContext[ac.size()]; @@ -1858,7 +1854,7 @@ private static void initialBucketedKeyAddition(QueryTable input, buildSources = reinterpretedKeySources; } - final RowSet rowSet = usePrev ? input.getRowSet().copyPrev() : input.getRowSet(); + final RowSet rowSet = usePrev ? input.getRowSet().prev() : input.getRowSet(); if (rowSet.isEmpty()) { return; @@ -1867,10 +1863,9 @@ private static void initialBucketedKeyAddition(QueryTable input, final int chunkSize = chunkSize(rowSet.size()); try (final SafeCloseable bc = stateManager.makeAggregationStateBuildContext(buildSources, chunkSize); - final SafeCloseable ignored1 = usePrev ? rowSet : null; - final SafeCloseable ignored2 = new SafeCloseableArray<>(getContexts); - final SafeCloseable ignored3 = findRuns ? new SafeCloseableArray<>(workingChunks) : null; - final SafeCloseable ignored4 = new SafeCloseableArray<>(bucketedContexts); + final SafeCloseable ignored1 = new SafeCloseableArray<>(getContexts); + final SafeCloseable ignored2 = findRuns ? new SafeCloseableArray<>(workingChunks) : null; + final SafeCloseable ignored3 = new SafeCloseableArray<>(bucketedContexts); final RowSequence.Iterator rsIt = rowSet.getRowSequenceIterator(); final WritableIntChunk outputPositions = WritableIntChunk.makeWritableChunk(chunkSize); final WritableIntChunk chunkPosition = WritableIntChunk.makeWritableChunk(chunkSize); @@ -1948,47 +1943,51 @@ private static void initialBucketedKeyAddition(QueryTable input, } } - private static void initialGroupedKeyAddition(QueryTable input, - ColumnSource[] reinterpretedKeySources, - AggregationContext ac, - OperatorAggregationStateManager stateManager, - MutableInt outputPosition, - RowSetBuilderRandom initialRowsBuilder, - boolean usePrev) { - final Pair> groupKeyIndexTable; - final RowSetIndexer indexer = RowSetIndexer.of(input.getRowSet()); - final Map grouping = usePrev ? indexer.getPrevGrouping(reinterpretedKeySources[0]) - : indexer.getGrouping(reinterpretedKeySources[0]); - // noinspection unchecked - groupKeyIndexTable = - GroupingUtils.groupingToFlatSources((ColumnSource) reinterpretedKeySources[0], grouping); - final int responsiveGroups = grouping.size(); + private static void initialIndexedKeyAddition( + final BasicDataIndex dataIndex, + final ColumnSource[] keySources, + final AggregationContext ac, + final OperatorAggregationStateManager stateManager, + final MutableInt outputPosition, + final RowSetBuilderRandom initialRowsBuilder, + final boolean usePrev) { + final Table indexTable = dataIndex.table(); + final int indexEntryCount = indexTable.intSize(); - if (responsiveGroups == 0) { + if (indexEntryCount == 0) { return; } - ac.ensureCapacity(responsiveGroups); + ac.ensureCapacity(indexEntryCount); - final ColumnSource[] groupedFlatKeySource = {groupKeyIndexTable.first}; - - try (final SafeCloseable bc = - stateManager.makeAggregationStateBuildContext(groupedFlatKeySource, responsiveGroups); - final RowSequence rs = RowSequenceFactory.forRange(0, responsiveGroups - 1); - final RowSequence.Iterator rsIt = rs.getRowSequenceIterator(); - final WritableIntChunk outputPositions = - WritableIntChunk.makeWritableChunk(responsiveGroups)) { - while (rsIt.hasMore()) { - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(CHUNK_SIZE); - stateManager.add(bc, chunkOk, groupedFlatKeySource, outputPosition, outputPositions); + final RowSet indexRowSet = usePrev ? indexTable.getRowSet().prev() : indexTable.getRowSet(); + final ColumnSource[] indexKeySources = Arrays.stream(dataIndex.keyColumns(keySources)) + .map(ReinterpretUtils::maybeConvertToPrimitive) + .map(cs -> usePrev && !cs.isImmutable() ? cs.getPrevSource() : cs) + .toArray(ColumnSource[]::new); + final ColumnSource groupRowSetColumn = usePrev + ? dataIndex.rowSetColumn().getPrevSource() // Note that RowSets are mutable + : dataIndex.rowSetColumn(); + + // @formatter:off + try (final SafeCloseable bc = stateManager.makeAggregationStateBuildContext(indexKeySources, indexEntryCount); + final RowSequence.Iterator indexRowSetIterator = indexRowSet.getRowSequenceIterator(); + final WritableIntChunk outputPositions = WritableIntChunk.makeWritableChunk(indexEntryCount)) { + // @formatter:on + while (indexRowSetIterator.hasMore()) { + final RowSequence chunkIndexRowKeys = indexRowSetIterator.getNextRowSequenceWithLength(CHUNK_SIZE); + stateManager.add(bc, chunkIndexRowKeys, indexKeySources, outputPosition, outputPositions); if (initialRowsBuilder != null) { initialRowsBuilder.addRowKeysChunk(outputPositions); } } - Assert.eq(outputPosition.intValue(), "outputPosition.intValue()", responsiveGroups, "responsiveGroups"); + Assert.eq(outputPosition.intValue(), "outputPosition.intValue()", indexEntryCount, "indexEntryCount"); } - doGroupedAddition(ac, groupKeyIndexTable.second::get, responsiveGroups, CHUNK_SIZE); + try (final CloseableIterator groupRowSetIterator = + ChunkedColumnIterator.make(groupRowSetColumn, indexRowSet)) { + doIndexedAddition(ac, groupRowSetIterator::next, indexEntryCount, CHUNK_SIZE); + } } private static RowSet makeNewStatesRowSet(final int first, final int last) { @@ -2046,11 +2045,10 @@ private static QueryTable noKeyAggregation( // to use allColumns as the modified columns parameter final IterativeChunkedAggregationOperator.SingletonContext[] opContexts = new IterativeChunkedAggregationOperator.SingletonContext[ac.size()]; - final RowSet rowSet = usePrev ? table.getRowSet().copyPrev() : table.getRowSet(); + final RowSet rowSet = usePrev ? table.getRowSet().prev() : table.getRowSet(); final int initialResultSize; - try (final SafeCloseable ignored1 = new SafeCloseableArray<>(opContexts); - final SafeCloseable ignored2 = usePrev ? rowSet : null) { - initialResultSize = preserveEmpty || rowSet.size() != 0 ? 1 : 0; + try (final SafeCloseable ignored = new SafeCloseableArray<>(opContexts)) { + initialResultSize = preserveEmpty || !rowSet.isEmpty() ? 1 : 0; ac.initializeSingletonContexts(opContexts, rowSet.size()); doNoKeyAddition(rowSet, ac, opContexts, allColumns, usePrev, allColumns); } @@ -2060,165 +2058,165 @@ private static QueryTable noKeyAggregation( // always will create one result for zerokey ac.propagateInitialStateToOperators(result, 1); - if (table.isRefreshing()) { + if (snapshotControl != null) { ac.startTrackingPrevValues(); final boolean isBlink = table.isBlink(); - final TableUpdateListener listener = - new BaseTable.ListenerImpl("groupBy(" + aggregationContextFactory + ")", table, result) { + final TableUpdateListener listener = new BaseTable.ListenerImpl( + "by(" + aggregationContextFactory + ")", table, result) { - final ModifiedColumnSet[] inputModifiedColumnSet = ac.getInputModifiedColumnSets(table); - final UnaryOperator[] resultModifiedColumnSetFactories = - ac.initializeRefreshing(result, this); + final ModifiedColumnSet[] inputModifiedColumnSet = ac.getInputModifiedColumnSets(table); + final UnaryOperator[] resultModifiedColumnSetFactories = + ac.initializeRefreshing(result, this); - int lastSize = initialResultSize; - int statesCreated = initialResultSize; + int lastSize = initialResultSize; + int statesCreated = initialResultSize; - @Override - public void onUpdate(@NotNull final TableUpdate upstream) { - final TableUpdate upstreamToUse = isBlink ? adjustForBlinkTable(upstream) : upstream; - if (upstreamToUse.empty()) { - return; - } - processNoKeyUpdate(upstreamToUse); + @Override + public void onUpdate(@NotNull final TableUpdate upstream) { + final TableUpdate upstreamToUse = isBlink ? adjustForBlinkTable(upstream) : upstream; + if (upstreamToUse.empty()) { + return; + } + processNoKeyUpdate(upstreamToUse); + } + + private void processNoKeyUpdate(@NotNull final TableUpdate upstream) { + ac.resetOperatorsForStep(upstream, 1); + + final ModifiedColumnSet upstreamModifiedColumnSet = + upstream.modified().isEmpty() ? ModifiedColumnSet.EMPTY + : upstream.modifiedColumnSet(); + + final IterativeChunkedAggregationOperator.SingletonContext[] opContexts = + new IterativeChunkedAggregationOperator.SingletonContext[ac.size()]; + try (final SafeCloseable ignored = new SafeCloseableArray<>(opContexts)) { + final OperatorDivision od = new OperatorDivision(ac, upstream.modified().isNonempty(), + upstreamModifiedColumnSet, inputModifiedColumnSet); + ac.initializeSingletonContexts(opContexts, upstream, + od.operatorsWithModifiedInputColumns); + + final boolean[] modifiedOperators = new boolean[ac.size()]; + // remove all the removals + if (upstream.removed().isNonempty()) { + doNoKeyRemoval(upstream.removed(), ac, opContexts, allColumns, modifiedOperators); } - private void processNoKeyUpdate(@NotNull final TableUpdate upstream) { - ac.resetOperatorsForStep(upstream, 1); - - final ModifiedColumnSet upstreamModifiedColumnSet = - upstream.modified().isEmpty() ? ModifiedColumnSet.EMPTY - : upstream.modifiedColumnSet(); - - final IterativeChunkedAggregationOperator.SingletonContext[] opContexts = - new IterativeChunkedAggregationOperator.SingletonContext[ac.size()]; - try (final SafeCloseable ignored = new SafeCloseableArray<>(opContexts)) { - final OperatorDivision od = new OperatorDivision(ac, upstream.modified().isNonempty(), - upstreamModifiedColumnSet, inputModifiedColumnSet); - ac.initializeSingletonContexts(opContexts, upstream, - od.operatorsWithModifiedInputColumns); - - final boolean[] modifiedOperators = new boolean[ac.size()]; - // remove all the removals - if (upstream.removed().isNonempty()) { - doNoKeyRemoval(upstream.removed(), ac, opContexts, allColumns, modifiedOperators); - } + final boolean processShifts = upstream.shifted().nonempty() && ac.requiresIndices(); + + if (upstream.modified().isNonempty() && (od.anyOperatorHasModifiedInputColumns + || od.anyOperatorWithoutModifiedInputColumnsRequiresIndices)) { + if (processShifts) { + // Also handles shifted modifications for modified-input operators that require + // indices (if any) + doNoKeyShifts(table, upstream, ac, opContexts, od.operatorsThatRequireIndices, + modifiedOperators); + + try (final RowSet unshiftedModifies = + extractUnshiftedModifiesFromUpstream(upstream)) { + // Do unshifted modifies for everyone + doNoKeyModifications(unshiftedModifies, unshiftedModifies, ac, opContexts, + true, + od.operatorsWithModifiedInputColumns, + od.operatorsWithoutModifiedInputColumnsThatRequireIndices, + modifiedOperators); - final boolean processShifts = upstream.shifted().nonempty() && ac.requiresIndices(); - - if (upstream.modified().isNonempty() && (od.anyOperatorHasModifiedInputColumns - || od.anyOperatorWithoutModifiedInputColumnsRequiresIndices)) { - if (processShifts) { - // Also handles shifted modifications for modified-input operators that require - // indices (if any) - doNoKeyShifts(table, upstream, ac, opContexts, od.operatorsThatRequireIndices, - modifiedOperators); - - try (final RowSet unshiftedModifies = - extractUnshiftedModifiesFromUpstream(upstream)) { - // Do unshifted modifies for everyone - doNoKeyModifications(unshiftedModifies, unshiftedModifies, ac, opContexts, - true, - od.operatorsWithModifiedInputColumns, - od.operatorsWithoutModifiedInputColumnsThatRequireIndices, + if (od.anyOperatorWithModifiedInputColumnsIgnoresIndices) { + // Do shifted modifies for RowSet-only and modified-input operators that + // don't require indices + try (final RowSet shiftedModifiesPost = + upstream.modified().minus(unshiftedModifies); + final WritableRowSet shiftedModifiesPre = + shiftedModifiesPost.copy()) { + upstream.shifted().unapply(shiftedModifiesPre); + doNoKeyModifications(shiftedModifiesPre, shiftedModifiesPost, ac, + opContexts, true, + od.operatorsWithModifiedInputColumnsThatIgnoreIndices, + od.operatorsThatRequireIndices, modifiedOperators); - - if (od.anyOperatorWithModifiedInputColumnsIgnoresIndices) { - // Do shifted modifies for RowSet-only and modified-input operators that - // don't require indices - try (final RowSet shiftedModifiesPost = - upstream.modified().minus(unshiftedModifies); - final WritableRowSet shiftedModifiesPre = - shiftedModifiesPost.copy()) { - upstream.shifted().unapply(shiftedModifiesPre); - doNoKeyModifications(shiftedModifiesPre, shiftedModifiesPost, ac, - opContexts, true, - od.operatorsWithModifiedInputColumnsThatIgnoreIndices, - od.operatorsThatRequireIndices, - modifiedOperators); - } - } else if (ac.requiresIndices()) { - // Do shifted modifies for RowSet-only operators - try (final RowSet shiftedModifiesPost = - upstream.modified().minus(unshiftedModifies)) { - doIndexOnlyNoKeyModifications(shiftedModifiesPost, ac, opContexts, - od.operatorsThatRequireIndices, modifiedOperators); - } - } } - } else if (od.anyOperatorHasModifiedInputColumns) { - doNoKeyModifications(upstream.getModifiedPreShift(), upstream.modified(), ac, - opContexts, ac.requiresIndices(), - od.operatorsWithModifiedInputColumns, - od.operatorsWithoutModifiedInputColumnsThatRequireIndices, - modifiedOperators); - - } else { - doIndexOnlyNoKeyModifications(upstream.modified(), ac, opContexts, - od.operatorsWithoutModifiedInputColumnsThatRequireIndices, - modifiedOperators); + } else if (ac.requiresIndices()) { + // Do shifted modifies for RowSet-only operators + try (final RowSet shiftedModifiesPost = + upstream.modified().minus(unshiftedModifies)) { + doIndexOnlyNoKeyModifications(shiftedModifiesPost, ac, opContexts, + od.operatorsThatRequireIndices, modifiedOperators); + } } - } else if (processShifts) { - doNoKeyShifts(table, upstream, ac, opContexts, od.operatorsThatRequireIndices, - modifiedOperators); - } - - if (upstream.added().isNonempty()) { - doNoKeyAddition(upstream.added(), ac, opContexts, allColumns, false, - modifiedOperators); } + } else if (od.anyOperatorHasModifiedInputColumns) { + doNoKeyModifications(upstream.getModifiedPreShift(), upstream.modified(), ac, + opContexts, ac.requiresIndices(), + od.operatorsWithModifiedInputColumns, + od.operatorsWithoutModifiedInputColumnsThatRequireIndices, + modifiedOperators); + + } else { + doIndexOnlyNoKeyModifications(upstream.modified(), ac, opContexts, + od.operatorsWithoutModifiedInputColumnsThatRequireIndices, + modifiedOperators); + } + } else if (processShifts) { + doNoKeyShifts(table, upstream, ac, opContexts, od.operatorsThatRequireIndices, + modifiedOperators); + } - final int newResultSize = - preserveEmpty || (isBlink && lastSize != 0) || table.size() != 0 ? 1 : 0; - final TableUpdateImpl downstream = new TableUpdateImpl(); - downstream.shifted = RowSetShiftData.EMPTY; - if ((lastSize == 0 && newResultSize == 1)) { - downstream.added = RowSetFactory.fromKeys(0); - downstream.removed = RowSetFactory.empty(); - downstream.modified = RowSetFactory.empty(); - result.getRowSet().writableCast().insert(0); - } else if (lastSize == 1 && newResultSize == 0) { - downstream.added = RowSetFactory.empty(); - downstream.removed = RowSetFactory.fromKeys(0); - downstream.modified = RowSetFactory.empty(); - result.getRowSet().writableCast().remove(0); - } else if (anyTrue(BooleanChunk.chunkWrap(modifiedOperators))) { - downstream.added = RowSetFactory.empty(); - downstream.removed = RowSetFactory.empty(); - downstream.modified = RowSetFactory.fromKeys(0); - } else { - downstream.added = RowSetFactory.empty(); - downstream.removed = RowSetFactory.empty(); - downstream.modified = RowSetFactory.empty(); - } - lastSize = newResultSize; + if (upstream.added().isNonempty()) { + doNoKeyAddition(upstream.added(), ac, opContexts, allColumns, false, + modifiedOperators); + } - final int newStatesCreated = Math.max(statesCreated, newResultSize); - try (final RowSet newStates = - makeNewStatesRowSet(statesCreated, newStatesCreated - 1)) { - ac.propagateChangesToOperators(downstream, newStates); - } - statesCreated = newStatesCreated; + final int newResultSize = + preserveEmpty || (isBlink && lastSize != 0) || !table.isEmpty() ? 1 : 0; + final TableUpdateImpl downstream = new TableUpdateImpl(); + downstream.shifted = RowSetShiftData.EMPTY; + if ((lastSize == 0 && newResultSize == 1)) { + downstream.added = RowSetFactory.fromKeys(0); + downstream.removed = RowSetFactory.empty(); + downstream.modified = RowSetFactory.empty(); + result.getRowSet().writableCast().insert(0); + } else if (lastSize == 1 && newResultSize == 0) { + downstream.added = RowSetFactory.empty(); + downstream.removed = RowSetFactory.fromKeys(0); + downstream.modified = RowSetFactory.empty(); + result.getRowSet().writableCast().remove(0); + } else if (anyTrue(BooleanChunk.chunkWrap(modifiedOperators))) { + downstream.added = RowSetFactory.empty(); + downstream.removed = RowSetFactory.empty(); + downstream.modified = RowSetFactory.fromKeys(0); + } else { + downstream.added = RowSetFactory.empty(); + downstream.removed = RowSetFactory.empty(); + downstream.modified = RowSetFactory.empty(); + } + lastSize = newResultSize; - extractDownstreamModifiedColumnSet(downstream, result.getModifiedColumnSetForUpdates(), - modifiedOperators, upstreamModifiedColumnSet, resultModifiedColumnSetFactories); + final int newStatesCreated = Math.max(statesCreated, newResultSize); + try (final RowSet newStates = + makeNewStatesRowSet(statesCreated, newStatesCreated - 1)) { + ac.propagateChangesToOperators(downstream, newStates); + } + statesCreated = newStatesCreated; - if (downstream.empty()) { - downstream.release(); - return; - } + extractDownstreamModifiedColumnSet(downstream, result.getModifiedColumnSetForUpdates(), + modifiedOperators, upstreamModifiedColumnSet, resultModifiedColumnSetFactories); - result.notifyListeners(downstream); - } + if (downstream.empty()) { + downstream.release(); + return; } - @Override - public void onFailureInternal(@NotNull final Throwable originalException, - final Entry sourceEntry) { - ac.propagateFailureToOperators(originalException, sourceEntry); - super.onFailureInternal(originalException, sourceEntry); - } - }; + result.notifyListeners(downstream); + } + } + + @Override + public void onFailureInternal(@NotNull final Throwable originalException, + final Entry sourceEntry) { + ac.propagateFailureToOperators(originalException, sourceEntry); + super.onFailureInternal(originalException, sourceEntry); + } + }; snapshotControl.setListenerAndResult(listener, result); } @@ -2257,7 +2255,9 @@ private static void doNoKeyModifications(RowSequence preIndex, RowSequence postI ac.initializeGetContexts(preSharedContext, preGetContexts, preIndex.size(), operatorsToProcess); ac.initializeGetContexts(postSharedContext, postGetContexts, postIndex.size(), operatorsToProcess); + // noinspection unchecked final Chunk[] workingPreChunks = new Chunk[ac.size()]; + // noinspection unchecked final Chunk[] workingPostChunks = new Chunk[ac.size()]; while (postIt.hasMore()) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index e21f8aa1870..97d5e6c1723 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -431,9 +431,13 @@ private InputToResultModifiedColumnSetFactory( @NotNull final QueryTable resultTable, @NotNull final String[] resultAggregatedColumnNames) { updateModifiedColumnSet = new ModifiedColumnSet(resultTable.getModifiedColumnSetForUpdates()); - allResultColumns = resultTable.newModifiedColumnSet(resultAggregatedColumnNames); + if (exposeRowSetsAs != null) { - allResultColumns.setAll(exposeRowSetsAs); + // resultAggregatedColumnNames may be empty (e.g. when the row set is the only result column) + allResultColumns = resultTable.newModifiedColumnSet(exposeRowSetsAs); + allResultColumns.setAll(resultAggregatedColumnNames); + } else { + allResultColumns = resultTable.newModifiedColumnSet(resultAggregatedColumnNames); } aggregatedColumnsTransformer = inputTable.newModifiedColumnSetTransformer( inputColumnNames, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/AbstractDataIndex.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/AbstractDataIndex.java new file mode 100644 index 00000000000..73c008fd750 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/AbstractDataIndex.java @@ -0,0 +1,134 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LivenessArtifact; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.TrackingRowSet; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.BaseTable; +import io.deephaven.engine.table.impl.OperationSnapshotControl; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; +import io.deephaven.engine.table.impl.sources.RowSetColumnSourceWrapper; +import io.deephaven.engine.updategraph.UpdateGraph; +import io.deephaven.util.SafeCloseable; +import org.apache.commons.lang3.mutable.Mutable; +import org.apache.commons.lang3.mutable.MutableObject; +import org.jetbrains.annotations.NotNull; + +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * Abstract base implementation of {@link DataIndex} that provides common functionality. + */ +public abstract class AbstractDataIndex extends LivenessArtifact implements DataIndex { + + /** + * The name of the {@link RowSet} column in the index {@link #table()} used by all {@link AbstractDataIndex} + * instances. + */ + protected static final String ROW_SET_COLUMN_NAME = "dh_row_set"; + + @Override + @NotNull + public final String rowSetColumnName() { + return ROW_SET_COLUMN_NAME; + } + + @Override + @NotNull + public final BasicDataIndex transform(@NotNull final DataIndexTransformer transformer) { + return TransformedDataIndex.from(this, transformer); + } + + @Override + @NotNull + public final DataIndex remapKeyColumns(@NotNull final Map, ColumnSource> oldToNewColumnMap) { + return RemappedDataIndex.from(this, oldToNewColumnMap); + } + + /** + * Whether this AbstractDataIndex is potentially usable. This will return {@code true} when there are no known + * issues for this data index. This performs fast checks, such as verifying all locations have index table files, + * but does not fully guarantee that the index is complete and loadable. + * + * @return true If the AbstractDataIndex is potentially usable, false otherwise + */ + public abstract boolean isValid(); + + /** + * Return a copy of {@code parent} with the {@link RowSet} column replaced with a {@link RowSetColumnSourceWrapper + * wrapper} column that adds {@link TrackingRowSet#prev() prev} calls on access to previous values. + * + * @param parent The {@link Table} to copy + * @param rowSetColumn The name of the {@link RowSet} column to wrap + * @return The copied {@link Table} + */ + @SuppressWarnings("unused") + protected static QueryTable indexTableWrapper( + @NotNull final QueryTable parent, + @NotNull final String rowSetColumn) { + return indexTableWrapper(parent, rowSetColumn, rowSetColumn); + } + + /** + * Return a copy of {@code parent} with the {@link RowSet} column replaced with a {@link RowSetColumnSourceWrapper + * wrapper} column that adds {@link TrackingRowSet#prev() prev} calls on access to previous values. + * + * @param parent The {@link Table} to copy + * @param rowSetColumn The name of the {@link RowSet} column to wrap + * @param renamedRowSetColumn The name of the {@link RowSet} column in the output {@link Table} + * @return The copied {@link Table} + */ + protected static QueryTable indexTableWrapper( + @NotNull final QueryTable parent, + @NotNull final String rowSetColumn, + @NotNull final String renamedRowSetColumn) { + parent.getDefinition().checkHasColumn(rowSetColumn, RowSet.class); + if (!rowSetColumn.equals(renamedRowSetColumn) && + parent.getDefinition().getColumnNameSet().contains(renamedRowSetColumn)) { + throw new IllegalArgumentException(String.format( + "Cannot rename %s to %s, table already contains a column named %s in %s", + rowSetColumn, renamedRowSetColumn, renamedRowSetColumn, parent.getDefinition().getColumnNames())); + } + + final UpdateGraph updateGraph = parent.getUpdateGraph(); + try (final SafeCloseable ignored = ExecutionContext.getContext().withUpdateGraph(updateGraph).open()) { + return QueryPerformanceRecorder.withNugget("wrapRowSetColumn()", parent.sizeForInstrumentation(), () -> { + final Mutable result = new MutableObject<>(); + final LinkedHashMap> resultColumnSourceMap = + new LinkedHashMap<>(parent.numColumns()); + parent.getColumnSourceMap().forEach((columnName, columnSource) -> { + if (columnName.equals(rowSetColumn)) { + resultColumnSourceMap.put( + renamedRowSetColumn, + RowSetColumnSourceWrapper.from(parent.getColumnSource(rowSetColumn))); + } else { + resultColumnSourceMap.put(columnName, columnSource); + } + }); + final OperationSnapshotControl snapshotControl = + parent.createSnapshotControlIfRefreshing(OperationSnapshotControl::new); + QueryTable.initializeWithSnapshot("wrapRowSetColumn", snapshotControl, (usePrev, beforeClockValue) -> { + final QueryTable resultTable = new QueryTable(TableDefinition.inferFrom(resultColumnSourceMap), + parent.getRowSet(), resultColumnSourceMap, null, parent.getAttributes()); + parent.propagateFlatness(resultTable); + if (snapshotControl != null) { + final BaseTable.ListenerImpl listener = + new BaseTable.ListenerImpl("wrapRowSetColumn()", parent, resultTable); + snapshotControl.setListenerAndResult(listener, resultTable); + } + + result.setValue(resultTable); + return true; + }); + + return result.getValue(); + }); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexBoxedKeySourceCompound.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexBoxedKeySourceCompound.java new file mode 100644 index 00000000000..bad6e07f942 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexBoxedKeySourceCompound.java @@ -0,0 +1,121 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.DataIndex; +import io.deephaven.engine.table.SharedContext; +import io.deephaven.engine.table.impl.DefaultChunkSource; +import io.deephaven.engine.table.impl.chunkboxer.ChunkBoxer; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.NotNull; + +import java.util.Arrays; +import java.util.stream.Stream; + +/** + * {@link ChunkSource} that produces data index {@link DataIndex.RowKeyLookup row set lookup} keys from multiple + * {@link ColumnSource sources}. This can be used to extract keys from a data index table, or from a table of probe + * values. + */ +final class DataIndexBoxedKeySourceCompound implements DefaultChunkSource.WithPrev { + + private final ColumnSource[] keySources; + private final int keyWidth; + + /** + * Construct a new DataIndexBoxedKeySourceCompound backed by the supplied {@link ColumnSource column sources}. + * + * @param keySources Sources corresponding to the key columns + */ + DataIndexBoxedKeySourceCompound(@NotNull final ColumnSource... keySources) { + this.keySources = Arrays.stream(keySources) + .map(ReinterpretUtils::maybeConvertToPrimitive).toArray(ColumnSource[]::new); + keyWidth = keySources.length; + } + + @Override + public ChunkType getChunkType() { + return ChunkType.Object; + } + + @Override + public void fillChunk( + @NotNull final ChunkSource.FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + fillChunkInternal(context, destination, rowSequence, false); + } + + public void fillPrevChunk( + @NotNull final ChunkSource.FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + fillChunkInternal(context, destination, rowSequence, true); + } + + private void fillChunkInternal( + @NotNull final ChunkSource.FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence, + final boolean usePrev) { + if (rowSequence.isEmpty()) { + destination.setSize(0); + return; + } + final FillContext fc = (FillContext) context; + for (int ci = 0; ci < keyWidth; ++ci) { + fc.boxedKeys[ci] = fc.keyBoxers[ci].box(usePrev + ? keySources[ci].getPrevChunk(fc.keyContexts[ci], rowSequence) + : keySources[ci].getChunk(fc.keyContexts[ci], rowSequence)); + } + final int size = rowSequence.intSize(); + destination.setSize(size); + final WritableObjectChunk typedDestination = destination.asWritableObjectChunk(); + for (int ri = 0; ri < size; ++ri) { + final Object[] columnValues = new Object[keyWidth]; + for (int ci = 0; ci < keyWidth; ++ci) { + columnValues[ci] = fc.boxedKeys[ci].get(ri); + } + typedDestination.set(ri, columnValues); + } + } + + private static class FillContext implements ChunkSource.FillContext { + + private final GetContext[] keyContexts; + private final ChunkBoxer.BoxerKernel[] keyBoxers; + private final ObjectChunk[] boxedKeys; + + private FillContext( + final int chunkCapacity, + @NotNull final ColumnSource[] keySources, + final SharedContext sharedContext) { + keyContexts = Stream.of(keySources) + .map(cs -> cs.makeGetContext(chunkCapacity, sharedContext)) + .toArray(GetContext[]::new); + keyBoxers = Stream.of(keySources) + .map(cs -> ChunkBoxer.getBoxer(cs.getChunkType(), chunkCapacity)) + .toArray(ChunkBoxer.BoxerKernel[]::new); + // noinspection unchecked + boxedKeys = new ObjectChunk[keySources.length]; + } + + @Override + public void close() { + SafeCloseable.closeAll(keyContexts); + SafeCloseable.closeAll(keyBoxers); + } + } + + @Override + public ChunkSource.FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { + return new FillContext(chunkCapacity, keySources, sharedContext); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexBoxedKeySourceSingle.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexBoxedKeySourceSingle.java new file mode 100644 index 00000000000..ef2a552785a --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexBoxedKeySourceSingle.java @@ -0,0 +1,99 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.DataIndex; +import io.deephaven.engine.table.SharedContext; +import io.deephaven.engine.table.impl.DefaultChunkSource; +import io.deephaven.engine.table.impl.chunkboxer.ChunkBoxer; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.NotNull; + +/** + * {@link ChunkSource} that produces data index {@link DataIndex.RowKeyLookup row set lookup} keys from a single + * {@link ColumnSource source}. This can be used to extract keys from a data index table, or from a table of probe + * values. + */ +final class DataIndexBoxedKeySourceSingle implements DefaultChunkSource.WithPrev { + + private final ColumnSource keySource; + + /** + * Construct a new DataIndexBoxedKeySourceSingle backed by the supplied {@link ColumnSource column source}. + * + * @param keySource Source corresponding to the key column + */ + DataIndexBoxedKeySourceSingle(@NotNull final ColumnSource keySource) { + this.keySource = ReinterpretUtils.maybeConvertToPrimitive(keySource); + } + + @Override + public ChunkType getChunkType() { + return ChunkType.Object; + } + + @Override + public void fillChunk( + @NotNull final ChunkSource.FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + fillChunkInternal(context, destination, rowSequence, false); + } + + public void fillPrevChunk( + @NotNull final ChunkSource.FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + fillChunkInternal(context, destination, rowSequence, true); + } + + private void fillChunkInternal( + @NotNull final ChunkSource.FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence, + final boolean usePrev) { + if (rowSequence.isEmpty()) { + destination.setSize(0); + return; + } + final FillContext fc = (FillContext) context; + final ObjectChunk boxedKeys = fc.keyBoxer.box(usePrev + ? keySource.getPrevChunk(fc.keyContext, rowSequence) + : keySource.getChunk(fc.keyContext, rowSequence)); + destination.setSize(boxedKeys.size()); + boxedKeys.copyToChunk(0, destination, 0, boxedKeys.size()); + } + + private static class FillContext implements ChunkSource.FillContext { + + private final GetContext keyContext; + private final ChunkBoxer.BoxerKernel keyBoxer; + + private FillContext( + final int chunkCapacity, + @NotNull final ColumnSource keySource, + final SharedContext sharedContext) { + keyContext = keySource.makeGetContext(chunkCapacity, sharedContext); + keyBoxer = ChunkBoxer.getBoxer(keySource.getChunkType(), chunkCapacity); + } + + @Override + public void close() { + SafeCloseable.closeAll(keyContext, keyBoxer); + } + } + + @Override + public ChunkSource.FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { + return new FillContext(chunkCapacity, keySource, sharedContext); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexKeySet.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexKeySet.java new file mode 100644 index 00000000000..ea48df067b1 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexKeySet.java @@ -0,0 +1,26 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import org.jetbrains.annotations.NotNull; + +import java.util.function.Consumer; + +/** + * A subset of the methods of {@link java.util.Set}, to be used for storing a representation of the keys in a + * {@link io.deephaven.engine.table.DataIndex}. Single key values are stored as reinterpreted-to-primitive, boxed + * {@link Object Objects}. Compound keys are stored as {@link Object[] Object arrays} of the same. + */ +public interface DataIndexKeySet { + + boolean add(Object key); + + boolean remove(Object key); + + boolean contains(Object key); + + void forEach(@NotNull final Consumer action); + + Object[] toArray(); +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexKeySetCompound.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexKeySetCompound.java new file mode 100644 index 00000000000..b89900eb992 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexKeySetCompound.java @@ -0,0 +1,73 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.base.verify.Require; +import io.deephaven.hash.KeyedObjectHashSet; +import io.deephaven.hash.KeyedObjectKey; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Arrays; +import java.util.function.Consumer; + +class DataIndexKeySetCompound implements DataIndexKeySet { + + private static final KeyedObjectKey KEY_HASH_ADAPTER = new KeyHashAdapter(); + + private static class KeyHashAdapter implements KeyedObjectKey { + + private KeyHashAdapter() {} + + @Override + public Object getKey(@NotNull final Object value) { + return value; + } + + @Override + public int hashKey(@Nullable final Object key) { + return Arrays.hashCode((Object[]) key); + } + + @Override + public boolean equalKey(@Nullable final Object key, @NotNull final Object value) { + return Arrays.equals((Object[]) key, (Object[]) value); + } + } + + private final KeyedObjectHashSet set; + + DataIndexKeySetCompound(final int initialCapacity) { + set = new KeyedObjectHashSet<>(Require.gtZero(initialCapacity, "initialCapacity"), KEY_HASH_ADAPTER); + } + + DataIndexKeySetCompound() { + set = new KeyedObjectHashSet<>(KEY_HASH_ADAPTER); + } + + @Override + public boolean add(final Object key) { + return set.add(key); + } + + @Override + public boolean remove(final Object key) { + return set.removeValue(key); + } + + @Override + public boolean contains(final Object key) { + return set.containsKey(key); + } + + @Override + public void forEach(@NotNull final Consumer action) { + set.forEach(action); + } + + @Override + public Object[] toArray() { + return set.toArray(); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexKeySetSingle.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexKeySetSingle.java new file mode 100644 index 00000000000..3fb23bee4bb --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexKeySetSingle.java @@ -0,0 +1,88 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.base.verify.Require; +import io.deephaven.hash.KeyedObjectHashSet; +import io.deephaven.hash.KeyedObjectKey; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.function.Consumer; + +class DataIndexKeySetSingle implements DataIndexKeySet { + + private static final KeyedObjectKey KEY_HASH_ADAPTER = new KeyHashAdapter(); + + private static class KeyHashAdapter implements KeyedObjectKey { + + private KeyHashAdapter() {} + + @Override + public Object getKey(@NotNull final Object value) { + return value; + } + + @Override + public int hashKey(@NotNull final Object key) { + return key.hashCode(); + } + + @Override + public boolean equalKey(@NotNull final Object key, @NotNull final Object value) { + return key.equals(value); + } + } + + private static final Object NULL_OBJECT_KEY = new Object(); + + private final KeyedObjectHashSet set; + + DataIndexKeySetSingle(final int initialCapacity) { + set = new KeyedObjectHashSet<>(Require.gtZero(initialCapacity, "initialCapacity"), KEY_HASH_ADAPTER); + } + + DataIndexKeySetSingle() { + set = new KeyedObjectHashSet<>(KEY_HASH_ADAPTER); + } + + @Override + public boolean add(final Object key) { + return set.add(adaptKey(key)); + } + + @Override + public boolean remove(final Object key) { + return set.remove(adaptKey(key)); + } + + @Override + public boolean contains(final Object key) { + return set.containsKey(adaptKey(key)); + } + + @NotNull + private static Object adaptKey(@Nullable final Object key) { + return key == null ? NULL_OBJECT_KEY : key; + } + + @Override + public void forEach(@NotNull final Consumer action) { + set.forEach(key -> action.accept(key == NULL_OBJECT_KEY ? null : key)); + } + + @Override + public Object[] toArray() { + final Object[] result = set.toArray(); + if (set.containsKey(NULL_OBJECT_KEY)) { + for (int ii = 0; ii < result.length; ii++) { + if (result[ii] == NULL_OBJECT_KEY) { + result[ii] = null; + break; + } + } + } + return result; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexUtils.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexUtils.java new file mode 100644 index 00000000000..6a94b06112e --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexUtils.java @@ -0,0 +1,173 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.primitive.iterator.CloseableIterator; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.iterators.ChunkedColumnIterator; +import io.deephaven.hash.KeyedObjectHashMap; +import io.deephaven.hash.KeyedObjectKey; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.Arrays; +import java.util.Objects; +import java.util.function.ToLongFunction; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; + +/** + * Tools for working with {@link DataIndex data indices}. + */ +public class DataIndexUtils { + + /** + * Make a {@link ChunkSource} that produces data index {@link DataIndex.RowKeyLookup lookup} keys from + * {@code keySources}. + * + * @param keySources The individual key sources + * @return The boxed key source + */ + public static ChunkSource.WithPrev makeBoxedKeySource(@NotNull final ColumnSource... keySources) { + switch (keySources.length) { + case 0: + throw new IllegalArgumentException("Data index must have at least one key column"); + case 1: + return new DataIndexBoxedKeySourceSingle(keySources[0]); + default: + return new DataIndexBoxedKeySourceCompound(keySources); + } + } + + /** + * Make a {@link DataIndexKeySet} that stores data index {@link DataIndex.RowKeyLookup lookup} keys that have + * {@code keyColumnCount} components. + * + * @param keyColumnCount The number of key components + * @return The key set + */ + public static DataIndexKeySet makeKeySet(final int keyColumnCount) { + if (keyColumnCount == 1) { + return new DataIndexKeySetSingle(); + } + if (keyColumnCount > 1) { + return new DataIndexKeySetCompound(); + } + throw new IllegalArgumentException("Data index must have at least one key column"); + } + + /** + * Make a {@link DataIndexKeySet} that stores data index {@link DataIndex.RowKeyLookup lookup} keys that have + * {@code keyColumnCount} components. + * + * @param keyColumnCount The number of key components + * @param initialCapacity The initial capacity + * @return The key set + */ + @SuppressWarnings("unused") + public static DataIndexKeySet makeKeySet(final int keyColumnCount, final int initialCapacity) { + if (keyColumnCount == 1) { + return new DataIndexKeySetSingle(initialCapacity); + } + if (keyColumnCount > 1) { + return new DataIndexKeySetCompound(initialCapacity); + } + throw new IllegalArgumentException("Data index must have at least one key column"); + } + + /** + * Test equality between two data index {@link DataIndex.RowKeyLookup lookup} keys. + * + * @return Whether the two keys are equal + */ + public static boolean lookupKeysEqual(@Nullable final Object key1, @Nullable final Object key2) { + if (key1 instanceof Object[] && key2 instanceof Object[]) { + return Arrays.equals((Object[]) key1, (Object[]) key2); + } + return Objects.equals(key1, key2); + } + + /** + * Compute the hash code for a data index {@link DataIndex.RowKeyLookup lookup} key. + * + * @param key The lookup key + * @return The hash code + */ + public static int hashLookupKey(@Nullable final Object key) { + if (key instanceof Object[]) { + return Arrays.hashCode((Object[]) key); + } + return Objects.hashCode(key); + } + + /** + * Build a mapping function from the lookup keys of the provided index {@link Table} to row keys in the table. + * + * @param indexTable The {@link Table} to search + * @param keyColumnNames The key columns to search + * @return A mapping function from lookup keys to {@code indexTable} row keys + */ + public static ToLongFunction buildRowKeyMappingFunction( + final Table indexTable, + final String[] keyColumnNames) { + + final KeyedObjectHashMap lookupKeyToRowKeyPairs = + new KeyedObjectHashMap<>(LookupKeyRowKeyPair.KEYED_OBJECT_KEY); + final ChunkSource.WithPrev lookupKeySource = makeBoxedKeySource(Arrays.stream(keyColumnNames) + .map(indexTable::getColumnSource) + .toArray(ColumnSource[]::new)); + try (final CloseableIterator lookupKeyIterator = + ChunkedColumnIterator.make(lookupKeySource, indexTable.getRowSet()); + final RowSet.Iterator rowKeyIterator = indexTable.getRowSet().iterator()) { + while (lookupKeyIterator.hasNext()) { + final Object lookupKey = lookupKeyIterator.next(); + final long rowKey = rowKeyIterator.nextLong(); + lookupKeyToRowKeyPairs.put(lookupKey, new LookupKeyRowKeyPair(lookupKey, rowKey)); + } + } + + return (final Object lookupKey) -> { + final LookupKeyRowKeyPair pair = lookupKeyToRowKeyPairs.get(lookupKey); + return pair == null ? NULL_ROW_KEY : pair.rowKey; + }; + } + + private static final class LookupKeyRowKeyPair { + + private static final KeyedObjectKey KEYED_OBJECT_KEY = + new LookupKeyedObjectKey<>() { + + @Override + public Object getKey(@NotNull final LookupKeyRowKeyPair lookupKeyRowKeyPair) { + return lookupKeyRowKeyPair.lookupKey; + } + }; + + private final Object lookupKey; + private final long rowKey; + + private LookupKeyRowKeyPair(final Object lookupKey, final long rowKey) { + this.lookupKey = lookupKey; + this.rowKey = rowKey; + } + } + + /** + * A {@link KeyedObjectKey} that for values keyed by a lookup key. + */ + public static abstract class LookupKeyedObjectKey implements KeyedObjectKey { + + @Override + public int hashKey(final Object key) { + return hashLookupKey(key); + } + + @Override + public boolean equalKey(final Object key, final VALUE_TYPE value) { + return lookupKeysEqual(key, getKey(value)); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/RemappedDataIndex.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/RemappedDataIndex.java new file mode 100644 index 00000000000..e8c15270359 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/RemappedDataIndex.java @@ -0,0 +1,112 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.DataIndex; +import io.deephaven.engine.table.Table; +import org.jetbrains.annotations.NotNull; + +import java.util.*; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * A {@link AbstractDataIndex} that remaps the key columns of another {@link AbstractDataIndex}. Used to implement + * {@link io.deephaven.engine.table.DataIndex#remapKeyColumns(Map)}. + */ +public class RemappedDataIndex extends AbstractDataIndex { + + private final AbstractDataIndex sourceIndex; + private final Map, ColumnSource> oldToNewColumnMap; + + private final Map, String> keyColumnNamesByIndexedColumn; + + public static DataIndex from( + @NotNull final AbstractDataIndex sourceIndex, + @NotNull final Map, ColumnSource> oldToNewColumnMap) { + if (sourceIndex instanceof RemappedDataIndex) { + // We should chain the remappings and delegate to the original source index. + final RemappedDataIndex sourceAsRemapped = (RemappedDataIndex) sourceIndex; + final Map, ColumnSource> sourceOldToNewColumnMap = sourceAsRemapped.oldToNewColumnMap; + final Map, ColumnSource> chainedOldToNewColumnMap = + sourceAsRemapped.sourceIndex.keyColumnNamesByIndexedColumn().keySet().stream() + .filter((final ColumnSource originalColumnSource) -> sourceOldToNewColumnMap + .containsKey(originalColumnSource) + || oldToNewColumnMap.containsKey(originalColumnSource)) + .collect(Collectors.toMap( + Function.identity(), + (final ColumnSource originalColumnSource) -> { + final ColumnSource sourceReplacement = + sourceOldToNewColumnMap.get(originalColumnSource); + if (sourceReplacement != null) { + final ColumnSource chainedReplacement = + oldToNewColumnMap.get(sourceReplacement); + return chainedReplacement == null ? sourceReplacement : chainedReplacement; + } + return Objects.requireNonNull(oldToNewColumnMap.get(originalColumnSource)); + })); + return new RemappedDataIndex(sourceAsRemapped.sourceIndex, chainedOldToNewColumnMap); + } + return new RemappedDataIndex(sourceIndex, oldToNewColumnMap); + } + + private RemappedDataIndex( + @NotNull final AbstractDataIndex sourceIndex, + @NotNull final Map, ColumnSource> oldToNewColumnMap) { + this.sourceIndex = sourceIndex; + this.oldToNewColumnMap = oldToNewColumnMap; + // Build a new map of column sources to index table key column names using either the original column + // sources or the remapped column sources. + keyColumnNamesByIndexedColumn = sourceIndex.keyColumnNamesByIndexedColumn().entrySet().stream() + .collect(Collectors.toMap( + (final Map.Entry, String> entry) -> { + // Use the remapped column source (or the original source if not remapped) as the key + final ColumnSource oldColumnSource = entry.getKey(); + final ColumnSource newColumnSource = oldToNewColumnMap.get(oldColumnSource); + return newColumnSource != null ? newColumnSource : oldColumnSource; + }, + Map.Entry::getValue, + Assert::neverInvoked, + LinkedHashMap::new)); + if (sourceIndex.isRefreshing()) { + manage(sourceIndex); + } + } + + @Override + @NotNull + public Map, String> keyColumnNamesByIndexedColumn() { + return keyColumnNamesByIndexedColumn; + } + + @Override + @NotNull + public List keyColumnNames() { + return sourceIndex.keyColumnNames(); + } + + @Override + @NotNull + public Table table() { + return sourceIndex.table(); + } + + @Override + @NotNull + public RowKeyLookup rowKeyLookup() { + return sourceIndex.rowKeyLookup(); + } + + @Override + public boolean isRefreshing() { + return sourceIndex.isRefreshing(); + } + + @Override + public boolean isValid() { + return sourceIndex.isValid(); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/RowSetCodec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/RowSetCodec.java new file mode 100644 index 00000000000..3578b4387be --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/RowSetCodec.java @@ -0,0 +1,69 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.impl.ExternalizableRowSetUtils; +import io.deephaven.util.codec.ObjectCodec; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.*; + +/** + * A codec to encode and decode generic row set to a column. + */ +public class RowSetCodec implements ObjectCodec { + + public RowSetCodec(@SuppressWarnings("unused") String arguments) {} + + @Override + @NotNull + public byte[] encode(@Nullable final RowSet input) { + if (input == null) { + throw new UnsupportedOperationException(getClass() + " does not support null input"); + } + try { + final ByteArrayOutputStream byteOutput = new ByteArrayOutputStream(); + final DataOutputStream dataOutputStream = new DataOutputStream(byteOutput); + ExternalizableRowSetUtils.writeExternalCompressedDeltas(dataOutputStream, input); + dataOutputStream.flush(); + return byteOutput.toByteArray(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public boolean isNullable() { + return false; + } + + @Override + public int getPrecision() { + return 0; + } + + @Override + public int getScale() { + return 0; + } + + @Nullable + @Override + public RowSet decode(@NotNull final byte[] input, final int offset, final int length) { + try { + final ByteArrayInputStream byteInput = new ByteArrayInputStream(input, offset, length); + final DataInputStream dataInputStream = new DataInputStream(byteInput); + return ExternalizableRowSetUtils.readExternalCompressedDelta(dataInputStream); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public int expectedObjectWidth() { + return VARIABLE_WIDTH_SENTINEL; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/StandaloneDataIndex.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/StandaloneDataIndex.java new file mode 100644 index 00000000000..b496c96171b --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/StandaloneDataIndex.java @@ -0,0 +1,74 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.engine.liveness.LivenessArtifact; +import io.deephaven.engine.table.BasicDataIndex; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import org.jetbrains.annotations.NotNull; + +import java.util.List; +import java.util.Map; + +/** + * {@link BasicDataIndex} implementation that holds an index {@link Table} and does not specify the {@link ColumnSource + * ColumnSources} that were indexed, and hence cannot support {@link #keyColumnNamesByIndexedColumn()}. This is useful + * for standalone indices that are not associated with a specific table, but rather used to accumulate a merged index + * for a merged table over the indexed data. + */ +public class StandaloneDataIndex extends LivenessArtifact implements BasicDataIndex { + + private final Table table; + private final List keyColumnNames; + private final String rowSetColumnName; + + public static StandaloneDataIndex from( + @NotNull final Table table, + @NotNull final String[] keyColumnNames, + @NotNull final String rowSetColumnName) { + return new StandaloneDataIndex(table.coalesce(), keyColumnNames, rowSetColumnName); + } + + private StandaloneDataIndex( + @NotNull final Table table, + @NotNull final String[] keyColumnNames, + @NotNull final String rowSetColumnName) { + this.table = table; + this.keyColumnNames = List.of(keyColumnNames); + this.rowSetColumnName = rowSetColumnName; + if (table.isRefreshing()) { + manage(table); + } + } + + @Override + @NotNull + public List keyColumnNames() { + return keyColumnNames; + } + + @Override + @NotNull + public Map, String> keyColumnNamesByIndexedColumn() { + throw new UnsupportedOperationException("Cannot provide a key column map for a standalone data index"); + } + + @Override + @NotNull + public String rowSetColumnName() { + return rowSetColumnName; + } + + @Override + @NotNull + public Table table() { + return table; + } + + @Override + public boolean isRefreshing() { + return table.isRefreshing(); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/TableBackedDataIndex.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/TableBackedDataIndex.java new file mode 100644 index 00000000000..25f3637fdef --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/TableBackedDataIndex.java @@ -0,0 +1,156 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.api.ColumnName; +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.by.*; +import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; +import io.deephaven.util.SafeCloseable; +import org.apache.commons.lang3.mutable.MutableObject; +import org.jetbrains.annotations.NotNull; + +import java.util.*; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static io.deephaven.engine.table.impl.by.AggregationProcessor.EXPOSED_GROUP_ROW_SETS; + +/** + * This class creates a possibly-{@link AbstractDataIndex#isRefreshing() refreshing} {@link AbstractDataIndex} for a + * table. At its core, the index is a {@link Table} containing the key column(s) and the + * {@link io.deephaven.engine.rowset.RowSet RowSets} that contain these values. + */ +public class TableBackedDataIndex extends AbstractDataIndex { + + @NotNull + private final Map, String> keyColumnNamesByIndexedColumn; + + @NotNull + final List keyColumnNames; + + private final boolean isRefreshing; + + private QueryTable sourceTable; + + /** + * The lookup function for the index table. Note that this is always set before {@link #indexTable}. + */ + private AggregationRowLookup lookupFunction; + + /** + * The index table. Note that we use this as a barrier to ensure {@link #lookupFunction} is visible. + */ + private volatile Table indexTable; + + public TableBackedDataIndex( + @NotNull final QueryTable sourceTable, + @NotNull final String... keyColumnNames) { + this.keyColumnNames = List.of(keyColumnNames); + + // Create an in-order reverse lookup map for the key column names. + keyColumnNamesByIndexedColumn = Collections.unmodifiableMap( + Arrays.stream(keyColumnNames).collect(Collectors.toMap( + sourceTable::getColumnSource, Function.identity(), Assert::neverInvoked, LinkedHashMap::new))); + + isRefreshing = sourceTable.isRefreshing(); + + // Defer the actual index table and lookup function creation until they are needed. + this.sourceTable = sourceTable; + if (isRefreshing) { + manage(sourceTable); + } + } + + @Override + @NotNull + public List keyColumnNames() { + return keyColumnNames; + } + + @Override + @NotNull + public Map, String> keyColumnNamesByIndexedColumn() { + return keyColumnNamesByIndexedColumn; + } + + @Override + @NotNull + public Table table() { + Table localIndexTable; + if ((localIndexTable = indexTable) != null) { + return localIndexTable; + } + synchronized (this) { + if ((localIndexTable = indexTable) != null) { + return localIndexTable; + } + + return computeTable(); + } + } + + /** + * Compute {@link #indexTable} and {@link #lookupFunction}. + * + * @return The newly-computed index table + */ + private Table computeTable() { + final MutableObject resultLookupFunction = new MutableObject<>(); + final Table resultIndexTable = QueryPerformanceRecorder.withNugget(String.format( + "Build table-backed DataIndex for %s on [%s]", + sourceTable.getDescription(), String.join(", ", keyColumnNames)), () -> { + try (final SafeCloseable ignored1 = + ExecutionContext.getContext().withUpdateGraph(sourceTable.getUpdateGraph()).open(); + final SafeCloseable ignored2 = isRefreshing() ? LivenessScopeStack.open() : null) { + final QueryTable groupedTable = ChunkedOperatorAggregationHelper.aggregation( + AggregationControl.IGNORE_INDEXING, AggregationProcessor.forExposeGroupRowSets(), + sourceTable, false, null, ColumnName.from(keyColumnNames)); + + resultLookupFunction.setValue(AggregationProcessor.getRowLookup(groupedTable)); + Assert.neqNull(resultLookupFunction.getValue(), "AggregationRowLookup"); + + final Table withWrappedRowSetSource = + indexTableWrapper(groupedTable, EXPOSED_GROUP_ROW_SETS.name(), ROW_SET_COLUMN_NAME); + if (isRefreshing()) { + manage(withWrappedRowSetSource); + } + return withWrappedRowSetSource; + } + }); + lookupFunction = resultLookupFunction.getValue(); + indexTable = resultIndexTable; + if (isRefreshing) { + unmanage(sourceTable); + } + sourceTable = null; + return resultIndexTable; + } + + @Override + @NotNull + public RowKeyLookup rowKeyLookup() { + table(); + return (final Object key, final boolean usePrev) -> { + // Pass the object to the aggregation lookup, then return the resulting row key. This index will be + // correct in prev or current space because of the aggregation's hash-based lookup. + return lookupFunction.get(key); + }; + } + + @Override + public boolean isRefreshing() { + return isRefreshing; + } + + @Override + public boolean isValid() { + return true; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/TransformedDataIndex.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/TransformedDataIndex.java new file mode 100644 index 00000000000..38e01019c8b --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/TransformedDataIndex.java @@ -0,0 +1,186 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.dataindex; + +import io.deephaven.api.ColumnName; +import io.deephaven.api.filter.Filter; +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.liveness.LivenessArtifact; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.table.BasicDataIndex; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.DataIndex; +import io.deephaven.engine.table.DataIndexTransformer; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.select.FunctionalColumn; +import io.deephaven.engine.table.impl.select.FunctionalColumnLong; +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +public class TransformedDataIndex extends LivenessArtifact implements BasicDataIndex { + + @NotNull + private final DataIndex parentIndex; + private final boolean isRefreshing; + + private DataIndexTransformer transformer; + + private volatile Table indexTable; + + public static TransformedDataIndex from( + @NotNull final DataIndex index, + @NotNull final DataIndexTransformer transformer) { + return new TransformedDataIndex(index, transformer); + } + + private TransformedDataIndex( + @NotNull final DataIndex parentIndex, + @NotNull final DataIndexTransformer transformer) { + this.parentIndex = parentIndex; + isRefreshing = !transformer.snapshotResult() && parentIndex.isRefreshing(); + this.transformer = transformer; + } + + @Override + @NotNull + public List keyColumnNames() { + return parentIndex.keyColumnNames(); + } + + @Override + @NotNull + public Map, String> keyColumnNamesByIndexedColumn() { + return parentIndex.keyColumnNamesByIndexedColumn(); + } + + @Override + @NotNull + public String rowSetColumnName() { + return parentIndex.rowSetColumnName(); + } + + @Override + @NotNull + public Table table() { + Table localIndexTable; + if ((localIndexTable = indexTable) != null) { + return localIndexTable; + } + synchronized (this) { + if ((localIndexTable = indexTable) != null) { + return localIndexTable; + } + + try (final SafeCloseable ignored = parentIndex.isRefreshing() ? LivenessScopeStack.open() : null) { + localIndexTable = parentIndex.table(); + localIndexTable = maybeIntersectAndInvert(localIndexTable); + localIndexTable = maybeSortByFirstKey(localIndexTable); + localIndexTable = localIndexTable.isRefreshing() && transformer.snapshotResult() + ? localIndexTable.snapshot() + : localIndexTable; + + if (localIndexTable.isRefreshing()) { + manage(localIndexTable); + } + + indexTable = localIndexTable; + // Don't hold onto the transformer after the index table is computed, we don't need to maintain + // reachability for its RowSets anymore. + transformer = null; + return localIndexTable; + } + } + } + + @Override + public boolean isRefreshing() { + return isRefreshing; + } + + // region DataIndex materialization operations + + private static Function getMutator( + @Nullable final RowSet intersectRowSet, + @Nullable final RowSet invertRowSet) { + final Function mutator; + if (invertRowSet == null) { + assert intersectRowSet != null; + // Only intersect + mutator = (final RowSet rowSet) -> { + final WritableRowSet intersected = rowSet.intersect(intersectRowSet); + if (intersected.isEmpty()) { + intersected.close(); + return null; + } + return intersected; + }; + } else if (intersectRowSet == null) { + // Only invert + mutator = invertRowSet::invert; + } else { + // Intersect and invert + mutator = (final RowSet rowSet) -> { + try (final WritableRowSet intersected = rowSet.intersect(intersectRowSet)) { + if (intersected.isEmpty()) { + return null; + } + return invertRowSet.invert(intersected); + } + }; + } + return mutator; + } + + /** + * Apply the intersect and invert transformations to the input index table, if specified. + * + * @param indexTable The input index table + * @return The table with intersections and inversions applied. + */ + private Table maybeIntersectAndInvert(@NotNull final Table indexTable) { + Assert.neqNull(transformer, "transformer"); + if (transformer.intersectRowSet().isEmpty() && transformer.invertRowSet().isEmpty()) { + return indexTable; + } + final Function mutator = + getMutator(transformer.intersectRowSet().orElse(null), transformer.invertRowSet().orElse(null)); + final Table mutated = indexTable + .update(List.of(new FunctionalColumn<>( + parentIndex.rowSetColumnName(), RowSet.class, + parentIndex.rowSetColumnName(), RowSet.class, + mutator))); + if (transformer.intersectRowSet().isPresent()) { + return mutated.where(Filter.isNotNull(ColumnName.of(parentIndex.rowSetColumnName()))); + } + return mutated; + } + + /** + * Sort the input index table by the first row key of its RowSet column, if specified. + * + * @param indexTable The input index table + * @return The table sorted by first row key, if requested, else the input index table + */ + protected Table maybeSortByFirstKey(final @NotNull Table indexTable) { + Assert.neqNull(transformer, "transformer"); + if (!transformer.sortByFirstRowKey()) { + return indexTable; + } + return indexTable + .updateView(List.of(new FunctionalColumnLong<>( + parentIndex.rowSetColumnName(), RowSet.class, + "__FRK__", RowSet::firstRowKey))) + .sort("__FRK__") + .dropColumns("__FRK__"); + } + + // endregion DataIndex materialization operations +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupNodeKeySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupNodeKeySource.java index c048497db61..3150843003d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupNodeKeySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/hierarchical/RollupNodeKeySource.java @@ -181,7 +181,7 @@ private static class FillContext implements ChunkSource.FillContext { private FillContext( final int chunkCapacity, @NotNull final ColumnSource depthSource, - @NotNull final ColumnSource[] groupByValueSources, + @NotNull final ColumnSource[] groupByValueSources, final SharedContext sharedContext) { depthContext = depthSource.makeGetContext(chunkCapacity, sharedContext); groupByValueContexts = Stream.of(groupByValueSources) @@ -201,7 +201,7 @@ public void close() { } @Override - public FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { + public ChunkSource.FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { return new FillContext(chunkCapacity, depthSource, groupByValueSources, sharedContext); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/DataIndexer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/DataIndexer.java new file mode 100644 index 00000000000..09065809080 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/DataIndexer.java @@ -0,0 +1,514 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.indexer; + +import com.google.common.collect.Sets; +import io.deephaven.base.verify.Require; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.TrackingRowSet; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.DataIndex; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.dataindex.AbstractDataIndex; +import io.deephaven.engine.table.impl.dataindex.TableBackedDataIndex; +import io.deephaven.engine.table.impl.util.FieldUtils; +import io.deephaven.engine.updategraph.UpdateGraph; +import org.apache.commons.lang3.mutable.MutableObject; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.lang.ref.Reference; +import java.lang.ref.WeakReference; +import java.util.*; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.Predicate; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +/** + * Indexer that provides single and multi-column clustering indexes for a table, linked to a {@link TrackingRowSet}. + * + * @apiNote DataIndexers should not be used after the host {@link TrackingRowSet} has been {@link RowSet#close() + * closed}. + */ +public class DataIndexer implements TrackingRowSet.Indexer { + + /** + * DataIndexer lookup method. Use this call when you will query from the returned DataIndexer but not add new ones. + * + * @param rowSet The row set to index + * @return The DataIndexer for {@code rowSet}, or null if none exists + */ + public static DataIndexer existingOf(@NotNull final TrackingRowSet rowSet) { + return rowSet.indexer(); + } + + /** + * DataIndexer factory method. Use this call when you will add indexes to the returned DataIndexer. + * + * @param rowSet The row set to index + * @return The DataIndexer for {@code rowSet}, created by this call if necessary + */ + public static DataIndexer of(@NotNull final TrackingRowSet rowSet) { + return rowSet.indexer((final TrackingRowSet indexedRowSet) -> new DataIndexer()); + } + + /** + * The root of a pseudo-trie of index caches. This is a complicated structure but has the strong benefit of allowing + * lookup without interfering with garbage collection of indexes or their indexed column sources. + */ + private final DataIndexCache rootCache = new DataIndexCache(); + + /** + * Cache priority assignments, which allows us to provide an arbitrary total ordering over all indexed column + * sources. This helps us to ensure that the "path" to any DataIndex is always the same, regardless of the order in + * which column sources are provided. + */ + private final Map, Integer> priorityAssignments = Collections.synchronizedMap(new WeakHashMap<>()); + + /** + * The next priority to assign to a column source. This is incremented each time a new column source is indexed. + */ + private int nextPriority = 0; + + /** + * Create a new DataIndexer. + */ + private DataIndexer() {} + + private List> pathFor(@NotNull final Collection> keyColumns) { + return keyColumns.stream() + .sorted(Comparator.comparingInt(this::priorityOf)) + .collect(Collectors.toList()); + } + + private int priorityOf(@NotNull final ColumnSource keyColumn) { + return priorityAssignments.computeIfAbsent(keyColumn, ignoredMissingKeyColumn -> nextPriority++); + } + + @NotNull + private static Collection> getColumnSources( + @NotNull final Table table, + @NotNull final String... keyColumnNames) { + return Arrays.stream(keyColumnNames) + .map(table::getColumnSource) + .collect(Collectors.toList()); + } + + /** + * Test whether {@code table} has a DataIndexer with a usable {@link DataIndex} for the given key columns. Note that + * a result from this method is a snapshot of current state, and does not guarantee anything about future calls to + * {@link #hasDataIndex}, {@link #getDataIndex}, or {@link #getOrCreateDataIndex(Table, String...)}. + * + * @param table The {@link Table} to check + * @param keyColumnNames The key column names to check + * @return Whether {@code table} has a DataIndexer with a {@link DataIndex} for the given key columns + */ + public static boolean hasDataIndex(@NotNull Table table, @NotNull final String... keyColumnNames) { + if (keyColumnNames.length == 0) { + return false; + } + table = table.coalesce(); + final DataIndexer indexer = DataIndexer.existingOf(table.getRowSet()); + if (indexer == null) { + return false; + } + return indexer.hasDataIndex(getColumnSources(table, keyColumnNames)); + } + + /** + * Test whether this DataIndexer has a usable {@link DataIndex} for the given key columns. Note that a result from + * this method is a snapshot of current state, and does not guarantee anything about future calls to + * {@link #hasDataIndex}, {@link #getDataIndex}, or {@link #getOrCreateDataIndex(Table, String...)}. + * + * @param keyColumns The key columns to check + * @return Whether this DataIndexer has a {@link DataIndex} for the given key columns + */ + public boolean hasDataIndex(@NotNull final ColumnSource... keyColumns) { + return hasDataIndex(Arrays.asList(keyColumns)); + } + + /** + * Test whether this DataIndexer has a usable {@link DataIndex} for the given key columns. Note that a result from + * this method is a snapshot of current state, and does not guarantee anything about future calls to + * {@link #hasDataIndex}, {@link #getDataIndex}, or {@link #getOrCreateDataIndex(Table, String...)}. + * + * @param keyColumns The key columns to check + * @return Whether this DataIndexer has a {@link DataIndex} for the given key columns + */ + public boolean hasDataIndex(@NotNull final Collection> keyColumns) { + if (keyColumns.isEmpty()) { + return false; + } + return rootCache.contains(pathFor(keyColumns)); + } + + /** + * If {@code table} has a DataIndexer, return a {@link DataIndex} for the given key columns, or {@code null} if no + * such index exists, if the cached index is invalid, or if the {@link DataIndex#isRefreshing() refreshing} cached + * index is no longer live. + * + * @param table The {@link Table} to check + * @param keyColumnNames The key column for which to retrieve a DataIndex + * @return The {@link DataIndex}, or {@code null} if one does not exist + */ + public static DataIndex getDataIndex(@NotNull Table table, final String... keyColumnNames) { + if (keyColumnNames.length == 0) { + return null; + } + table = table.coalesce(); + final DataIndexer indexer = DataIndexer.existingOf(table.getRowSet()); + if (indexer == null) { + return null; + } + return indexer.getDataIndex(getColumnSources(table, keyColumnNames)); + } + + /** + * Return a {@link DataIndex} for the given key columns, or {@code null} if no such index exists, if the cached + * index is invalid, or if the {@link DataIndex#isRefreshing() refreshing} cached index is no longer live. + * + * @param keyColumns The {@link ColumnSource column sources} for which to retrieve a {@link DataIndex} + * @return The {@link DataIndex}, or {@code null} if one does not exist + */ + public DataIndex getDataIndex(@NotNull final ColumnSource... keyColumns) { + return getDataIndex(Arrays.asList(keyColumns)); + } + + /** + * Return a {@link DataIndex} for the given key columns, or {@code null} if no such index exists, if the cached + * index is invalid, or if the {@link DataIndex#isRefreshing() refreshing} cached index is no longer live. + * + * @param keyColumns The {@link ColumnSource column sources} for which to retrieve a {@link DataIndex} + * @return The {@link DataIndex}, or {@code null} if one does not exist + */ + public DataIndex getDataIndex(@NotNull final Collection> keyColumns) { + if (keyColumns.isEmpty()) { + return null; + } + return rootCache.get(pathFor(keyColumns)); + } + + /** + * Return a valid, live {@link DataIndex} for a strict subset of the given key columns, or {@code null} if no such + * index exists. Will choose the DataIndex that results in the largest index table, following the assumption that + * the largest index table will divide the source table into the most specific partitions. + * + * @param table The indexed {@link Table} + * @param keyColumnNames The key column names to include + * + * @return The optimal partial {@link DataIndex}, or {@code null} if no such index exists + */ + @Nullable + public static DataIndex getOptimalPartialIndex(Table table, final String... keyColumnNames) { + if (keyColumnNames.length == 0) { + return null; + } + if (table.isRefreshing()) { + table.getUpdateGraph().checkInitiateSerialTableOperation(); + } + table = table.coalesce(); + final DataIndexer indexer = DataIndexer.existingOf(table.getRowSet()); + if (indexer == null) { + return null; + } + final Set> keyColumns = Arrays.stream(keyColumnNames) + .map(table::getColumnSource) + .collect(Collectors.toCollection(LinkedHashSet::new)); + return LivenessScopeStack.computeEnclosed(() -> Sets.powerSet(keyColumns).stream() + .filter(subset -> !subset.isEmpty() && subset.size() < keyColumns.size()) + .map(indexer::getDataIndex) + .filter(Objects::nonNull) + .max(Comparator.comparingLong(dataIndex -> dataIndex.table().size())) + .orElse(null), + table::isRefreshing, DataIndex::isRefreshing); + } + + /** + * Create a {@link DataIndex} for {@code table} indexing {@code keyColumns}, if no valid, live data index already + * exists for these inputs. + * + * @param table The {@link Table} to index + * @param keyColumnNames The key column names to include + * @return The existing or newly created {@link DataIndex} + * @apiNote This method causes the returned {@link DataIndex} to be managed by the enclosing liveness manager. + */ + public static DataIndex getOrCreateDataIndex( + @NotNull final Table table, + @NotNull final String... keyColumnNames) { + if (keyColumnNames.length == 0) { + throw new IllegalArgumentException("Cannot create a DataIndex without any key columns"); + } + final QueryTable tableToUse = (QueryTable) table.coalesce(); + final DataIndexer dataIndexer = DataIndexer.of(tableToUse.getRowSet()); + return dataIndexer.rootCache.computeIfAbsent(dataIndexer.pathFor(getColumnSources(tableToUse, keyColumnNames)), + () -> new TableBackedDataIndex(tableToUse, keyColumnNames)); + } + + /** + * Add a {@link DataIndex} to this DataIndexer. + * + * @param dataIndex The {@link DataIndex} to add + * @throws IllegalStateException If a valid, live {@link DataIndex} already exists for the given key columns + */ + public void addDataIndex(@NotNull final DataIndex dataIndex) { + if (dataIndex.keyColumnNamesByIndexedColumn().isEmpty()) { + throw new IllegalArgumentException("DataIndex must have at least one key column"); + } + if (!rootCache.add(pathFor(dataIndex.keyColumnNamesByIndexedColumn().keySet()), dataIndex)) { + throw new IllegalStateException(String.format("Attempted to add a duplicate index %s for key columns %s", + dataIndex, dataIndex.keyColumnNamesByIndexedColumn().keySet())); + } + } + + /** + * @param ensureValidAndLive Whether to ensure that returned {@link DataIndex DataIndexes} are valid and live + * @return All the {@link DataIndex DataIndexes} in this DataIndexer + */ + public List dataIndexes(final boolean ensureValidAndLive) { + final List result = new ArrayList<>(); + rootCache.getAll(result, ensureValidAndLive); + return result; + } + + /** + * Test whether {@code dataIndex} is {@code null} or invalid for use. This test does not evaluate liveness. + * + * @param dataIndex The {@link DataIndex} to test + * @return Whether {@code dataIndex} is {@code null} or invalid for use + */ + private static boolean isInvalid(@Nullable final DataIndex dataIndex) { + if (dataIndex == null) { + return true; + } + if (dataIndex instanceof AbstractDataIndex && !((AbstractDataIndex) dataIndex).isValid()) { + return true; + } + return dataIndex.isRefreshing() && dataIndex.table().isFailed(); + } + + /** + * Test whether {@code dataIndex} is a non-{@code null} {@link DataIndex} that is currently valid and live for use. + * + * @param dataIndex The {@link DataIndex} to test + * @return Whether {@code dataIndex} is valid and live for use + */ + private static boolean isValidAndLive(@Nullable final DataIndex dataIndex) { + if (isInvalid(dataIndex)) { + return false; + } + boolean retained = false; + try { + return !dataIndex.isRefreshing() || (retained = dataIndex.tryRetainReference()); + } finally { + if (retained) { + dataIndex.dropReference(); + } + } + } + + /** + * Similar to {@link io.deephaven.engine.liveness.Liveness#verifyCachedObjectForReuse(Object)}, but for + * {@link DataIndex DataIndexes}. + * + * @param dataIndex The {@link DataIndex} to validate and manage if appropriate + * @return {@code dataIndex}, if it can be used and is now safely managed by the enclosing + * {@link io.deephaven.engine.liveness.LivenessManager} if {@link DataIndex#isRefreshing() refreshing}, else + * {@code null} + */ + private static DataIndex validateAndManageCachedDataIndex(@Nullable final DataIndex dataIndex) { + if (isInvalid(dataIndex)) { + return null; + } + if (!dataIndex.isRefreshing()) { + return dataIndex; + } + /* + * If we're looking up a data index from an update graph thread, this means we're almost certainly in a listener + * and instantiating a new table operation. This could be a listener created directly by a user, or a + * PartitionedTable transform (either from a PartitionedTable.Proxy or an explicit transform function). In any + * case, if the data index isn't already satisfied, we would not want the operation being instantiated to wait + * for it to become satisfied from an update graph thread. Consequently, it's best to pretend we don't have a + * valid, live data index in this case. If a user or engine developer really wants to use a data index from an + * update graph thread, they should have already looked it up and made their listener's Notification.canExecute + * dependent on the data index table's satisfaction. + */ + final UpdateGraph updateGraph = dataIndex.table().getUpdateGraph(); + if (updateGraph.currentThreadProcessesUpdates() + && !dataIndex.table().satisfied(updateGraph.clock().currentStep())) { + return null; + } + if (!LivenessScopeStack.peek().tryManage(dataIndex)) { + return null; + } + return dataIndex; + } + + /** + * Node structure for our multi-level cache of indexes. + */ + private static class DataIndexCache { + + private static final Map, DataIndexCache> EMPTY_DESCENDANT_CACHES = Map.of(); + @SuppressWarnings("rawtypes") + private static final AtomicReferenceFieldUpdater DESCENDANT_CACHES_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(DataIndexCache.class, Map.class, "descendantCaches"); + private static final Reference MISSING_INDEX_REFERENCE = new WeakReference<>(null); + + /** The sub-indexes below this level. */ + @SuppressWarnings("FieldMayBeFinal") + private volatile Map, DataIndexCache> descendantCaches = EMPTY_DESCENDANT_CACHES; + + /** A reference to the index at this level. Note that there will never be an index at the "root" level. */ + private volatile Reference dataIndexReference = MISSING_INDEX_REFERENCE; + + private DataIndexCache() {} + + private Map, DataIndexCache> ensureDescendantCaches() { + // noinspection unchecked + return FieldUtils.ensureField(this, DESCENDANT_CACHES_UPDATER, EMPTY_DESCENDANT_CACHES, + () -> Collections.synchronizedMap(new WeakHashMap<>())); + } + + private DataIndexCache ensureCache(@NotNull final ColumnSource keyColumn) { + return ensureDescendantCaches().computeIfAbsent(keyColumn, ignored -> new DataIndexCache()); + } + + /** + * Recursively traverse the cache rooted at this node, calling {@code destinationCacheVisitor} if and when the + * destination node is reached. + * + * @param path The column sources that define the path from the root to the destination + * @param nextPathIndex The column source whose node we're trying to visit next when calling traverse + * @param createMissingCaches Whether to create missing caches when traversing, or to return false if a cache is + * not found + * @param destinationCacheVisitor The visitor to call when the destination node is reached + * @return {@code false} if the destination node was not reached (only possible when + * {@code createMissingCaches==true}, else the result of {@code destinationCacheVisitor} + */ + private boolean traverse( + @NotNull List> path, + final int nextPathIndex, + final boolean createMissingCaches, + @NotNull final Predicate destinationCacheVisitor) { + Require.inRange(nextPathIndex, "nextPathIndex", path.size(), "path.size()"); + final ColumnSource nextColumnSource = path.get(nextPathIndex); + final boolean nextIsDestination = nextPathIndex == path.size() - 1; + DataIndexCache nextCache = descendantCaches.get(nextColumnSource); + if (nextCache == null) { + if (!createMissingCaches) { + return false; + } + nextCache = ensureCache(nextColumnSource); + } + if (nextIsDestination) { + return destinationCacheVisitor.test(nextCache); + } + return nextCache.traverse(path, nextPathIndex + 1, createMissingCaches, destinationCacheVisitor); + } + + /** + * @param keyColumns The key columns to check + * @return Whether this DataIndexCache contains a valid, live {@link DataIndex} for the given key columns + */ + private boolean contains(@NotNull final List> keyColumns) { + return traverse(keyColumns, 0, false, cache -> isValidAndLive(cache.dataIndexReference.get())); + } + + /** + * @param keyColumns The key columns to check + * @return This DataIndexCache's valid, live {@link DataIndex} for the given key columns if it exists, else + * {@code null} + * @apiNote This method causes the returned {@link DataIndex} to be managed by the enclosing liveness manager if + * it is {@link DataIndex#isRefreshing() refreshing} + */ + @Nullable + private DataIndex get(@NotNull final List> keyColumns) { + final MutableObject resultHolder = new MutableObject<>(); + traverse(keyColumns, 0, false, cache -> { + resultHolder.setValue(validateAndManageCachedDataIndex(cache.dataIndexReference.get())); + return true; + }); + return resultHolder.getValue(); + } + + /** + * Get all the {@link DataIndex DataIndexes} in this cache and its descendants. + * + * @param result The list to which to add the {@link DataIndex DataIndexes} + * @param ensureValidAndLive Whether to first ensure that each {@link DataIndex} is valid and live for use + */ + private void getAll(@NotNull final List result, final boolean ensureValidAndLive) { + final DataIndex localDataIndex = ensureValidAndLive + ? validateAndManageCachedDataIndex(dataIndexReference.get()) + : dataIndexReference.get(); + if (localDataIndex != null) { + result.add(localDataIndex); + } + descendantCaches.values().forEach(cache -> cache.getAll(result, ensureValidAndLive)); + } + + /** + * @param keyColumns The key columns + * @param dataIndex The {@link DataIndex} to add + * @return Whether the {@code dataIndex} was added. {@code false} means a valid, live {@link DataIndex} for + * {@code keyColumns} was already present. + * @apiNote No validation is done of the {@link DataIndex}. We intentionally defer until {@link #contains(List) + * contains}, {@link #get(List) get}, or {@link #computeIfAbsent(List, Supplier) computeIfAbsent} + * accesses it. + */ + private boolean add(@NotNull final List> keyColumns, @NotNull final DataIndex dataIndex) { + return traverse(keyColumns, 0, true, cache -> { + if (!isValidAndLive(cache.dataIndexReference.get())) { + // noinspection SynchronizationOnLocalVariableOrMethodParameter + synchronized (cache) { + if (!isValidAndLive(cache.dataIndexReference.get())) { + cache.dataIndexReference = new WeakReference<>(dataIndex); + return true; + } + } + } + return false; + }); + } + + /** + * @param keyColumns The key columns + * @param dataIndexFactory A {@link Supplier} for a new {@link DataIndex}, invoked if no valid, live + * {@link DataIndex} currently exists in the cache + * @return The currently cached {@link DataIndex} + * @apiNote If an existing {@link DataIndex} is returned, this method will test that it remains valid and will + * cause it to be managed by the enclosing liveness manager if it is {@link DataIndex#isRefreshing() + * refreshing}. If a new {@link DataIndex} is created, it is the caller's responsibility to ensure that + * it is valid and managed; this method defers validation as in {@link #add(List, DataIndex) add}. + */ + private DataIndex computeIfAbsent( + @NotNull final List> keyColumns, + @NotNull final Supplier dataIndexFactory) { + final MutableObject resultHolder = new MutableObject<>(); + traverse(keyColumns, 0, true, cache -> { + DataIndex dataIndex = validateAndManageCachedDataIndex(cache.dataIndexReference.get()); + if (dataIndex == null) { + // noinspection SynchronizationOnLocalVariableOrMethodParameter + synchronized (cache) { + dataIndex = validateAndManageCachedDataIndex(cache.dataIndexReference.get()); + if (dataIndex == null) { + // Don't manage the factory result with the enclosing liveness manager. + // It's the caller's responsibility to make sure we produce a valid data index that is + // managed by the appropriate scope for the caller's own use. Further validation is deferred + // as in add. + dataIndex = dataIndexFactory.get(); + cache.dataIndexReference = new WeakReference<>(dataIndex); + } + } + } + resultHolder.setValue(dataIndex); + return true; + }); + return resultHolder.getValue(); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/RowSetIndexer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/RowSetIndexer.java deleted file mode 100644 index d35795d93ed..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/indexer/RowSetIndexer.java +++ /dev/null @@ -1,642 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.engine.table.impl.indexer; - -import io.deephaven.base.verify.Require; -import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.TupleSource; -import io.deephaven.engine.table.impl.TupleSourceFactory; -import io.deephaven.tuple.EmptyTuple; -import org.jetbrains.annotations.NotNull; - -import java.util.*; -import java.util.function.BiConsumer; -import java.util.function.UnaryOperator; -import java.util.stream.Collectors; - -/** - * Indexer that provides "grouping" information based on key values extracted from a {@link TupleSource}, linked to a - * {@link TrackingRowSet}. - * - * @apiNote RowSetIndexers should not be used after the host {@link TrackingRowSet} has been {@link RowSet#close() - * closed}. - */ -public class RowSetIndexer implements TrackingRowSet.Indexer { - - public static RowSetIndexer of(TrackingRowSet rowSet) { - return rowSet.indexer(RowSetIndexer::new); - } - - private final TrackingRowSet rowSet; - - /** - * These mappings last forever, and only include column sources that are immutable. Whenever the indexed RowSet is - * changed, we must clear the mappings. - */ - private WeakHashMap, MappingInfo> mappings = null; - - /** - * These mappings do not survive past a single LogicalClock tick or any indexed RowSet changes. - */ - private WeakHashMap, MappingInfo> ephemeralMappings = null; - - /** - * These prev mappings do not survive past a single LogicalClock tick or any indexed RowSet changes. - */ - private WeakHashMap, MappingInfo> ephemeralPrevMappings = null; - - private static class MappingInfo { - - private final TupleSource tupleSource; - private final Map mapping; - private final long creationTick; - - private MappingInfo(final TupleSource tupleSource, final Map mapping, final long creationTick) { - this.tupleSource = tupleSource; - this.mapping = mapping; - this.creationTick = creationTick; - } - } - - public RowSetIndexer(@NotNull final TrackingRowSet rowSet) { - this.rowSet = rowSet; - } - - @Override - public void rowSetChanged() { - if (mappings != null) { - mappings.clear(); - } - if (ephemeralMappings != null) { - ephemeralMappings.clear(); - } - if (ephemeralPrevMappings != null) { - ephemeralPrevMappings.clear(); - } - } - - public boolean hasGrouping(final ColumnSource... keyColumns) { - if (keyColumns.length == 0) { - return true; - } - final List sourcesKey = Arrays.asList(keyColumns); - final Map groupingCandidate = lookupMapping(sourcesKey); - return groupingCandidate != null || keyColumns.length == 1 && keyColumns[0].getGroupToRange() != null; - } - - public Map getGrouping(final TupleSource tupleSource) { - // noinspection unchecked - final List sourcesKey = tupleSource.getColumnSources(); - final Map lookupResult = lookupMapping(sourcesKey); - if (lookupResult != null) { - return lookupResult; - } - - final Map result = new LinkedHashMap<>(); - final BiConsumer resultCollector = result::put; - collectGrouping(rowSet, rowSet::intersect, mappings, ephemeralMappings, resultCollector, tupleSource, - sourcesKey); - - if (areColumnsImmutable(sourcesKey)) { - if (mappings == null) { - mappings = new WeakHashMap<>(); - } - mappings.put(sourcesKey, new MappingInfo(tupleSource, result, 0)); - } else { - if (ephemeralMappings == null) { - ephemeralMappings = new WeakHashMap<>(); - } - ephemeralMappings.put(sourcesKey, new MappingInfo(tupleSource, result, - ExecutionContext.getContext().getUpdateGraph().clock().currentStep())); - } - - return result; - } - - public Map getPrevGrouping(final TupleSource tupleSource) { - // noinspection unchecked - final List sourcesKey = tupleSource.getColumnSources(); - - Map result = lookupPrevMapping(sourcesKey); - if (result != null) { - return result; - } - result = new LinkedHashMap<>(); - if (sourcesKey.isEmpty()) { - result.put(EmptyTuple.INSTANCE, rowSet.copy()); - } else { - final Map resultBuilder = new LinkedHashMap<>(); - for (final RowSet.Iterator iterator = rowSet.iterator(); iterator.hasNext();) { - final long next = iterator.nextLong(); - final Object key = tupleSource.createPreviousTuple(next); - resultBuilder.computeIfAbsent(key, k -> RowSetFactory.builderSequential()) - .appendKey(next); - } - result = new LinkedHashMap<>(); - for (final Map.Entry objectRowSetBuilderEntry : resultBuilder.entrySet()) { - result.put(objectRowSetBuilderEntry.getKey(), objectRowSetBuilderEntry.getValue().build()); - } - } - if (areColumnsImmutable(sourcesKey)) { - if (mappings == null) { - mappings = new WeakHashMap<>(); - } - mappings.put(sourcesKey, new MappingInfo(tupleSource, result, 0)); - } else { - if (ephemeralPrevMappings == null) { - ephemeralPrevMappings = new WeakHashMap<>(); - } - ephemeralPrevMappings.put(sourcesKey, - new MappingInfo(tupleSource, result, - ExecutionContext.getContext().getUpdateGraph().clock().currentStep())); - } - return result; - } - - public void copyImmutableGroupings(final TupleSource source, final TupleSource dest) { - // noinspection unchecked - final List sourcesKey = source.getColumnSources(); - // noinspection unchecked - final List destKey = dest.getColumnSources(); - - // copy immutable groupings - if (mappings != null) { - final MappingInfo toCopy; - if ((toCopy = mappings.get(sourcesKey)) != null) { - mappings.put(destKey, toCopy); - } - } - } - - /** - * Return a grouping that contains row keys that match the values in {@code keys}. - * - * @param keys A set of values that {@code TupleSource} should match. For a single {@link ColumnSource}, the values - * within the set are the values that we would like to find. For compound {@link TupleSource} instances, the - * values are SmartKeys. - * @param tupleSource The tuple factory for singular or compound keys - * @return A map from keys to {@link RowSet}, for each of the {@code keys} present in the indexed RowSet's view of - * {@code tupleSource} - */ - public Map getGroupingForKeySet(final Set keys, final TupleSource tupleSource) { - final Map result = new LinkedHashMap<>(); - collectGroupingForKeySet(keys, tupleSource, result::put); - return result; - } - - /** - * Return a subset that contains row keys that match the values in {@code keys}. - * - * @param keys A set of values that {@code TupleSource} should match. For a single {@link ColumnSource}, the values - * within the set are the values that we would like to find. For compound {@link TupleSource} instances, the - * values are SmartKeys. - * @param tupleSource The tuple factory for singular or compound keys - * @return A {@link WritableRowSet} with all row keys from the indexed RowSet whose value in {@code tupleSource} was - * present in {@code keys} - */ - public RowSet getSubSetForKeySet(final Set keys, final TupleSource tupleSource) { - final RowSetBuilderRandom rowSetBuilder = RowSetFactory.builderRandom(); - final BiConsumer resultCollector = - (key, rowSet) -> rowSetBuilder.addRowSet(rowSet); - collectGroupingForKeySet(keys, tupleSource, resultCollector); - return rowSetBuilder.build(); - } - - private static Map getGrouping( - final RowSet rowSetRowSet, - final UnaryOperator rowSetOp, - final WeakHashMap, MappingInfo> mappings, - final WeakHashMap, MappingInfo> ephemeralMappings, - final TupleSource tupleSource) { - // noinspection unchecked - final List sourcesKey = tupleSource.getColumnSources(); - final Map lookupResult = lookupMapping(mappings, ephemeralMappings, sourcesKey); - if (lookupResult != null) { - return lookupResult; - } - - final Map result = new LinkedHashMap<>(); - final BiConsumer resultCollector = result::put; - collectGrouping(rowSetRowSet, rowSetOp, mappings, ephemeralMappings, resultCollector, tupleSource, sourcesKey); - return result; - } - - private Map lookupMapping(List columnSourceKey) { - return lookupMapping(mappings, ephemeralMappings, columnSourceKey); - } - - private static Map lookupMapping( - WeakHashMap, MappingInfo> mappings, - WeakHashMap, MappingInfo> ephemeralMappings, - List columnSourceKey) { - final Map immutableMapping = lookupImmutableMapping(mappings, columnSourceKey); - if (immutableMapping != null) { - return immutableMapping; - } - return lookupEphemeralMapping(columnSourceKey, ephemeralMappings); - } - - private static void collectGrouping( - final RowSet rowSetRowSet, - final UnaryOperator rowSetOp, - final WeakHashMap, MappingInfo> mappings, - final WeakHashMap, MappingInfo> ephemeralMappings, - final BiConsumer resultCollector, - final TupleSource tupleSource, - final List keyColumns) { - if (keyColumns.isEmpty()) { - resultCollector.accept(EmptyTuple.INSTANCE, rowSetRowSet.copy()); - } else if (keyColumns.size() == 1 && keyColumns.get(0).getGroupToRange() != null) { - @SuppressWarnings("unchecked") - final Map sourceGrouping = keyColumns.get(0).getGroupToRange(); - for (final Map.Entry objectRowSetEntry : sourceGrouping.entrySet()) { - final RowSet resultRowSet = rowSetOp.apply(objectRowSetEntry.getValue()); - if (resultRowSet.size() > 0) { - resultCollector.accept(objectRowSetEntry.getKey(), resultRowSet); - } - } - } else { - final long columnsWithGrouping = keyColumns.stream().filter(cs -> cs.getGroupToRange() != null).count(); - final boolean canUseAllConstituents = columnsWithGrouping == keyColumns.size(); - final boolean canUseAnyConstituents = columnsWithGrouping > 0; - - if (canUseAllConstituents) { - // we can generate a grouping using just the pre-existing groupings - generateGrouping(rowSetOp, resultCollector, tupleSource, keyColumns, 0, new Object[keyColumns.size()], - null); - } else if (canUseAnyConstituents) { - generatePartialGrouping(rowSetRowSet, rowSetOp, mappings, ephemeralMappings, resultCollector, - tupleSource, - keyColumns); - } else { - final Map resultBuilder = new LinkedHashMap<>(); - for (final RowSet.Iterator iterator = rowSetRowSet.iterator(); iterator.hasNext();) { - final long next = iterator.nextLong(); - final Object key = tupleSource.createTuple(next); - resultBuilder.computeIfAbsent(key, k -> RowSetFactory.builderSequential()) - .appendKey(next); - } - resultBuilder.forEach((k, v) -> resultCollector.accept(k, v.build())); - } - } - } - - private static void generatePartialGrouping( - final RowSet rowSetRowSet, - final UnaryOperator rowSetOp, - final WeakHashMap, MappingInfo> mappings, - final WeakHashMap, MappingInfo> ephemeralMappings, - final BiConsumer resultCollector, - final TupleSource tupleSource, - final List keyColumns) { - // we can generate the grouping partially from our constituents - final ColumnSource[] groupedKeyColumns = - keyColumns.stream().filter(cs -> cs.getGroupToRange() != null).toArray(ColumnSource[]::new); - final ColumnSource[] notGroupedKeyColumns = - keyColumns.stream().filter(cs -> cs.getGroupToRange() == null).toArray(ColumnSource[]::new); - - final TupleSource groupedTupleSource = TupleSourceFactory.makeTupleSource(groupedKeyColumns); - final Map groupedColumnsGrouping = - getGrouping(rowSetRowSet, rowSetOp, mappings, ephemeralMappings, groupedTupleSource); - generatePartialGroupingSecondHalf(groupedKeyColumns, notGroupedKeyColumns, groupedTupleSource, - groupedColumnsGrouping, - resultCollector, tupleSource, keyColumns); - } - - private static void generatePartialGroupingSecondHalf( - final ColumnSource[] groupedKeyColumns, final ColumnSource[] notGroupedKeyColumns, - final TupleSource groupedTupleSource, final Map groupedColumnsGrouping, - final BiConsumer resultCollector, final TupleSource tupleSource, - final List keyColumns) { - final Map resultBuilder = new LinkedHashMap<>(); - - final int[] groupedKeysIndices = new int[groupedKeyColumns.length]; - final int[] notGroupedKeysIndices = new int[notGroupedKeyColumns.length]; - int jj = 0, kk = 0; - for (int ii = 0; ii < keyColumns.size(); ++ii) { - if (keyColumns.get(ii).getGroupToRange() != null) { - groupedKeysIndices[jj++] = ii; - } else { - notGroupedKeysIndices[kk++] = ii; - } - } - - for (final Map.Entry entry : groupedColumnsGrouping.entrySet()) { - final Object[] partialKeyValues = new Object[keyColumns.size()]; - if (groupedKeyColumns.length == 1) { - partialKeyValues[groupedKeysIndices[0]] = entry.getKey(); - } else { - final Object groupedTuple = entry.getKey(); - for (int ii = 0; ii < groupedKeysIndices.length; ++ii) { - // noinspection unchecked - partialKeyValues[groupedKeysIndices[ii]] = - groupedTupleSource.exportElementReinterpreted(groupedTuple, ii); - } - } - - final RowSet resultRowSet = entry.getValue(); - for (final RowSet.Iterator iterator = resultRowSet.iterator(); iterator.hasNext();) { - final long next = iterator.nextLong(); - - for (int ii = 0; ii < notGroupedKeysIndices.length; ++ii) { - partialKeyValues[notGroupedKeysIndices[ii]] = notGroupedKeyColumns[ii].get(next); - } - - resultBuilder.computeIfAbsent(tupleSource.createTupleFromReinterpretedValues(partialKeyValues), - k -> RowSetFactory.builderSequential()).appendKey(next); - } - } - - resultBuilder.forEach((k, v) -> resultCollector.accept(k, v.build())); - } - - private void generatePartialGroupingForKeySet(final BiConsumer resultCollector, - final TupleSource tupleSource, - final List keyColumns, Set keys) { - // we can generate the grouping partially from our constituents - final ColumnSource[] groupedKeyColumns = - keyColumns.stream().filter(cs -> cs.getGroupToRange() != null).toArray(ColumnSource[]::new); - final ColumnSource[] notGroupedKeyColumns = - keyColumns.stream().filter(cs -> cs.getGroupToRange() == null).toArray(ColumnSource[]::new); - - Require.gtZero(groupedKeyColumns.length, "groupedKeyColumns.length"); - Require.gtZero(notGroupedKeyColumns.length, "notGroupedKeyColumns.length"); - - final TupleSource groupedTupleSource = TupleSourceFactory.makeTupleSource(groupedKeyColumns); - - final Map resultBuilder = new LinkedHashMap<>(); - - final int[] groupedKeysIndices = new int[groupedKeyColumns.length]; - final int[] notGroupedKeysIndices = new int[notGroupedKeyColumns.length]; - int jj = 0, kk = 0; - for (int ii = 0; ii < keyColumns.size(); ++ii) { - if (keyColumns.get(ii).getGroupToRange() != null) { - groupedKeysIndices[jj++] = ii; - } else { - notGroupedKeysIndices[kk++] = ii; - } - } - - final Set groupPruningSet = new HashSet<>(); - if (groupedKeysIndices.length == 1) { - // noinspection unchecked - keys.forEach(x -> groupPruningSet.add(tupleSource.exportElementReinterpreted(x, groupedKeysIndices[0]))); - } else { - final Object[] groupingKeyValues = new Object[groupedKeysIndices.length]; - keys.forEach(x -> { - for (int ii = 0; ii < groupingKeyValues.length; ++ii) { - // noinspection unchecked - groupingKeyValues[ii] = tupleSource.exportElementReinterpreted(x, groupedKeysIndices[ii]); - } - groupPruningSet.add(groupedTupleSource.createTupleFromReinterpretedValues(groupingKeyValues)); - }); - } - - final Map groupedColumnsGrouping = - getGroupingForKeySet(groupPruningSet, groupedTupleSource); - - final Object[] lookupKeyValues = new Object[keyColumns.size()]; - - for (final Map.Entry entry : groupedColumnsGrouping.entrySet()) { - final RowSet resultRowSet = entry.getValue().intersect(rowSet); - if (resultRowSet.isEmpty()) { - continue; - } - - if (groupedKeyColumns.length == 1) { - lookupKeyValues[groupedKeysIndices[0]] = entry.getKey(); - } else { - for (int ii = 0; ii < groupedKeysIndices.length; ++ii) { - // noinspection unchecked - lookupKeyValues[groupedKeysIndices[ii]] = - groupedTupleSource.exportElementReinterpreted(entry.getKey(), ii); - } - } - - for (final RowSet.Iterator iterator = resultRowSet.iterator(); iterator.hasNext();) { - final long next = iterator.nextLong(); - - for (int ii = 0; ii < notGroupedKeysIndices.length; ++ii) { - lookupKeyValues[notGroupedKeysIndices[ii]] = notGroupedKeyColumns[ii].get(next); - } - - final Object key = tupleSource.createTupleFromReinterpretedValues(lookupKeyValues); - if (!keys.contains(key)) { - continue; - } - - final RowSetBuilderSequential rowSetForKey = - resultBuilder.computeIfAbsent(key, k -> RowSetFactory.builderSequential()); - rowSetForKey.appendKey(next); - } - } - - resultBuilder.forEach((k, v) -> resultCollector.accept(k, v.build())); - } - - private void collectGroupingForKeySet(final Set keys, final TupleSource tupleSource, - final BiConsumer resultCollector) { - // noinspection unchecked - final List keyColumns = tupleSource.getColumnSources(); - if (keyColumns.isEmpty()) { - resultCollector.accept(EmptyTuple.INSTANCE, rowSet.copy()); - } else if (keyColumns.size() == 1 && keyColumns.get(0).getGroupToRange() != null) { - @SuppressWarnings("unchecked") - final Map sourceGrouping = keyColumns.get(0).getGroupToRange(); - sourceGrouping.entrySet().stream().filter(objectRowSetEntry -> keys.contains(objectRowSetEntry.getKey())) - .forEach(objectRowSetEntry -> { - final RowSet resultRowSet = objectRowSetEntry.getValue().intersect(rowSet); - if (resultRowSet.size() > 0) { - resultCollector.accept(objectRowSetEntry.getKey(), resultRowSet); - } - }); - } else { - final long columnsWithGrouping = keyColumns.stream().filter(cs -> cs.getGroupToRange() != null).count(); - final boolean canUseAllConstituents = columnsWithGrouping == keyColumns.size(); - final boolean canUseAnyConstituents = columnsWithGrouping > 0; - - if (canUseAllConstituents) { - generateGrouping(resultCollector, tupleSource, keyColumns, 0, new Object[keyColumns.size()], null, - keys); - } else if (canUseAnyConstituents) { - generatePartialGroupingForKeySet(resultCollector, tupleSource, keyColumns, keys); - } else { - final Map resultBuilder = new LinkedHashMap<>(); - for (final RowSet.Iterator iterator = rowSet.iterator(); iterator.hasNext();) { - final long next = iterator.nextLong(); - final Object key = tupleSource.createTuple(next); - if (keys.contains(key)) { - resultBuilder.computeIfAbsent(key, k -> RowSetFactory.builderSequential()) - .appendKey(next); - } - } - for (final Map.Entry objectRowSetBuilderEntry : resultBuilder - .entrySet()) { - resultCollector.accept(objectRowSetBuilderEntry.getKey(), - objectRowSetBuilderEntry.getValue().build()); - } - } - } - } - - private static void generateGrouping( - final UnaryOperator rowSetOp, - final BiConsumer resultCollector, - final TupleSource tupleSource, - final List keyColumns, - final int position, - final Object[] partialValues, - final RowSet partiallyIntersectedRowSet) { - for (final Object objectEntry : keyColumns.get(position).getGroupToRange().entrySet()) { - // noinspection unchecked - final Map.Entry entry = - (Map.Entry) objectEntry; - partialValues[position] = entry.getKey(); - final RowSet subRowSet; - if (position == 0) { - subRowSet = rowSetOp.apply(entry.getValue()); - } else { - subRowSet = partiallyIntersectedRowSet.intersect(entry.getValue()); - } - if (subRowSet.isNonempty()) { - if (position == keyColumns.size() - 1) { - // we're at the very last bit, so we should start shoving our tuples into the result map - resultCollector.accept(tupleSource.createTupleFromReinterpretedValues(partialValues), subRowSet); - } else { - generateGrouping(rowSetOp, resultCollector, tupleSource, keyColumns, position + 1, partialValues, - subRowSet); - } - } - } - } - - private void generateGrouping(final BiConsumer resultCollector, final TupleSource tupleSource, - final List keyColumns, final int position, final Object[] partialValues, - final RowSet partiallyIntersectedRowSet, - final Set keyRestriction) { - final boolean finalPosition = position == keyColumns.size() - 1; - - final List subSources = keyColumns.subList(0, position + 1); - final TupleSource subTupleSource = - TupleSourceFactory.makeTupleSource(subSources.toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY)); - - final Set pruningSet; - if (finalPosition) { - pruningSet = keyRestriction; - } else if (position == 0) { - // noinspection unchecked - pruningSet = keyRestriction.stream().map(x -> tupleSource.exportElementReinterpreted(x, 0)) - .collect(Collectors.toCollection(HashSet::new)); - } else { - pruningSet = new HashSet<>(); - final Object[] partialKey = new Object[position + 1]; - keyRestriction.forEach(key -> { - for (int ii = 0; ii < partialKey.length; ++ii) { - // noinspection unchecked - partialKey[ii] = tupleSource.exportElementReinterpreted(key, ii); - } - pruningSet.add(subTupleSource.createTupleFromReinterpretedValues(partialKey)); - }); - } - - // noinspection unchecked - final Map groupToRange = - (Map) keyColumns.get(position).getGroupToRange(); - final Object[] pruningKey = Arrays.copyOf(partialValues, position + 1); - for (final Map.Entry entry : groupToRange.entrySet()) { - pruningKey[position] = partialValues[position] = entry.getKey(); - - final Object tuple; - if (finalPosition) { - tuple = tupleSource.createTupleFromReinterpretedValues(partialValues); - if (!keyRestriction.contains(tuple)) { - continue; - } - } else if (position == 0) { - if (!pruningSet.contains(entry.getKey())) { - continue; - } - - tuple = null; - } else { - if (!pruningSet.contains(subTupleSource.createTupleFromReinterpretedValues(pruningKey))) { - continue; - } - - tuple = null; - } - - final RowSet subRowSet; - if (position == 0) { - subRowSet = rowSet.intersect(entry.getValue()); - } else { - subRowSet = partiallyIntersectedRowSet.intersect(entry.getValue()); - } - - if (subRowSet.isNonempty()) { - if (finalPosition) { - // we're at the very last bit, so we should start shoving our smart keys into the result map - resultCollector.accept(tuple, subRowSet); - } else { - generateGrouping(resultCollector, tupleSource, keyColumns, position + 1, partialValues, subRowSet, - keyRestriction); - } - } - } - } - - private Map lookupPrevMapping(List columnSourceKey) { - return lookupPrevMapping(mappings, ephemeralPrevMappings, columnSourceKey); - } - - private static Map lookupPrevMapping( - WeakHashMap, MappingInfo> mappings, - WeakHashMap, MappingInfo> ephemeralPrevMappings, - List columnSourceKey) { - final Map immutableMapping = lookupImmutableMapping(mappings, columnSourceKey); - if (immutableMapping != null) { - return immutableMapping; - } - return lookupEphemeralMapping(columnSourceKey, ephemeralPrevMappings); - } - - private static Map lookupImmutableMapping( - WeakHashMap, MappingInfo> mappings, - List columnSourceKey) { - if (mappings == null) { - return null; - } - final MappingInfo mappingInfo = mappings.get(columnSourceKey); - if (mappingInfo == null) { - return null; - } - return mappingInfo.mapping; - } - - private static Map lookupEphemeralMapping(List columnSourceKey, - WeakHashMap, MappingInfo> groupingMap) { - if (groupingMap == null) { - return null; - } - - final MappingInfo resultInfo = groupingMap.get(columnSourceKey); - if (resultInfo == null) { - return null; - } - - if (resultInfo.creationTick != ExecutionContext.getContext().getUpdateGraph().clock().currentStep()) { - groupingMap.remove(columnSourceKey); - return null; - } - - return resultInfo.mapping; - } - - private boolean areColumnsImmutable(final List sourcesKey) { - return sourcesKey.stream().allMatch(ColumnSource::isImmutable); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/ColumnLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/ColumnLocation.java index 63b2a177dd5..686aaa89ef0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/ColumnLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/ColumnLocation.java @@ -45,15 +45,26 @@ public interface ColumnLocation extends StringUtils.StringKeyedObject, NamedImpl *

* Get the metadata object stored with this column, or null if no such data exists. *

- * This is typically a value to range map (grouping metadata). The value to range map, if non-null, is a map from - * unique (boxed) column values for this location to the associated ranges in which they occur. Ranges are either - * 2-element int[]s, or 2-element long[]s. + * Historically, this was typically a value to range map (grouping metadata). The value to range map, if non-null, + * is a map from unique (boxed) column values for this location to the associated ranges in which they occur. Ranges + * are either 2-element int[]s, or 2-element long[]s. * * @return The metadata stored with this column, or null if no such data exists */ + @SuppressWarnings("unused") @Nullable METADATA_TYPE getMetadata(@NotNull ColumnDefinition columnDefinition); + /** + * Get this column location cast to the specified type + * + * @return {@code this}, with the appropriate cast applied + */ + default CL cast() { + // noinspection unchecked + return (CL) this; + } + // ------------------------------------------------------------------------------------------------------------------ // ColumnRegion Factories // ------------------------------------------------------------------------------------------------------------------ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/GroupingProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/GroupingProvider.java deleted file mode 100644 index a37361c509b..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/GroupingProvider.java +++ /dev/null @@ -1,52 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.engine.table.impl.locations; - -import io.deephaven.base.Pair; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.impl.locations.impl.ParallelDeferredGroupingProvider; -import io.deephaven.engine.table.impl.sources.DeferredGroupingColumnSource; -import io.deephaven.engine.rowset.RowSet; -import org.jetbrains.annotations.NotNull; - -import java.util.Map; - -/** - * Interface used by {@link DeferredGroupingColumnSource} to compute groupings. - */ -public interface GroupingProvider { - - /** - * Make a new {@link GroupingProvider} for the specified {@link ColumnDefinition} and current global configuration. - * - * @param columnDefinition The column definition - * @return A new {@link GroupingProvider} - */ - @NotNull - static GroupingProvider makeGroupingProvider( - @NotNull final ColumnDefinition columnDefinition) { - return new ParallelDeferredGroupingProvider<>(columnDefinition); - } - - /** - * Returns a grouping structure, possibly constructed on-demand. - * - * @return a Map from grouping keys to Indices, or null if the group could not be constructed - */ - Map getGroupToRange(); - - /** - * Returns a grouping structure, possibly constructed on-demand; the grouping is only required to include groupings - * for values that exist within the hint RowSet; but it may include more. The hint allows the underlying - * implementation to optionally optimize out groupings that do not overlap hint. - *

- * The return value is a pair, containing a "complete" indicator. If the complete indicator is true, then the caller - * may safely cache the resultant Map. - * - * @param hint required indices within the resultant Map - * @return a Pair containing a Map from grouping keys to Indices, which includes at least the hint indices; and a - * Boolean which indicates that the grouping is complete - */ - Pair, Boolean> getGroupToRange(RowSet hint); -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/KeyRangeGroupingProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/KeyRangeGroupingProvider.java deleted file mode 100644 index 7e47fc4228d..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/KeyRangeGroupingProvider.java +++ /dev/null @@ -1,23 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.engine.table.impl.locations; - -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.TrackingWritableRowSet; -import org.jetbrains.annotations.NotNull; - -/** - * Interface for {@link GroupingProvider}s that operate on metadata derived from a {@link ColumnLocation} for a given - * {@link RowSet} key range. - */ -public interface KeyRangeGroupingProvider extends GroupingProvider { - - /** - * Add a column location for consideration when constructing groupings. - * - * @param columnLocation The column location to add - * @param locationRowSetInTable The location's RowSet in the table - */ - void addSource(@NotNull ColumnLocation columnLocation, @NotNull RowSet locationRowSetInTable); -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java index 0771071f05c..9d3eb5b55e7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java @@ -3,12 +3,18 @@ // package io.deephaven.engine.table.impl.locations; +import io.deephaven.api.SortColumn; import io.deephaven.base.log.LogOutput; import io.deephaven.base.log.LogOutputAppendable; +import io.deephaven.engine.table.BasicDataIndex; +import io.deephaven.engine.table.Table; import io.deephaven.io.log.impl.LogOutputStringImpl; import io.deephaven.util.annotations.FinalDefault; import io.deephaven.util.type.NamedImplementation; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.List; /** * Building block for Deephaven "source" tables, with helper methods for discovering locations and their sizes. A @@ -77,6 +83,47 @@ interface Listener extends BasicTableDataListener { */ void refresh(); + /** + * Get an ordered list of columns this location is sorted by. + * + * @return A non-null ordered list of {@link SortColumn SortColumns} + */ + @NotNull + List getSortedColumns(); + + /** + * Get a list of the columns by which this location is indexed + * + * @return A non-null list of {@code String[]} arrays containing the key column names for each existing index + */ + @NotNull + List getDataIndexColumns(); + + /** + * Check if this TableLocation has a data index for the specified columns. + * + * @param columns The set of columns to check for + * @return Whether the TableLocation has an index for the specified columns + * + * @apiNote Implementations must guarantee that the result of this method remains constant over the life of an + * instance, and is consistent with the result of {@link #getDataIndex(String...)}. + */ + boolean hasDataIndex(@NotNull String... columns); + + /** + * Get the data index table for the specified set of columns. Note that the order of columns does not matter here. + * + * @param columns The key columns for the index + * @return The index table or null if one does not exist + * @apiNote If this TableLocation is not static, the returned table must be {@link Table#isRefreshing() refreshing}, + * and should be updated to reflect changes in a manner that is consistent with the results provided to a + * subscriber. + * @implNote Implementations should attempt to provide a lazily-coalesced result wherever possible, allowing work to + * be deferred or parallelized. + */ + @Nullable + BasicDataIndex getDataIndex(@NotNull String... columns); + /** * @param name The column name * @return The ColumnLocation for the defined column under this table location diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index be4949f05bc..2bc11d60976 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java @@ -4,11 +4,22 @@ package io.deephaven.engine.table.impl.locations.impl; import io.deephaven.base.verify.Require; +import io.deephaven.datastructures.util.CollectionUtil; +import io.deephaven.engine.table.BasicDataIndex; +import io.deephaven.engine.table.impl.util.FieldUtils; import io.deephaven.engine.util.string.StringUtils; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.rowset.RowSet; import io.deephaven.hash.KeyedObjectHashMap; +import io.deephaven.hash.KeyedObjectKey; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.lang.ref.SoftReference; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; /** * Partial TableLocation implementation for use by TableDataService implementations. @@ -24,6 +35,18 @@ public abstract class AbstractTableLocation private final KeyedObjectHashMap columnLocations = new KeyedObjectHashMap<>(StringUtils.charSequenceKey()); + @SuppressWarnings("rawtypes") + private static final AtomicReferenceFieldUpdater CACHED_DATA_INDEXES_UPDATER = + AtomicReferenceFieldUpdater.newUpdater( + AbstractTableLocation.class, KeyedObjectHashMap.class, "cachedDataIndexes"); + private static final SoftReference NO_INDEX_SENTINEL = new SoftReference<>(null); + private static final KeyedObjectKey, CachedDataIndex> CACHED_DATA_INDEX_KEY = + new KeyedObjectKey.BasicAdapter<>(CachedDataIndex::getColumns); + + /** A map of data index columns to cache nodes for materialized data indexes for this location. */ + @SuppressWarnings("unused") + private volatile KeyedObjectHashMap, CachedDataIndex> cachedDataIndexes; + /** * @param tableKey Table key for the table this location belongs to * @param tableLocationKey Table location key that identifies this location @@ -138,4 +161,69 @@ public final ColumnLocation getColumnLocation(@NotNull final CharSequence name) protected final void clearColumnLocations() { columnLocations.clear(); } + + /** + * Caching structure for loaded data indexes. + */ + private class CachedDataIndex { + + private final List columns; + + private volatile SoftReference indexReference; + + private CachedDataIndex(@NotNull final List columns) { + this.columns = columns; + } + + private List getColumns() { + return columns; + } + + private BasicDataIndex getDataIndex() { + SoftReference localReference = indexReference; + BasicDataIndex localIndex; + if (localReference == NO_INDEX_SENTINEL) { + return null; + } + if (localReference != null && (localIndex = localReference.get()) != null) { + return localIndex; + } + synchronized (this) { + localReference = indexReference; + if (localReference == NO_INDEX_SENTINEL) { + return null; + } + if (localReference != null && (localIndex = localReference.get()) != null) { + return localIndex; + } + localIndex = loadDataIndex(columns.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY)); + indexReference = localIndex == null ? NO_INDEX_SENTINEL : new SoftReference<>(localIndex); + return localIndex; + } + } + } + + @Override + @Nullable + public final BasicDataIndex getDataIndex(@NotNull final String... columns) { + final List columnNames = new ArrayList<>(columns.length); + Collections.addAll(columnNames, columns); + columnNames.sort(String::compareTo); + + // noinspection unchecked + final KeyedObjectHashMap, CachedDataIndex> localCachedDataIndexes = + FieldUtils.ensureField(this, CACHED_DATA_INDEXES_UPDATER, null, + () -> new KeyedObjectHashMap<>(CACHED_DATA_INDEX_KEY)); + return localCachedDataIndexes.putIfAbsent(columnNames, CachedDataIndex::new).getDataIndex(); + } + + /** + * Load the data index from the location implementation. Implementations of this method should not perform any + * result caching. + * + * @param columns The columns to load an index for + * @return The data index, or {@code null} if none exists + */ + @Nullable + protected abstract BasicDataIndex loadDataIndex(@NotNull String... columns); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/NonexistentTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/NonexistentTableLocation.java index 42c19821cd3..fb9591d3a33 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/NonexistentTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/NonexistentTableLocation.java @@ -3,12 +3,17 @@ // package io.deephaven.engine.table.impl.locations.impl; +import io.deephaven.api.SortColumn; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.BasicDataIndex; import io.deephaven.engine.table.impl.locations.ColumnLocation; import io.deephaven.engine.table.impl.locations.TableKey; import io.deephaven.engine.table.impl.locations.TableLocation; import io.deephaven.engine.table.impl.locations.TableLocationKey; -import io.deephaven.engine.rowset.RowSetFactory; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.List; /** * {@link TableLocation} implementation for locations that are found to not actually exist when accessed. @@ -31,9 +36,32 @@ public String getImplementationName() { @Override public void refresh() {} + @Override + @NotNull + public List getSortedColumns() { + return List.of(); + } + + @Override + @NotNull + public List getDataIndexColumns() { + return List.of(); + } + + @Override + public boolean hasDataIndex(@NotNull final String... columns) { + return false; + } + @NotNull @Override - protected ColumnLocation makeColumnLocation(@NotNull String name) { + protected ColumnLocation makeColumnLocation(@NotNull final String name) { + throw new UnsupportedOperationException(); + } + + @Override + @Nullable + protected BasicDataIndex loadDataIndex(@NotNull final String... columns) { throw new UnsupportedOperationException(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/ParallelDeferredGroupingProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/ParallelDeferredGroupingProvider.java deleted file mode 100644 index 845e1bdd617..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/ParallelDeferredGroupingProvider.java +++ /dev/null @@ -1,288 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.engine.table.impl.locations.impl; - -import io.deephaven.base.Pair; -import io.deephaven.base.verify.Assert; -import io.deephaven.base.verify.Require; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderSequential; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; -import io.deephaven.engine.table.impl.locations.ColumnLocation; -import io.deephaven.engine.table.impl.locations.KeyRangeGroupingProvider; -import org.jetbrains.annotations.NotNull; - -import java.lang.ref.SoftReference; -import java.util.*; -import java.util.stream.Collectors; - -/** - * Grouping provider that loads column location metadata and assembles grouping indexes lazily on demand. - */ -public class ParallelDeferredGroupingProvider implements KeyRangeGroupingProvider { - - private static final boolean SORT_RANGES = false; - - private final ColumnDefinition columnDefinition; - - public ParallelDeferredGroupingProvider(@NotNull final ColumnDefinition columnDefinition) { - this.columnDefinition = columnDefinition; - } - - private interface RangeAccessor extends Comparator { - - @Override - default int compare(@NotNull final T o1, @NotNull final T o2) { - return Long.compare(getRangeStartInclusive(o1), getRangeStartInclusive(o2)); - } - - long getRangeStartInclusive(@NotNull T range); - - long getRangeEndInclusive(@NotNull T range); - } - - private static final RangeAccessor INT_RANGE_ACCESSOR = new RangeAccessor() { - - @Override - public long getRangeStartInclusive(@NotNull final int[] range) { - return range[0]; - } - - @Override - public long getRangeEndInclusive(@NotNull final int[] range) { - return range[1] - 1; - } - }; - - private static final RangeAccessor LONG_RANGE_ACCESSOR = new RangeAccessor() { - - @Override - public long getRangeStartInclusive(@NotNull final long[] range) { - return range[0]; - } - - @Override - public long getRangeEndInclusive(@NotNull final long[] range) { - return range[1] - 1; - } - }; - - private static class Source { - - private final ColumnLocation columnLocation; - private final long firstKey; - private final long lastKey; - - /** - * We a reference to the resultant grouping information. This allows us to avoid re-reading grouping - * information. - *

- * The reference is soft to avoid having the cache result in an OOM. - */ - private SoftReference> cachedResult; - - private Source(@NotNull final ColumnLocation columnLocation, final long firstKey, final long lastKey) { - Require.neqNull(columnLocation, "columnLocation"); - Require.leq(firstKey, "firstRowKey", lastKey, "lastRowKey"); - this.columnLocation = columnLocation; - this.firstKey = firstKey; - this.lastKey = lastKey; - } - - /** - * Load the grouping metadata for from this source. - *

- * If the grouping information was already read, get it from the cached results map. Any grouping information - * read by this method will be stored in the cached results map. - * - * @param columnDefinition The definition of this column - * @return Grouping metadata as a map from value to position range within this source, or null if the grouping - * information was not present - */ - private Map loadMetadata(@NotNull final ColumnDefinition columnDefinition) { - if (!columnLocation.exists()) { - // noinspection unchecked - return (Map) Collections.singletonMap(null, - new long[] {0, lastKey - firstKey + 1}); - } - - Map valuesToLocationIndexRange = null; - if (cachedResult != null) { - valuesToLocationIndexRange = cachedResult.get(); - } - - if (valuesToLocationIndexRange == null) { - // noinspection unchecked - if ((valuesToLocationIndexRange = - (Map) columnLocation.getMetadata(columnDefinition)) != null) { - cachedResult = new SoftReference<>(valuesToLocationIndexRange); - } - } - - return valuesToLocationIndexRange; - } - - /** - * Apply validation and transformation steps for this source's result. - * - * @param valuesToLocationIndexRange The metadata as a map from value to position range within this source - * @return A list of grouping items to be applied to grouping builder. - */ - private List> validateAndTransformMetadata( - @NotNull final Map valuesToLocationIndexRange) { - final long locationSize = lastKey - firstKey + 1; - if (valuesToLocationIndexRange.isEmpty()) { - // NB: It's impossible for the location to be legitimately empty, since the constructor validates that - // firstRowKey <= lastRowKey. - throw new IllegalStateException( - "Invalid empty grouping for " + columnLocation + ": expected " + locationSize + " rows"); - } - - final Object indicativeValue = - valuesToLocationIndexRange.values().stream().findAny().orElseThrow(IllegalStateException::new); - final RangeAccessor rangeAccessor; - if (indicativeValue.getClass() == int[].class) { - // noinspection unchecked - rangeAccessor = (RangeAccessor) INT_RANGE_ACCESSOR; - } else if (indicativeValue.getClass() == long[].class) { - // noinspection unchecked - rangeAccessor = (RangeAccessor) LONG_RANGE_ACCESSOR; - } else { - throw new UnsupportedOperationException("Unexpected range type " + indicativeValue.getClass() - + " in grouping metadata for " + columnLocation); - } - - final List> result = new ArrayList<>(valuesToLocationIndexRange.size()); - long totalRangeSize = 0; - if (SORT_RANGES) { - final Map reversedMap = new TreeMap<>(rangeAccessor); - for (final Map.Entry entry : valuesToLocationIndexRange.entrySet()) { - reversedMap.put(entry.getValue(), entry.getKey()); - } - for (final Map.Entry entry : reversedMap.entrySet()) { - final long firstPositionInclusive = rangeAccessor.getRangeStartInclusive(entry.getKey()); - final long lastPositionInclusive = rangeAccessor.getRangeEndInclusive(entry.getKey()); - result.add(new GroupingItem<>(entry.getValue(), firstPositionInclusive + firstKey, - lastPositionInclusive + firstKey)); - totalRangeSize += lastPositionInclusive - firstPositionInclusive + 1; - } - } else { - for (final Map.Entry entry : valuesToLocationIndexRange.entrySet()) { - final long firstPositionInclusive = rangeAccessor.getRangeStartInclusive(entry.getValue()); - final long lastPositionInclusive = rangeAccessor.getRangeEndInclusive(entry.getValue()); - result.add(new GroupingItem<>(entry.getKey(), firstPositionInclusive + firstKey, - lastPositionInclusive + firstKey)); - totalRangeSize += lastPositionInclusive - firstPositionInclusive + 1; - } - } - if (locationSize != totalRangeSize) { - throw new IllegalStateException("Invalid grouping for " + columnLocation + ": found " + totalRangeSize - + " rows, but expected " + locationSize); - } - return result; - } - - /** - * Get a list of grouping items that represent the grouping information from this source, or null if grouping - * information was not present. - *

- * If the grouping information was already read, get it from the cached results map. Any grouping information - * read by this method will be stored in the cached results map. - * - * @param columnDefinition The definition of this column - * @return A list of grouping items on success, else null - */ - private List> getTransformedMetadata( - @NotNull final ColumnDefinition columnDefinition) { - final Map metadata = loadMetadata(columnDefinition); - return metadata == null ? null : validateAndTransformMetadata(metadata); - } - } - - private static class GroupingItem { - - private final DATA_TYPE value; - private final long firstKey; - private final long lastKey; - - private GroupingItem(final DATA_TYPE value, final long firstKey, final long lastKey) { - this.value = value; - this.firstKey = firstKey; - this.lastKey = lastKey; - } - - private void updateBuilder(@NotNull final Map valueToBuilder) { - valueToBuilder.computeIfAbsent(value, v -> RowSetFactory.builderSequential()).appendRange(firstKey, - lastKey); - } - } - - private final List> sources = new ArrayList<>(); - - @Override - public void addSource(@NotNull final ColumnLocation columnLocation, - @NotNull final RowSet locationRowSetInTable) { - final long firstKey = locationRowSetInTable.firstRowKey(); - final long lastKey = locationRowSetInTable.lastRowKey(); - if (lastKey - firstKey + 1 != locationRowSetInTable.size()) { - /* - * TODO (https://github.com/deephaven/deephaven-core/issues/816): This constraint is valid for all existing - * formats that support grouping. Address when we integrate grouping/index tables. - */ - throw new IllegalArgumentException( - ParallelDeferredGroupingProvider.class + " only supports a single range per location"); - } - sources.add(new Source<>(columnLocation, firstKey, lastKey)); - } - - private Map buildGrouping(@NotNull final List> includedSources) { - return QueryPerformanceRecorder.withNugget("Build deferred grouping", () -> { - // noinspection unchecked - final List>[] perSourceGroupingLists = - QueryPerformanceRecorder.withNugget("Read and transform grouping metadata", - () -> includedSources.parallelStream() - .map(source -> source.getTransformedMetadata(columnDefinition)) - .toArray(List[]::new)); - - final Map valueToBuilder = - QueryPerformanceRecorder.withNugget("Integrate grouping metadata", () -> { - final Map result = new LinkedHashMap<>(); - for (final List> groupingList : perSourceGroupingLists) { - if (groupingList == null) { - return null; - } - for (final GroupingItem grouping : groupingList) { - grouping.updateBuilder(result); - } - } - return result; - }); - if (valueToBuilder == null) { - return null; - } - - return QueryPerformanceRecorder.withNugget("Build and aggregate group indexes", - () -> valueToBuilder.entrySet().parallelStream() - .map(e -> new Pair<>(e.getKey(), e.getValue().build())) - .collect(Collectors.toMap(Pair::getFirst, Pair::getSecond, Assert::neverInvoked, - LinkedHashMap::new))); - }); - } - - @Override - public Map getGroupToRange() { - return buildGrouping(sources); - } - - @Override - public Pair, Boolean> getGroupToRange(@NotNull final RowSet hint) { - final List> includedSources = sources.stream() - .filter(source -> RowSetFactory.fromRange(source.firstKey, source.lastKey) - .overlaps(hint)) - .collect(Collectors.toList()); - return new Pair<>(buildGrouping(includedSources), includedSources.size() == sources.size()); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/IncrementalNaturalJoinStateManagerTypedBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/IncrementalNaturalJoinStateManagerTypedBase.java index 1506f8649f1..c3521667462 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/IncrementalNaturalJoinStateManagerTypedBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/IncrementalNaturalJoinStateManagerTypedBase.java @@ -401,6 +401,83 @@ public String keyString(int slot) { protected abstract void buildFromRightSide(RowSequence rowSequence, Chunk[] sourceKeyChunks); + public WritableRowRedirection buildIndexedRowRedirection( + QueryTable leftTable, + boolean exactMatch, + InitialBuildContext ibc, + ColumnSource indexRowSets, + JoinControl.RedirectionType redirectionType) { + Assert.eqZero(rehashPointer, "rehashPointer"); + + switch (redirectionType) { + case Contiguous: { + if (!leftTable.isFlat() || leftTable.getRowSet().lastRowKey() > Integer.MAX_VALUE) { + throw new IllegalStateException("Left table is not flat for contiguous row redirection build!"); + } + // we can use an array, which is perfect for a small enough flat table + final long[] innerIndex = new long[leftTable.intSize("contiguous redirection build")]; + + for (int ii = 0; ii < tableSize; ++ii) { + final WritableRowSet leftRowSet = this.mainLeftRowSet.getUnsafe(ii); + if (leftRowSet != null && !leftRowSet.isEmpty()) { + Assert.eq(leftRowSet.size(), "leftRowSet.size()", 1); + // Load the row set from the index row set column. + final RowSet leftRowSetForKey = indexRowSets.get(leftRowSet.firstRowKey()); + // Reset mainLeftRowSet to contain the indexed row set. + mainLeftRowSet.set(ii, leftRowSetForKey.copy()); + final long rightRowKeyForState = mainRightRowKey.getUnsafe(ii); + checkExactMatch(exactMatch, leftRowSet.firstRowKey(), rightRowKeyForState); + leftRowSetForKey.forAllRowKeys(pos -> innerIndex[(int) pos] = rightRowKeyForState); + } + } + + return new ContiguousWritableRowRedirection(innerIndex); + } + case Sparse: { + final LongSparseArraySource sparseRedirections = new LongSparseArraySource(); + for (int ii = 0; ii < tableSize; ++ii) { + final WritableRowSet leftRowSet = this.mainLeftRowSet.getUnsafe(ii); + if (leftRowSet != null && !leftRowSet.isEmpty()) { + Assert.eq(leftRowSet.size(), "leftRowSet.size()", 1); + // Load the row set from the index row set column. + final RowSet leftRowSetForKey = indexRowSets.get(leftRowSet.firstRowKey()); + // Reset mainLeftRowSet to contain the indexed row set. + mainLeftRowSet.set(ii, leftRowSetForKey.copy()); + final long rightRowKeyForState = mainRightRowKey.getUnsafe(ii); + if (rightRowKeyForState != RowSet.NULL_ROW_KEY) { + leftRowSetForKey.forAllRowKeys(pos -> sparseRedirections.set(pos, rightRowKeyForState)); + } else { + checkExactMatch(exactMatch, leftRowSet.firstRowKey(), rightRowKeyForState); + } + } + } + return new LongColumnSourceWritableRowRedirection(sparseRedirections); + } + case Hash: { + final WritableRowRedirection rowRedirection = + WritableRowRedirectionLockFree.FACTORY.createRowRedirection(leftTable.intSize()); + for (int ii = 0; ii < tableSize; ++ii) { + final WritableRowSet leftRowSet = this.mainLeftRowSet.getUnsafe(ii); + if (leftRowSet != null && !leftRowSet.isEmpty()) { + Assert.eq(leftRowSet.size(), "leftRowSet.size()", 1); + // Load the row set from the index row set column. + final RowSet leftRowSetForKey = indexRowSets.get(leftRowSet.firstRowKey()); + // Reset mainLeftRowSet to contain the indexed row set. + mainLeftRowSet.set(ii, leftRowSetForKey.copy()); + final long rightRowKeyForState = mainRightRowKey.getUnsafe(ii); + if (rightRowKeyForState != RowSet.NULL_ROW_KEY) { + leftRowSetForKey.forAllRowKeys(pos -> rowRedirection.put(pos, rightRowKeyForState)); + } else { + checkExactMatch(exactMatch, leftRowSet.firstRowKey(), rightRowKeyForState); + } + } + } + return rowRedirection; + } + } + throw new IllegalStateException("Bad redirectionType: " + redirectionType); + } + public WritableRowRedirection buildRowRedirectionFromRedirections(QueryTable leftTable, boolean exactMatch, InitialBuildContext ibc, JoinControl.RedirectionType redirectionType) { Assert.eqZero(rehashPointer, "rehashPointer"); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/RightIncrementalNaturalJoinStateManagerTypedBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/RightIncrementalNaturalJoinStateManagerTypedBase.java index dbdc3a7b295..04d4fc702a9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/RightIncrementalNaturalJoinStateManagerTypedBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/RightIncrementalNaturalJoinStateManagerTypedBase.java @@ -18,7 +18,6 @@ import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; import io.deephaven.engine.table.impl.sources.LongArraySource; import io.deephaven.engine.table.impl.sources.LongSparseArraySource; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; import io.deephaven.engine.table.impl.sources.immutable.ImmutableLongArraySource; import io.deephaven.engine.table.impl.sources.immutable.ImmutableObjectArraySource; import io.deephaven.engine.table.impl.util.*; @@ -44,10 +43,10 @@ public abstract class RightIncrementalNaturalJoinStateManagerTypedBase extends R // the keys for our hash entries protected final ChunkType[] chunkTypes; - protected final WritableColumnSource[] mainKeySources; + protected final WritableColumnSource[] mainKeySources; protected ImmutableObjectArraySource leftRowSet = - new ImmutableObjectArraySource(WritableRowSet.class, null); + new ImmutableObjectArraySource<>(WritableRowSet.class, null); protected ImmutableLongArraySource rightRowKey = new ImmutableLongArraySource(); protected ImmutableLongArraySource modifiedTrackerCookieSource = new ImmutableLongArraySource(); @@ -196,8 +195,7 @@ public void buildFromLeftSide(Table leftTable, ColumnSource[] leftSources, In protected abstract void buildFromLeftSide(RowSequence rowSequence, Chunk[] sourceKeyChunks); @Override - public void convertLeftGroups(int groupingSize, InitialBuildContext ibc, - ObjectArraySource rowSetSource) { + public void convertLeftDataIndex(int groupingSize, InitialBuildContext ibc, ColumnSource rowSetSource) { for (int ii = 0; ii < tableSize; ++ii) { final WritableRowSet leftRowSet = this.leftRowSet.getUnsafe(ii); if (leftRowSet != null) { @@ -209,8 +207,7 @@ public void convertLeftGroups(int groupingSize, InitialBuildContext ibc, throw new IllegalStateException( "When converting left group position to row keys more than one LHS value was found!"); } - final long groupPosition = leftRowSet.get(0); - this.leftRowSet.set(ii, rowSetSource.get(groupPosition)); + this.leftRowSet.set(ii, rowSetSource.get(leftRowSet.firstRowKey()).copy()); } } } @@ -287,8 +284,8 @@ public WritableRowRedirection buildRowRedirectionFromHashSlot(QueryTable leftTab } @Override - public WritableRowRedirection buildRowRedirectionFromHashSlotGrouped(QueryTable leftTable, - ObjectArraySource rowSetSource, int groupingSize, boolean exactMatch, + public WritableRowRedirection buildRowRedirectionFromHashSlotIndexed(QueryTable leftTable, + ColumnSource rowSetSource, int groupingSize, boolean exactMatch, InitialBuildContext ibc, JoinControl.RedirectionType redirectionType) { return buildRowRedirectionFromHashSlot(leftTable, exactMatch, ibc, redirectionType); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/StaticHashedNaturalJoinStateManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/StaticHashedNaturalJoinStateManager.java index dec0dc76261..f12721547f5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/StaticHashedNaturalJoinStateManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/StaticHashedNaturalJoinStateManager.java @@ -18,6 +18,7 @@ import java.util.function.LongUnaryOperator; public abstract class StaticHashedNaturalJoinStateManager extends StaticNaturalJoinStateManager { + protected StaticHashedNaturalJoinStateManager(ColumnSource[] keySourcesForErrorMessages) { super(keySourcesForErrorMessages); } @@ -38,13 +39,18 @@ public abstract WritableRowRedirection buildRowRedirectionFromHashSlot(QueryTabl public abstract WritableRowRedirection buildRowRedirectionFromRedirections(QueryTable leftTable, boolean exactMatch, LongArraySource leftRedirections, JoinControl.RedirectionType redirectionType); - public abstract WritableRowRedirection buildGroupedRowRedirection(QueryTable leftTable, boolean exactMatch, - long groupingSize, IntegerArraySource leftHashSlots, ArrayBackedColumnSource leftIndices, - JoinControl.RedirectionType redirectionType); + public abstract WritableRowRedirection buildIndexedRowRedirectionFromRedirections(QueryTable leftTable, + boolean exactMatch, RowSet indexTableRowSet, LongArraySource leftRedirections, + ColumnSource indexRowSets, JoinControl.RedirectionType redirectionType); + + public abstract WritableRowRedirection buildIndexedRowRedirectionFromHashSlots(QueryTable leftTable, + boolean exactMatch, RowSet indexTableRowSet, IntegerArraySource leftHashSlots, + ColumnSource indexRowSets, JoinControl.RedirectionType redirectionType); - protected WritableRowRedirection buildGroupedRowRedirection(QueryTable leftTable, boolean exactMatch, - long groupingSize, LongUnaryOperator groupPositionToRightSide, ArrayBackedColumnSource leftIndices, + protected WritableRowRedirection buildIndexedRowRedirection(QueryTable leftTable, boolean exactMatch, + RowSet indexTableRowSet, LongUnaryOperator groupPositionToRightSide, ColumnSource leftRowSets, JoinControl.RedirectionType redirectionType) { + final int rowSetCount = indexTableRowSet.intSize(); switch (redirectionType) { case Contiguous: { if (!leftTable.isFlat()) { @@ -52,10 +58,10 @@ protected WritableRowRedirection buildGroupedRowRedirection(QueryTable leftTable } // we can use an array, which is perfect for a small enough flat table final long[] innerIndex = new long[leftTable.intSize("contiguous redirection build")]; - for (int ii = 0; ii < groupingSize; ++ii) { + for (int ii = 0; ii < rowSetCount; ++ii) { final long rightSide = groupPositionToRightSide.applyAsLong(ii); checkExactMatch(exactMatch, ii, rightSide); - final RowSet leftRowSetForKey = leftIndices.get(ii); + final RowSet leftRowSetForKey = leftRowSets.get(indexTableRowSet.get(ii)); leftRowSetForKey.forAllRowKeys((long ll) -> innerIndex[(int) ll] = rightSide); } return new ContiguousWritableRowRedirection(innerIndex); @@ -63,12 +69,12 @@ protected WritableRowRedirection buildGroupedRowRedirection(QueryTable leftTable case Sparse: { final LongSparseArraySource sparseRedirections = new LongSparseArraySource(); - for (int ii = 0; ii < groupingSize; ++ii) { + for (int ii = 0; ii < rowSetCount; ++ii) { final long rightSide = groupPositionToRightSide.applyAsLong(ii); checkExactMatch(exactMatch, ii, rightSide); if (rightSide != NO_RIGHT_ENTRY_VALUE) { - final RowSet leftRowSetForKey = leftIndices.get(ii); + final RowSet leftRowSetForKey = leftRowSets.get(indexTableRowSet.get(ii)); leftRowSetForKey.forAllRowKeys((long ll) -> sparseRedirections.set(ll, rightSide)); } } @@ -78,12 +84,12 @@ protected WritableRowRedirection buildGroupedRowRedirection(QueryTable leftTable final WritableRowRedirection rowRedirection = WritableRowRedirectionLockFree.FACTORY.createRowRedirection(leftTable.intSize()); - for (int ii = 0; ii < groupingSize; ++ii) { + for (int ii = 0; ii < rowSetCount; ++ii) { final long rightSide = groupPositionToRightSide.applyAsLong(ii); checkExactMatch(exactMatch, ii, rightSide); if (rightSide != NO_RIGHT_ENTRY_VALUE) { - final RowSet leftRowSetForKey = leftIndices.get(ii); + final RowSet leftRowSetForKey = leftRowSets.get(indexTableRowSet.get(ii)); leftRowSetForKey.forAllRowKeys((long ll) -> rowRedirection.put(ll, rightSide)); } } @@ -95,9 +101,9 @@ protected WritableRowRedirection buildGroupedRowRedirection(QueryTable leftTable } public void errorOnDuplicates(IntegerArraySource leftHashSlots, long size, - LongUnaryOperator groupPositionToRightSide, LongUnaryOperator firstLeftKey) { + LongUnaryOperator indexPositionToRightSide, LongUnaryOperator firstLeftKey) { for (int ii = 0; ii < size; ++ii) { - final long rightSide = groupPositionToRightSide.applyAsLong(ii); + final long rightSide = indexPositionToRightSide.applyAsLong(ii); if (rightSide == DUPLICATE_RIGHT_VALUE) { throw new IllegalStateException("Natural Join found duplicate right key for " + extractKeyStringFromSourceTable(firstLeftKey.applyAsLong(ii))); @@ -105,7 +111,7 @@ public void errorOnDuplicates(IntegerArraySource leftHashSlots, long size, } } - public void errorOnDuplicatesGrouped(IntegerArraySource leftHashSlots, long size, + public void errorOnDuplicatesIndexed(IntegerArraySource leftHashSlots, long size, ObjectArraySource rowSetSource) { throw new UnsupportedOperationException(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/StaticNaturalJoinStateManagerTypedBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/StaticNaturalJoinStateManagerTypedBase.java index 95fd2e0c48b..c6322bb2f29 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/StaticNaturalJoinStateManagerTypedBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/naturaljoin/StaticNaturalJoinStateManagerTypedBase.java @@ -240,18 +240,33 @@ public WritableRowRedirection buildRowRedirectionFromRedirections(QueryTable lef return buildRowRedirection(leftTable, exactMatch, leftRedirections::getUnsafe, redirectionType); } - public WritableRowRedirection buildGroupedRowRedirection(QueryTable leftTable, boolean exactMatch, - long groupingSize, IntegerArraySource leftHashSlots, ArrayBackedColumnSource leftIndices, + public WritableRowRedirection buildIndexedRowRedirectionFromRedirections( + QueryTable leftTable, + boolean exactMatch, + RowSet indexTableRowSet, + LongArraySource leftRedirections, + ColumnSource indexRowSets, JoinControl.RedirectionType redirectionType) { - return buildGroupedRowRedirection(leftTable, exactMatch, groupingSize, - (long groupPosition) -> mainRightRowKey.getUnsafe(leftHashSlots.getUnsafe(groupPosition)), leftIndices, + return buildIndexedRowRedirection(leftTable, exactMatch, indexTableRowSet, + leftRedirections::getUnsafe, indexRowSets, redirectionType); + } + + public WritableRowRedirection buildIndexedRowRedirectionFromHashSlots( + QueryTable leftTable, + boolean exactMatch, + RowSet indexTableRowSet, + IntegerArraySource leftHashSlots, + ColumnSource indexRowSets, + JoinControl.RedirectionType redirectionType) { + return buildIndexedRowRedirection(leftTable, exactMatch, indexTableRowSet, + (long groupPosition) -> mainRightRowKey.getUnsafe(leftHashSlots.getUnsafe(groupPosition)), indexRowSets, redirectionType); } - public void errorOnDuplicatesGrouped(IntegerArraySource leftHashSlots, long size, + public void errorOnDuplicatesIndexed(IntegerArraySource leftHashSlots, long size, ObjectArraySource rowSetSource) { errorOnDuplicates(leftHashSlots, size, - (long groupPosition) -> mainRightRowKey.getUnsafe(leftHashSlots.getUnsafe(groupPosition)), + (long indexPosition) -> mainRightRowKey.getUnsafe(leftHashSlots.getUnsafe(indexPosition)), (long row) -> rowSetSource.getUnsafe(row).firstRowKey()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/partitioned/PartitionedTableProxyImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/partitioned/PartitionedTableProxyImpl.java index 24f9111bf77..1ae21286f1b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/partitioned/PartitionedTableProxyImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/partitioned/PartitionedTableProxyImpl.java @@ -473,12 +473,14 @@ public PartitionedTable.Proxy where(Filter filter) { @Override public PartitionedTable.Proxy whereIn(TableOperations rightTable, Collection columnsToMatch) { + // TODO (https://github.com/deephaven/deephaven-core/issues/5261): Share set tables when possible return complexTransform(rightTable, (ct, ot) -> ct.whereIn(ot, columnsToMatch), columnsToMatch); } @Override public PartitionedTable.Proxy whereNotIn(TableOperations rightTable, Collection columnsToMatch) { + // TODO (https://github.com/deephaven/deephaven-core/issues/5261): Share set tables when possible return complexTransform(rightTable, (ct, ot) -> ct.whereNotIn(ot, columnsToMatch), columnsToMatch); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java index 7b680c04814..14ac7076896 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java @@ -8,7 +8,6 @@ import io.deephaven.base.log.LogOutputAppendable; import io.deephaven.base.verify.Assert; import io.deephaven.configuration.Configuration; -import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.exceptions.SnapshotUnsuccessfulException; import io.deephaven.engine.updategraph.*; @@ -1353,7 +1352,8 @@ public static long callDataSnapshotFunction( } } state.startLockedSnapshot(control); - try { + final LivenessScope snapshotLivenessScope = new LivenessScope(); + try (final SafeCloseable ignored = LivenessScopeStack.open(snapshotLivenessScope, true)) { final long beforeClockValue = updateGraph.clock().currentValue(); final Boolean previousValuesRequested = control.usePreviousValues(beforeClockValue); @@ -1383,6 +1383,9 @@ public static long callDataSnapshotFunction( log.debug().append(logPrefix).append(" Non-concurrent Snapshot Function took ") .append(System.currentTimeMillis() - attemptStart).append("ms").endl(); } + if (functionSuccessful && consistent) { + snapshotLivenessScope.transferTo(initialLivenessManager); + } step = LogicalClock.getStep(afterClockValue); } finally { state.endLockedSnapshot(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/QueryReplayGroupedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/QueryReplayGroupedTable.java index f228f05b460..55619e12b0e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/QueryReplayGroupedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/QueryReplayGroupedTable.java @@ -3,19 +3,23 @@ // package io.deephaven.engine.table.impl.replay; + +import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.TrackingRowSet; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; +import io.deephaven.engine.table.BasicDataIndex; import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.sources.RedirectedColumnSource; -import io.deephaven.engine.table.TupleSource; -import io.deephaven.engine.table.impl.TupleSourceFactory; -import io.deephaven.engine.table.impl.util.*; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import org.jetbrains.annotations.NotNull; import java.time.Instant; -import java.util.*; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Objects; +import java.util.PriorityQueue; public abstract class QueryReplayGroupedTable extends ReplayTableBase implements Runnable { @@ -38,8 +42,8 @@ protected static class IteratorsAndNextTime implements Comparable columnSource; - Instant lastTime; - long lastIndex; + protected Instant lastTime; + protected long lastIndex; public final long pos; private IteratorsAndNextTime(RowSet.Iterator iterator, ColumnSource columnSource, long pos) { @@ -50,7 +54,7 @@ private IteratorsAndNextTime(RowSet.Iterator iterator, ColumnSource col lastTime = columnSource.get(lastIndex); } - IteratorsAndNextTime next() { + protected IteratorsAndNextTime next() { if (iterator.hasNext()) { lastIndex = iterator.nextLong(); lastTime = columnSource.get(lastIndex); @@ -71,33 +75,32 @@ public int compareTo(IteratorsAndNextTime o) { protected QueryReplayGroupedTable( @NotNull final String description, - @NotNull final TrackingRowSet rowSet, - @NotNull final Map> input, + @NotNull final Table source, @NotNull final String timeColumn, @NotNull final Replayer replayer, @NotNull final WritableRowRedirection rowRedirection, @NotNull final String[] groupingColumns) { - super(description, RowSetFactory.empty().toTracking(), getResultSources(input, rowRedirection)); + super(description, RowSetFactory.empty().toTracking(), + getResultSources(source.getColumnSourceMap(), rowRedirection)); this.rowRedirection = rowRedirection; this.replayer = Objects.requireNonNull(replayer, "replayer"); - Map grouping; - final ColumnSource[] columnSources = - Arrays.stream(groupingColumns).map(input::get).toArray(ColumnSource[]::new); - final TupleSource tupleSource = TupleSourceFactory.makeTupleSource(columnSources); - grouping = RowSetIndexer.of(rowSet).getGrouping(tupleSource); + final BasicDataIndex dataIndex = DataIndexer.getOrCreateDataIndex(source, groupingColumns); + final Table indexTable = dataIndex.table(); - // noinspection unchecked - ColumnSource timeSource = (ColumnSource) input.get(timeColumn); + ColumnSource timeSource = source.getColumnSource(timeColumn, Instant.class); int pos = 0; - for (RowSet groupRowSet : grouping.values()) { - RowSet.Iterator iterator = groupRowSet.iterator(); - if (iterator.hasNext()) { - allIterators.add(new IteratorsAndNextTime(iterator, timeSource, pos++)); + try (final CloseableIterator it = indexTable.columnIterator(dataIndex.rowSetColumnName())) { + while (it.hasNext()) { + RowSet.Iterator iterator = it.next().iterator(); + if (iterator.hasNext()) { + allIterators.add(new IteratorsAndNextTime(iterator, timeSource, pos++)); + } } } + run(); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/ReplayGroupedFullTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/ReplayGroupedFullTable.java index cbff2136cb1..c7ac04393e7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/ReplayGroupedFullTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/ReplayGroupedFullTable.java @@ -6,22 +6,17 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetBuilderRandom; import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.TrackingRowSet; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.util.*; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.util.WritableRowRedirection; import io.deephaven.time.DateTimeUtils; -import java.util.Map; - public class ReplayGroupedFullTable extends QueryReplayGroupedTable { + private int redirIndexSize; - public ReplayGroupedFullTable(TrackingRowSet rowSet, Map> input, - String timeColumn, - Replayer replayer, String groupingColumn) { - super("ReplayGroupedFullTable", rowSet, input, timeColumn, replayer, - WritableRowRedirection.FACTORY.createRowRedirection((int) rowSet.size()), - new String[] {groupingColumn}); + public ReplayGroupedFullTable(Table source, String timeColumn, Replayer replayer, String groupingColumn) { + super("ReplayGroupedFullTable", source, timeColumn, replayer, + WritableRowRedirection.FACTORY.createRowRedirection(source.intSize()), new String[] {groupingColumn}); redirIndexSize = 0; // We do not modify existing entries in the WritableRowRedirection (we only add at the end), so there's no need // to ask the WritableRowRedirection to track previous values. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/ReplayLastByGroupedTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/ReplayLastByGroupedTable.java index 1c3cd92e637..1e7734dc0b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/ReplayLastByGroupedTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/ReplayLastByGroupedTable.java @@ -6,23 +6,18 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetBuilderRandom; import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.TrackingRowSet; -import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.util.*; import io.deephaven.time.DateTimeUtils; import java.time.Instant; -import java.util.Map; public class ReplayLastByGroupedTable extends QueryReplayGroupedTable { - public ReplayLastByGroupedTable(TrackingRowSet rowSet, Map> input, - String timeColumn, - Replayer replayer, String[] groupingColumns) { - super("ReplayLastByGroupedTable", rowSet, input, timeColumn, replayer, + public ReplayLastByGroupedTable(Table source, String timeColumn, Replayer replayer, String[] groupingColumns) { + super("ReplayLastByGroupedTable", source, timeColumn, replayer, WritableRowRedirection.FACTORY.createRowRedirection(100), groupingColumns); - // noinspection unchecked - replayer.registerTimeSource(rowSet, (ColumnSource) input.get(timeColumn)); + replayer.registerTimeSource(source.getRowSet(), source.getColumnSource(timeColumn, Instant.class)); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/Replayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/Replayer.java index 247fbaab748..854d176b395 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/Replayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/replay/Replayer.java @@ -218,10 +218,12 @@ public Table replay(Table dataSource, String timeColumn) { */ @Override public Table replayGrouped(Table dataSource, String timeColumn, String groupingColumn) { + if (dataSource.isRefreshing()) { + dataSource = dataSource.snapshot(); + } final ReplayGroupedFullTable result; try (final SafeCloseable ignored = ExecutionContext.getContext().withUpdateGraph(updateGraph).open()) { - result = new ReplayGroupedFullTable( - dataSource.getRowSet(), dataSource.getColumnSourceMap(), timeColumn, this, groupingColumn); + result = new ReplayGroupedFullTable(dataSource, timeColumn, this, groupingColumn); } currentTables.add(result); if (deltaNanos < Long.MAX_VALUE) { @@ -240,10 +242,12 @@ public Table replayGrouped(Table dataSource, String timeColumn, String groupingC */ @Override public Table replayGroupedLastBy(Table dataSource, String timeColumn, String... groupingColumns) { + if (dataSource.isRefreshing()) { + dataSource = dataSource.snapshot(); + } final ReplayLastByGroupedTable result; try (final SafeCloseable ignored = ExecutionContext.getContext().withUpdateGraph(updateGraph).open()) { - result = new ReplayLastByGroupedTable( - dataSource.getRowSet(), dataSource.getColumnSourceMap(), timeColumn, this, groupingColumns); + result = new ReplayLastByGroupedTable(dataSource, timeColumn, this, groupingColumns); } currentTables.add(result); if (deltaNanos < Long.MAX_VALUE) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ComposedFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ComposedFilter.java index 9777c518f5b..4bc67ce3e8b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ComposedFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ComposedFilter.java @@ -3,11 +3,16 @@ // package io.deephaven.engine.table.impl.select; +import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.BaseTable; import io.deephaven.engine.updategraph.NotificationQueue; import io.deephaven.engine.liveness.LivenessArtifact; import io.deephaven.engine.table.impl.DependencyStreamProvider; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.SafeCloseableList; import io.deephaven.util.annotations.TestUseOnly; +import org.jetbrains.annotations.NotNull; import java.util.*; import java.util.stream.Stream; @@ -62,6 +67,20 @@ public void init(TableDefinition tableDefinition) { } } + @Override + public SafeCloseable beginOperation(@NotNull final Table sourceTable) { + return Arrays.stream(componentFilters) + .map((final WhereFilter whereFilter) -> whereFilter.beginOperation(sourceTable)) + .collect(SafeCloseableList.COLLECTOR); + } + + @Override + public void validateSafeForRefresh(@NotNull final BaseTable sourceTable) { + for (WhereFilter filter : componentFilters) { + filter.validateSafeForRefresh(sourceTable); + } + } + @Override public boolean isSimpleFilter() { for (WhereFilter filter : componentFilters) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/DynamicWhereFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/DynamicWhereFilter.java index bbed868385f..7e1b2d7f4bf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/DynamicWhereFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/DynamicWhereFilter.java @@ -5,27 +5,32 @@ import io.deephaven.base.log.LogOutput; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.datastructures.util.CollectionUtil; +import io.deephaven.engine.liveness.LivenessScopeStack; 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.*; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; +import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.table.impl.dataindex.DataIndexUtils; +import io.deephaven.engine.table.impl.dataindex.DataIndexKeySet; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.iterators.ChunkedColumnIterator; import io.deephaven.engine.updategraph.NotificationQueue; -import io.deephaven.engine.updategraph.DynamicNode; -import io.deephaven.engine.table.impl.*; -import io.deephaven.engine.table.impl.select.setinclusion.SetInclusionKernel; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.WritableBooleanChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.engine.table.impl.TupleSourceFactory; import io.deephaven.engine.updategraph.UpdateGraph; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.hash.KeyedObjectHashMap; +import io.deephaven.hash.KeyedObjectKey; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.annotations.ReferentialIntegrity; +import org.apache.commons.lang3.mutable.MutableInt; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.*; +import java.util.function.Function; +import java.util.stream.Collectors; /** * A where filter that extracts a set of inclusion or exclusion keys from a set table. @@ -33,47 +38,53 @@ * Each time the set table ticks, the entire where filter is recalculated. */ public class DynamicWhereFilter extends WhereFilterLivenessArtifactImpl implements NotificationQueue.Dependency { + private static final int CHUNK_SIZE = 1 << 16; - private final boolean setRefreshing; private final MatchPair[] matchPairs; - private final TupleSource setTupleSource; private final boolean inclusion; - private final HashSet liveValues = new HashSet<>(); - private boolean liveValuesArrayValid = false; - private boolean kernelValid = false; - private Object[] liveValuesArray = null; - private SetInclusionKernel setInclusionKernel = null; + private final DataIndexKeySet liveValues; - // this reference must be maintained for reachability - @SuppressWarnings({"FieldCanBeLocal", "unused"}) private final QueryTable setTable; + private final ChunkSource.WithPrev setKeySource; @SuppressWarnings("FieldCanBeLocal") - // this reference must be maintained for reachability + @ReferentialIntegrity private final InstrumentedTableUpdateListener setUpdateListener; + private Object[] liveValuesArray; + + private ColumnSource[] sourceKeyColumns; + /** + * The optimal data index for this filter. + */ + @Nullable + private DataIndex sourceDataIndex; + private int[] indexToSetKeyOffsets; + private RecomputeListener listener; private QueryTable resultTable; - public DynamicWhereFilter(final QueryTable setTable, final boolean inclusion, final MatchPair... setColumnsNames) { - setRefreshing = setTable.isRefreshing(); - if (setRefreshing) { + public DynamicWhereFilter( + @NotNull final QueryTable setTable, + final boolean inclusion, + final MatchPair... setColumnsNames) { + if (setTable.isRefreshing()) { updateGraph.checkInitiateSerialTableOperation(); } - this.matchPairs = setColumnsNames; this.inclusion = inclusion; + liveValues = DataIndexUtils.makeKeySet(setColumnsNames.length); + final ColumnSource[] setColumns = Arrays.stream(matchPairs) .map(mp -> setTable.getColumnSource(mp.rightColumn())).toArray(ColumnSource[]::new); - if (setRefreshing) { - this.setTable = setTable; - setTupleSource = TupleSourceFactory.makeTupleSource(setColumns); + if (setTable.isRefreshing()) { + setKeySource = DataIndexUtils.makeBoxedKeySource(setColumns); if (setTable.getRowSet().isNonempty()) { try (final CloseableIterator initialKeysIterator = ChunkedColumnIterator.make( - setTupleSource, setTable.getRowSet(), getChunkSize(setTable.getRowSet()))) { + setKeySource, setTable.getRowSet(), getChunkSize(setTable.getRowSet()))) { initialKeysIterator.forEachRemaining(this::addKey); } } @@ -81,6 +92,7 @@ public DynamicWhereFilter(final QueryTable setTable, final boolean inclusion, fi final String[] setColumnNames = Arrays.stream(matchPairs).map(MatchPair::rightColumn).toArray(String[]::new); final ModifiedColumnSet setColumnsMCS = setTable.newModifiedColumnSet(setColumnNames); + this.setTable = setTable; setUpdateListener = new InstrumentedTableUpdateListenerAdapter( "DynamicWhereFilter(" + Arrays.toString(setColumnsNames) + ")", setTable, false) { @@ -97,7 +109,7 @@ public void onUpdate(final TableUpdate upstream) { // Remove removed keys if (hasRemoves) { try (final CloseableIterator removedKeysIterator = ChunkedColumnIterator.make( - setTupleSource.getPrevSource(), upstream.removed(), getChunkSize(upstream.removed()))) { + setKeySource.getPrevSource(), upstream.removed(), getChunkSize(upstream.removed()))) { removedKeysIterator.forEachRemaining(DynamicWhereFilter.this::removeKey); } } @@ -107,10 +119,10 @@ public void onUpdate(final TableUpdate upstream) { if (hasModifies) { // @formatter:off try (final CloseableIterator preModifiedKeysIterator = ChunkedColumnIterator.make( - setTupleSource.getPrevSource(), upstream.getModifiedPreShift(), + setKeySource.getPrevSource(), upstream.getModifiedPreShift(), getChunkSize(upstream.getModifiedPreShift())); final CloseableIterator postModifiedKeysIterator = ChunkedColumnIterator.make( - setTupleSource, upstream.modified(), + setKeySource, upstream.modified(), getChunkSize(upstream.modified()))) { // @formatter:on while (preModifiedKeysIterator.hasNext()) { @@ -132,7 +144,7 @@ public void onUpdate(final TableUpdate upstream) { // Add added keys if (hasAdds) { try (final CloseableIterator addedKeysIterator = ChunkedColumnIterator.make( - setTupleSource, upstream.added(), getChunkSize(upstream.added()))) { + setKeySource, upstream.added(), getChunkSize(upstream.added()))) { addedKeysIterator.forEachRemaining(DynamicWhereFilter.this::addKey); } } @@ -169,20 +181,33 @@ public void onFailureInternal(Throwable originalException, Entry sourceEntry) { manage(setUpdateListener); } else { this.setTable = null; - setTupleSource = null; + setKeySource = null; + setUpdateListener = null; if (setTable.getRowSet().isNonempty()) { - final TupleSource temporaryTupleSource = TupleSourceFactory.makeTupleSource(setColumns); + final ChunkSource.WithPrev tmpKeySource = DataIndexUtils.makeBoxedKeySource(setColumns); try (final CloseableIterator initialKeysIterator = ChunkedColumnIterator.make( - temporaryTupleSource, setTable.getRowSet(), getChunkSize(setTable.getRowSet()))) { + tmpKeySource, setTable.getRowSet(), getChunkSize(setTable.getRowSet()))) { initialKeysIterator.forEachRemaining(this::addKeyUnchecked); } } - kernelValid = liveValuesArrayValid = false; - setInclusionKernel = null; - setUpdateListener = null; } } + /** + * "Copy constructor" for DynamicWhereFilter's with static set tables. + */ + private DynamicWhereFilter( + @NotNull final DataIndexKeySet liveValues, + final boolean inclusion, + final MatchPair... setColumnsNames) { + this.liveValues = liveValues; + this.matchPairs = setColumnsNames; + this.inclusion = inclusion; + setTable = null; + setKeySource = null; + setUpdateListener = null; + } + @Override public UpdateGraph getUpdateGraph() { return updateGraph; @@ -190,11 +215,10 @@ public UpdateGraph getUpdateGraph() { private void removeKey(Object key) { final boolean removed = liveValues.remove(key); - if (!removed) { + if (!removed && key != null) { throw new RuntimeException("Inconsistent state, key not found in set: " + key); } - kernelValid = liveValuesArrayValid = false; - setInclusionKernel = null; + liveValuesArray = null; } private void addKey(Object key) { @@ -202,14 +226,125 @@ private void addKey(Object key) { if (!added) { throw new RuntimeException("Inconsistent state, key already in set:" + key); } - kernelValid = liveValuesArrayValid = false; - setInclusionKernel = null; + liveValuesArray = null; } private void addKeyUnchecked(Object key) { liveValues.add(key); } + /** + * {@inheritDoc} + *

+ * If {@code sourceTable#isRefreshing()}, this method must only be invoked when it's + * {@link UpdateGraph#checkInitiateSerialTableOperation() safe} to initialize serial table operations. + */ + @Override + public SafeCloseable beginOperation(@NotNull final Table sourceTable) { + if (sourceDataIndex != null) { + throw new IllegalStateException("Inputs already initialized, use copy() instead of re-using a WhereFilter"); + } + getUpdateGraph(this, sourceTable); + final String[] keyColumnNames = MatchPair.getLeftColumns(matchPairs); + sourceKeyColumns = Arrays.stream(matchPairs) + .map(mp -> sourceTable.getColumnSource(mp.leftColumn())).toArray(ColumnSource[]::new); + try (final SafeCloseable ignored = sourceTable.isRefreshing() ? LivenessScopeStack.open() : null) { + sourceDataIndex = optimalIndex(sourceTable, keyColumnNames); + if (sourceDataIndex != null) { + if (sourceDataIndex.isRefreshing()) { + manage(sourceDataIndex); + } + indexToSetKeyOffsets = computeIndexToSetKeyOffsets(sourceDataIndex, sourceKeyColumns); + } + } + return () -> { + }; + } + + /** + * Returns the optimal data index for the supplied table, or null if no index is available. The ideal index would + * contain all key columns but a partial match is also acceptable. + */ + @Nullable + private static DataIndex optimalIndex(final Table inputTable, final String[] keyColumnNames) { + final DataIndex fullIndex = DataIndexer.getDataIndex(inputTable, keyColumnNames); + if (fullIndex != null) { + return fullIndex; + } + return DataIndexer.getOptimalPartialIndex(inputTable, keyColumnNames); + } + + /** + * Calculates a mapping from the index of a {@link ColumnSource} in the data index to the index of the corresponding + * {@link ColumnSource} in the key sources from the source table for a DynamicWhereFilter. This allows for mapping + * keys from the {@link #liveValues} to keys in the {@link #sourceDataIndex}. + * + * @param dataIndex The {@link DataIndex} to use for the mapping + * @param keySources The key {@link ColumnSource ColumnSources} from the source table + * @return A mapping from the data index source offset to the key source offset for the same {@link ColumnSource}, + * or {@code null} if there is no mapping needed (because {@code dataIndex} and {@code keySources} have the + * same columns in the same order + */ + private static int[] computeIndexToSetKeyOffsets( + @NotNull final DataIndex dataIndex, + @NotNull final ColumnSource[] keySources) { + + final class SourceOffsetPair { + + private final ColumnSource source; + private final int offset; + + private SourceOffsetPair(@NotNull final ColumnSource source, final int offset) { + this.source = source; + this.offset = offset; + } + } + + final MutableInt dataIndexSourceOffset = new MutableInt(0); + final KeyedObjectHashMap, SourceOffsetPair> dataIndexSources = + dataIndex.keyColumnNamesByIndexedColumn().keySet() + .stream() + .collect(Collectors.toMap( + cs -> cs, + cs -> new SourceOffsetPair(cs, dataIndexSourceOffset.getAndIncrement()), + Assert::neverInvoked, + () -> new KeyedObjectHashMap<>(new KeyedObjectKey.ExactAdapter<>(sop -> sop.source)))); + + final int[] indexToKeySourceOffsets = new int[dataIndexSources.size()]; + boolean isAscending = true; + for (int kci = 0; kci < keySources.length; ++kci) { + final SourceOffsetPair dataIndexSource = dataIndexSources.get(keySources[kci]); + if (dataIndexSource != null) { + indexToKeySourceOffsets[dataIndexSource.offset] = kci; + isAscending &= dataIndexSource.offset == kci; + if (dataIndexSourceOffset.decrementAndGet() == 0) { + return isAscending && keySources.length == indexToKeySourceOffsets.length + ? null + : indexToKeySourceOffsets; + } + } + } + + throw new IllegalArgumentException(String.format( + "The provided key sources %s don't match the data index key sources %s", + Arrays.toString(keySources), dataIndex.keyColumnNamesByIndexedColumn().keySet())); + } + + @NotNull + private Function compoundKeyMappingFunction() { + if (indexToSetKeyOffsets == null) { + return Function.identity(); + } + final Object[] keysInDataIndexOrder = new Object[indexToSetKeyOffsets.length]; + return (final Object key) -> { + final Object[] keysInSetOrder = (Object[]) key; + for (int ki = 0; ki < keysInDataIndexOrder.length; ++ki) { + keysInDataIndexOrder[ki] = keysInSetOrder[indexToSetKeyOffsets[ki]]; + } + return keysInDataIndexOrder; + }; + } + @Override public List getColumns() { return Arrays.asList(MatchPair.getLeftColumns(matchPairs)); @@ -234,94 +369,144 @@ public WritableRowSet filter( throw new PreviousFilteringNotSupported(); } - final ColumnSource[] keyColumns = Arrays.stream(matchPairs) - .map(mp -> table.getColumnSource(mp.leftColumn())).toArray(ColumnSource[]::new); - final TupleSource tupleSource = TupleSourceFactory.makeTupleSource(keyColumns); - final TrackingRowSet trackingSelection = selection.isTracking() ? selection.trackingCast() : null; - if (matchPairs.length == 1) { - // this is just a single column filter so it will actually be exactly right - if (!liveValuesArrayValid) { - liveValuesArray = liveValues.toArray(CollectionUtil.ZERO_LENGTH_OBJECT_ARRAY); - liveValuesArrayValid = true; + // Single column filter, delegate to the column source. + if (liveValuesArray == null) { + liveValuesArray = liveValues.toArray(); } - return table.getColumnSource(matchPairs[0].leftColumn()).match(!inclusion, false, false, selection, - liveValuesArray); + // Our keys are reinterpreted, so we need to reinterpret the column source for correct matching. + final ColumnSource source = ReinterpretUtils.maybeConvertToPrimitive(sourceKeyColumns[0]); + return source.match(!inclusion, false, false, sourceDataIndex, selection, liveValuesArray); } - // pick something sensible - if (trackingSelection != null) { - final RowSetIndexer selectionIndexer = RowSetIndexer.of(trackingSelection); - if (selectionIndexer.hasGrouping(keyColumns)) { - if (selection.size() > (selectionIndexer.getGrouping(tupleSource).size() * 2L)) { - return filterGrouping(trackingSelection, selectionIndexer, tupleSource); + if (sourceDataIndex != null) { + // Does our index contain every key column? + + if (sourceDataIndex.keyColumnNames().size() == sourceKeyColumns.length) { + // Even if we have an index, we may be better off with a linear search. + if (selection.size() > (sourceDataIndex.table().size() * 2L)) { + return filterFullIndex(selection); } else { - return filterLinear(selection, tupleSource); + return filterLinear(selection, inclusion); } } - final boolean allGrouping = Arrays.stream(keyColumns).allMatch(selectionIndexer::hasGrouping); - if (allGrouping) { - return filterGrouping(trackingSelection, selectionIndexer, tupleSource); - } - final ColumnSource[] sourcesWithGroupings = Arrays.stream(keyColumns) - .filter(selectionIndexer::hasGrouping).toArray(ColumnSource[]::new); - final OptionalInt minGroupCount = Arrays.stream(sourcesWithGroupings) - .mapToInt(x -> selectionIndexer.getGrouping(x).size()).min(); - if (minGroupCount.isPresent() && (minGroupCount.getAsInt() * 4L) < selection.size()) { - return filterGrouping(trackingSelection, selectionIndexer, tupleSource); + // We have a partial index, should we use it? + if (selection.size() > (sourceDataIndex.table().size() * 4L)) { + return filterPartialIndex(selection); } } - return filterLinear(selection, tupleSource); + return filterLinear(selection, inclusion); } - private WritableRowSet filterGrouping( - TrackingRowSet selection, - RowSetIndexer selectionIndexer, - TupleSource tupleSource) { - final RowSet matchingKeys = selectionIndexer.getSubSetForKeySet(liveValues, tupleSource); - return (inclusion ? matchingKeys.copy() : selection.minus(matchingKeys)); + @NotNull + private WritableRowSet filterFullIndex(@NotNull final RowSet selection) { + Assert.neqNull(sourceDataIndex, "sourceDataIndex"); + Assert.gt(sourceKeyColumns.length, "sourceKeyColumns.length", 1); + + final WritableRowSet filtered = inclusion ? RowSetFactory.empty() : selection.copy(); + // noinspection DataFlowIssue + final DataIndex.RowKeyLookup rowKeyLookup = sourceDataIndex.rowKeyLookup(); + final ColumnSource rowSetColumn = sourceDataIndex.rowSetColumn(); + final Function keyMappingFunction = compoundKeyMappingFunction(); + + liveValues.forEach(key -> { + final Object mappedKey = keyMappingFunction.apply(key); + final long rowKey = rowKeyLookup.apply(mappedKey, false); + final RowSet rowSet = rowSetColumn.get(rowKey); + if (rowSet != null) { + if (inclusion) { + try (final RowSet intersected = rowSet.intersect(selection)) { + filtered.insert(intersected); + } + } else { + filtered.remove(rowSet); + } + } + }); + return filtered; } - private WritableRowSet filterLinear(RowSet selection, TupleSource tupleSource) { - if (selection.isEmpty()) { - return RowSetFactory.empty(); - } + @NotNull + private WritableRowSet filterPartialIndex(@NotNull final RowSet selection) { + Assert.neqNull(sourceDataIndex, "sourceDataIndex"); + Assert.gt(sourceKeyColumns.length, "sourceKeyColumns.length", 1); + + final WritableRowSet matching; + try (final WritableRowSet possiblyMatching = RowSetFactory.empty()) { + // First, compute a possibly-matching subset of selection based on the partial index. + + // noinspection DataFlowIssue + final DataIndex.RowKeyLookup rowKeyLookup = sourceDataIndex.rowKeyLookup(); + final ColumnSource rowSetColumn = sourceDataIndex.rowSetColumn(); + + if (indexToSetKeyOffsets.length == 1) { + // We could delegate to the column source, but we'd have to create a single-column view of liveValues. + + // Only one indexed source, so we can use the RowSetLookup directly on the right sub-key. + final int keyOffset = indexToSetKeyOffsets[0]; + liveValues.forEach(key -> { + final Object[] keysInSetOrder = (Object[]) key; + final long rowKey = rowKeyLookup.apply(keysInSetOrder[keyOffset], false); + final RowSet rowSet = rowSetColumn.get(rowKey); + if (rowSet != null) { + try (final RowSet intersected = rowSet.intersect(selection)) { + possiblyMatching.insert(intersected); + } + } + }); + } else { + final Function keyMappingFunction = compoundKeyMappingFunction(); + liveValues.forEach(key -> { + final Object mappedKey = keyMappingFunction.apply(key); + final long rowKey = rowKeyLookup.apply(mappedKey, false); + final RowSet rowSet = rowSetColumn.get(rowKey); + if (rowSet != null) { + try (final RowSet intersected = rowSet.intersect(selection)) { + possiblyMatching.insert(intersected); + } + } + }); + } - if (!kernelValid) { - setInclusionKernel = SetInclusionKernel.makeKernel(tupleSource.getChunkType(), liveValues, inclusion); - kernelValid = true; + // Now, do linear filter on possiblyMatching to determine the values to include or exclude from selection. + matching = filterLinear(possiblyMatching, true); + } + if (inclusion) { + return matching; + } + try (final SafeCloseable ignored = matching) { + return selection.minus(matching); } + } + + private WritableRowSet filterLinear(final RowSet selection, final boolean filterInclusion) { + // Any single column filter is pushed through AbstractColumnSource.match() + Assert.gt(sourceKeyColumns.length, "sourceKeyColumns.length", 1); final RowSetBuilderSequential indexBuilder = RowSetFactory.builderSequential(); + final ChunkSource keySource = DataIndexUtils.makeBoxedKeySource(sourceKeyColumns); + final int maxChunkSize = getChunkSize(selection); // @formatter:off - try (final ColumnSource.GetContext keyGetContext = tupleSource.makeGetContext(maxChunkSize); - final RowSequence.Iterator selectionIterator = selection.getRowSequenceIterator(); - final WritableLongChunk selectionRowKeyChunk = - WritableLongChunk.makeWritableChunk(maxChunkSize); - final WritableBooleanChunk matches = WritableBooleanChunk.makeWritableChunk(maxChunkSize)) { + try (final ChunkSource.GetContext keyGetContext = keySource.makeGetContext(maxChunkSize); + final RowSequence.Iterator selectionIterator = selection.getRowSequenceIterator()) { // @formatter:on while (selectionIterator.hasMore()) { final RowSequence selectionChunk = selectionIterator.getNextRowSequenceWithLength(maxChunkSize); - - final Chunk keyChunk = Chunk.downcast(tupleSource.getChunk(keyGetContext, selectionChunk)); + final LongChunk selectionRowKeyChunk = selectionChunk.asRowKeyChunk(); + final ObjectChunk keyChunk = + keySource.getChunk(keyGetContext, selectionChunk).asObjectChunk(); final int thisChunkSize = keyChunk.size(); - setInclusionKernel.matchValues(keyChunk, matches); - - selectionRowKeyChunk.setSize(thisChunkSize); - selectionChunk.fillRowKeyChunk(selectionRowKeyChunk); for (int ii = 0; ii < thisChunkSize; ++ii) { - if (matches.get(ii)) { + if (liveValues.contains(keyChunk.get(ii)) == filterInclusion) indexBuilder.appendKey(selectionRowKeyChunk.get(ii)); - } } } } - return indexBuilder.build(); } @@ -337,26 +522,30 @@ public boolean isSimpleFilter() { @Override public boolean isRefreshing() { - return setRefreshing; + return setUpdateListener != null; } @Override public void setRecomputeListener(RecomputeListener listener) { this.listener = listener; this.resultTable = listener.getTable(); - if (DynamicNode.isDynamicAndIsRefreshing(setTable)) { + if (isRefreshing()) { listener.setIsRefreshing(true); } } @Override public DynamicWhereFilter copy() { + if (setTable == null) { + return new DynamicWhereFilter(liveValues, inclusion, matchPairs); + } return new DynamicWhereFilter(setTable, inclusion, matchPairs); } @Override public boolean satisfied(final long step) { - return setUpdateListener == null || setUpdateListener.satisfied(step); + final boolean indexSatisfied = sourceDataIndex == null || sourceDataIndex.table().satisfied(step); + return indexSatisfied && (setUpdateListener == null || setUpdateListener.satisfied(step)); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FunctionalColumn.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FunctionalColumn.java index 511de8ee358..e7651213c6b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FunctionalColumn.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FunctionalColumn.java @@ -19,11 +19,10 @@ import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.function.BiFunction; import java.util.function.Function; public class FunctionalColumn implements SelectColumn { @@ -37,13 +36,19 @@ public class FunctionalColumn implements SelectColumn { @NotNull private final Class destDataType; @NotNull - private final BiFunction function; - @NotNull + private final RowKeyAndValueFunction function; + @Nullable private final Class componentType; private ColumnSource sourceColumnSource; - public FunctionalColumn(@NotNull String sourceName, + @FunctionalInterface + public interface RowKeyAndValueFunction { + D apply(long rowKey, S value); + } + + public FunctionalColumn( + @NotNull String sourceName, @NotNull Class sourceDataType, @NotNull String destName, @NotNull Class destDataType, @@ -51,34 +56,37 @@ public FunctionalColumn(@NotNull String sourceName, this(sourceName, sourceDataType, destName, destDataType, (l, v) -> function.apply(v)); } - public FunctionalColumn(@NotNull String sourceName, + public FunctionalColumn( + @NotNull String sourceName, @NotNull Class sourceDataType, @NotNull String destName, @NotNull Class destDataType, - @NotNull Class componentType, + @Nullable Class componentType, @NotNull Function function) { this(sourceName, sourceDataType, destName, destDataType, componentType, (l, v) -> function.apply(v)); } - public FunctionalColumn(@NotNull String sourceName, + public FunctionalColumn( + @NotNull String sourceName, @NotNull Class sourceDataType, @NotNull String destName, @NotNull Class destDataType, - @NotNull BiFunction function) { - this(sourceName, sourceDataType, destName, destDataType, Object.class, function); + @NotNull RowKeyAndValueFunction function) { + this(sourceName, sourceDataType, destName, destDataType, null, function); } - public FunctionalColumn(@NotNull String sourceName, + public FunctionalColumn( + @NotNull String sourceName, @NotNull Class sourceDataType, @NotNull String destName, @NotNull Class destDataType, - @NotNull Class componentType, - @NotNull BiFunction function) { + @Nullable Class componentType, + @NotNull RowKeyAndValueFunction function) { this.sourceName = NameValidator.validateColumnName(sourceName); this.sourceDataType = Require.neqNull(sourceDataType, "sourceDataType"); this.destName = NameValidator.validateColumnName(destName); this.destDataType = Require.neqNull(destDataType, "destDataType"); - this.componentType = Require.neqNull(componentType, "componentType"); + this.componentType = componentType; this.function = function; Require.gtZero(destName.length(), "destName.length()"); } @@ -127,26 +135,27 @@ public Class getReturnedType() { @Override public List getColumns() { - return Collections.singletonList(sourceName); + return List.of(sourceName); } @Override public List getColumnArrays() { - return Collections.emptyList(); + return List.of(); } @NotNull @Override public ColumnSource getDataView() { return new ViewColumnSource<>(destDataType, componentType, new Formula(null) { + @Override - public Object getPrev(long rowKey) { - return function.apply(rowKey, sourceColumnSource.getPrev(rowKey)); + public Object get(final long rowKey) { + return function.apply(rowKey, sourceColumnSource.get(rowKey)); } @Override - public Object get(long rowKey) { - return function.apply(rowKey, sourceColumnSource.get(rowKey)); + public Object getPrev(final long rowKey) { + return function.apply(rowKey, sourceColumnSource.getPrev(rowKey)); } @Override @@ -155,13 +164,13 @@ public ChunkType getChunkType() { } @Override - public FillContext makeFillContext(int chunkCapacity) { - // Not sure this is right. + public FillContext makeFillContext(final int chunkCapacity) { return new FunctionalColumnFillContext(getChunkType()); } @Override - public void fillChunk(@NotNull FillContext fillContext, + public void fillChunk( + @NotNull final FillContext fillContext, @NotNull final WritableChunk destination, @NotNull final RowSequence rowSequence) { final FunctionalColumnFillContext ctx = (FunctionalColumnFillContext) fillContext; @@ -169,7 +178,8 @@ public void fillChunk(@NotNull FillContext fillContext, } @Override - public void fillPrevChunk(@NotNull FillContext fillContext, + public void fillPrevChunk( + @NotNull final FillContext fillContext, @NotNull final WritableChunk destination, @NotNull final RowSequence rowSequence) { final FunctionalColumnFillContext ctx = (FunctionalColumnFillContext) fillContext; @@ -179,9 +189,10 @@ public void fillPrevChunk(@NotNull FillContext fillContext, } private static class FunctionalColumnFillContext implements Formula.FillContext { - final ChunkFiller chunkFiller; - FunctionalColumnFillContext(final ChunkType chunkType) { + private final ChunkFiller chunkFiller; + + private FunctionalColumnFillContext(final ChunkType chunkType) { chunkFiller = ChunkFiller.forChunkType(chunkType); } } @@ -189,7 +200,6 @@ private static class FunctionalColumnFillContext implements Formula.FillContext @NotNull @Override public ColumnSource getLazyView() { - // TODO: memoize return getDataView(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FunctionalColumnLong.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FunctionalColumnLong.java new file mode 100644 index 00000000000..ca72e42b00c --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/FunctionalColumnLong.java @@ -0,0 +1,223 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.select; + +import io.deephaven.api.util.NameValidator; +import io.deephaven.base.verify.Require; +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.TrackingRowSet; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.impl.MatchPair; +import io.deephaven.engine.table.impl.NoSuchColumnException; +import io.deephaven.engine.table.impl.chunkfillers.ChunkFiller; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.SparseArrayColumnSource; +import io.deephaven.engine.table.impl.sources.ViewColumnSource; +import io.deephaven.util.type.TypeUtils; +import org.jetbrains.annotations.NotNull; + +import java.util.List; +import java.util.Map; +import java.util.function.ToLongFunction; + +public class FunctionalColumnLong implements SelectColumn { + + @NotNull + private final String sourceName; + @NotNull + private final Class sourceDataType; + @NotNull + private final String destName; + @NotNull + private final RowKeyAndValueFunction function; + + private ColumnSource sourceColumnSource; + + @FunctionalInterface + public interface RowKeyAndValueFunction { + long applyAsLong(long rowKey, S value); + } + + public FunctionalColumnLong( + @NotNull String sourceName, + @NotNull Class sourceDataType, + @NotNull String destName, + @NotNull ToLongFunction function) { + this(sourceName, sourceDataType, destName, (l, v) -> function.applyAsLong(v)); + } + + public FunctionalColumnLong( + @NotNull String sourceName, + @NotNull Class sourceDataType, + @NotNull String destName, + @NotNull RowKeyAndValueFunction function) { + this.sourceName = NameValidator.validateColumnName(sourceName); + this.sourceDataType = Require.neqNull(sourceDataType, "sourceDataType"); + this.destName = NameValidator.validateColumnName(destName); + this.function = function; + Require.gtZero(destName.length(), "destName.length()"); + } + + @Override + public String toString() { + return "function(" + sourceName + ',' + destName + ')'; + } + + @Override + public List initInputs(TrackingRowSet rowSet, Map> columnsOfInterest) { + // noinspection unchecked + final ColumnSource localSourceColumnSource = (ColumnSource) columnsOfInterest.get(sourceName); + if (localSourceColumnSource == null) { + throw new NoSuchColumnException(columnsOfInterest.keySet(), sourceName); + } + if (!(sourceDataType.isAssignableFrom(localSourceColumnSource.getType()) || sourceDataType + .isAssignableFrom(TypeUtils.getBoxedType(localSourceColumnSource.getType())))) { + throw new IllegalArgumentException("Source column " + sourceName + " has wrong data type " + + localSourceColumnSource.getType() + ", expected " + sourceDataType); + } + // noinspection unchecked + sourceColumnSource = (ColumnSource) columnsOfInterest.get(sourceName); + return getColumns(); + } + + @Override + public List initDef(Map> columnDefinitionMap) { + // noinspection unchecked + final ColumnDefinition sourceColumnDefinition = (ColumnDefinition) columnDefinitionMap.get(sourceName); + if (sourceColumnDefinition == null) { + throw new NoSuchColumnException(columnDefinitionMap.keySet(), sourceName); + } + if (!(sourceDataType.isAssignableFrom(sourceColumnDefinition.getDataType()) + || sourceDataType.isAssignableFrom(TypeUtils.getBoxedType(sourceColumnDefinition.getDataType())))) { + throw new IllegalArgumentException("Source column " + sourceName + " has wrong data type " + + sourceColumnDefinition.getDataType() + ", expected " + sourceDataType); + } + return getColumns(); + } + + @Override + public Class getReturnedType() { + return long.class; + } + + @Override + public List getColumns() { + return List.of(sourceName); + } + + @Override + public List getColumnArrays() { + return List.of(); + } + + @NotNull + @Override + public ColumnSource getDataView() { + return new ViewColumnSource<>(long.class, new Formula(null) { + + @Override + public long getLong(final long rowKey) { + return function.applyAsLong(rowKey, sourceColumnSource.get(rowKey)); + } + + @Override + public long getPrevLong(final long rowKey) { + return function.applyAsLong(rowKey, sourceColumnSource.getPrev(rowKey)); + } + + @Override + public Object get(final long rowKey) { + return TypeUtils.box(getLong(rowKey)); + } + + @Override + public Object getPrev(final long rowKey) { + return TypeUtils.box(getPrevLong(rowKey)); + } + + @Override + public ChunkType getChunkType() { + return ChunkType.Long; + } + + @Override + public FillContext makeFillContext(final int chunkCapacity) { + return new FunctionalColumnFillContext(); + } + + @Override + public void fillChunk( + @NotNull final FillContext fillContext, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + final FunctionalColumnFillContext ctx = (FunctionalColumnFillContext) fillContext; + ctx.chunkFiller.fillByIndices(this, rowSequence, destination); + } + + @Override + public void fillPrevChunk( + @NotNull final FillContext fillContext, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + final FunctionalColumnFillContext ctx = (FunctionalColumnFillContext) fillContext; + ctx.chunkFiller.fillPrevByIndices(this, rowSequence, destination); + } + }, false); + } + + private static class FunctionalColumnFillContext implements Formula.FillContext { + + private final ChunkFiller chunkFiller; + + private FunctionalColumnFillContext() { + chunkFiller = ChunkFiller.forChunkType(ChunkType.Long); + } + } + + @NotNull + @Override + public ColumnSource getLazyView() { + return getDataView(); + } + + @Override + public String getName() { + return destName; + } + + @Override + public MatchPair getMatchPair() { + throw new UnsupportedOperationException(); + } + + @Override + public final WritableColumnSource newDestInstance(final long size) { + return SparseArrayColumnSource.getSparseMemoryColumnSource(size, long.class); + } + + @Override + public final WritableColumnSource newFlatDestInstance(final long size) { + return InMemoryColumnSource.getImmutableMemoryColumnSource(size, long.class, null); + } + + @Override + public boolean isRetain() { + return false; + } + + @Override + public boolean isStateless() { + return false; + } + + @Override + public FunctionalColumnLong copy() { + return new FunctionalColumnLong<>(sourceName, sourceDataType, destName, function); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java index 0cd410d13d8..421ac67feeb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java @@ -6,25 +6,33 @@ import io.deephaven.api.literal.Literal; import io.deephaven.base.string.cache.CompressedString; import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.context.QueryScope; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.DataIndex; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.DependencyStreamProvider; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.preview.DisplayWrapper; -import io.deephaven.engine.context.QueryScope; +import io.deephaven.engine.updategraph.NotificationQueue; import io.deephaven.time.DateTimeUtils; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.type.ArrayTypeUtils; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.rowset.RowSet; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import org.jpy.PyObject; import java.math.BigDecimal; import java.math.BigInteger; import java.time.Instant; import java.util.*; +import java.util.stream.Stream; -public class MatchFilter extends WhereFilterImpl { +public class MatchFilter extends WhereFilterImpl implements DependencyStreamProvider { private static final long serialVersionUID = 1L; @@ -40,11 +48,24 @@ static MatchFilter ofLiterals( @NotNull private final String columnName; - private Object[] values; // TODO: Does values need to be declared volatile (if we go back to the double-check)? + private Object[] values; private final String[] strValues; private final boolean invertMatch; private final boolean caseInsensitive; - private boolean initialized = false; + + private boolean initialized; + + /** + * The {@link DataIndex} for this filter, if any. Only ever non-{@code null} during operation initialization. + */ + private DataIndex dataIndex; + /** + * Whether our dependencies have been gathered at least once. We expect dependencies to be gathered one time after + * {@link #beginOperation(Table)} (when we know if we're using a {@link DataIndex}), and then again after for every + * instantiation attempt when initializing the listener. Since we only use the DataIndex during instantiation, we + * don't need the listener to depend on it. + */ + private boolean initialDependenciesGathered; public enum MatchType { Regular, Inverted, @@ -54,27 +75,45 @@ public enum CaseSensitivity { MatchCase, IgnoreCase } - public MatchFilter(MatchType matchType, String columnName, Object... values) { - this.columnName = columnName; - this.values = values; - this.strValues = null; - this.invertMatch = (matchType == MatchType.Inverted); - this.caseInsensitive = false; + public MatchFilter( + @NotNull final MatchType matchType, + @NotNull final String columnName, + @NotNull final Object... values) { + this(CaseSensitivity.MatchCase, matchType, columnName, null, values); } - public MatchFilter(String columnName, Object... values) { - this(MatchType.Regular, columnName, values); + public MatchFilter( + @NotNull final String columnName, + @NotNull final Object... values) { + this(CaseSensitivity.IgnoreCase, MatchType.Regular, columnName, null, values); } - public MatchFilter(CaseSensitivity sensitivity, String columnName, String... strValues) { - this(sensitivity, MatchType.Regular, columnName, strValues); + public MatchFilter( + @NotNull final CaseSensitivity sensitivity, + @NotNull final String columnName, + @NotNull final String... strValues) { + this(sensitivity, MatchType.Regular, columnName, strValues, null); } - public MatchFilter(CaseSensitivity sensitivity, MatchType matchType, String columnName, String... strValues) { + public MatchFilter( + @NotNull final CaseSensitivity sensitivity, + @NotNull final MatchType matchType, + @NotNull final String columnName, + @NotNull final String... strValues) { + this(sensitivity, matchType, columnName, strValues, null); + } + + private MatchFilter( + @NotNull final CaseSensitivity sensitivity, + @NotNull final MatchType matchType, + @NotNull final String columnName, + @Nullable String[] strValues, + @Nullable final Object[] values) { + this.caseInsensitive = sensitivity == CaseSensitivity.IgnoreCase; + this.invertMatch = (matchType == MatchType.Inverted); this.columnName = columnName; this.strValues = strValues; - this.caseInsensitive = (sensitivity == CaseSensitivity.IgnoreCase); - this.invertMatch = (matchType == MatchType.Inverted); + this.values = values; } public MatchFilter renameFilter(String newName) { @@ -164,12 +203,46 @@ public synchronized void init(TableDefinition tableDefinition) { initialized = true; } + @Override + public SafeCloseable beginOperation(@NotNull final Table sourceTable) { + if (initialDependenciesGathered || dataIndex != null) { + throw new IllegalStateException("Inputs already initialized, use copy() instead of re-using a WhereFilter"); + } + try (final SafeCloseable ignored = sourceTable.isRefreshing() ? LivenessScopeStack.open() : null) { + dataIndex = DataIndexer.getDataIndex(sourceTable, columnName); + if (dataIndex != null && dataIndex.isRefreshing()) { + dataIndex.retainReference(); + } + } + return dataIndex != null ? this::completeOperation : () -> { + }; + } + + private void completeOperation() { + if (dataIndex.isRefreshing()) { + dataIndex.dropReference(); + } + dataIndex = null; + } + + @Override + public Stream getDependencyStream() { + if (initialDependenciesGathered) { + return Stream.empty(); + } + initialDependenciesGathered = true; + if (dataIndex == null || !dataIndex.isRefreshing()) { + return Stream.empty(); + } + return Stream.of(dataIndex.table()); + } + @NotNull @Override public WritableRowSet filter( @NotNull RowSet selection, @NotNull RowSet fullSet, @NotNull Table table, boolean usePrev) { final ColumnSource columnSource = table.getColumnSource(columnName); - return columnSource.match(invertMatch, usePrev, caseInsensitive, selection, values); + return columnSource.match(invertMatch, usePrev, caseInsensitive, dataIndex, selection, values); } @NotNull @@ -177,7 +250,7 @@ public WritableRowSet filter( public WritableRowSet filterInverse( @NotNull RowSet selection, @NotNull RowSet fullSet, @NotNull Table table, boolean usePrev) { final ColumnSource columnSource = table.getColumnSource(columnName); - return columnSource.match(!invertMatch, usePrev, caseInsensitive, selection, values); + return columnSource.match(!invertMatch, usePrev, caseInsensitive, dataIndex, selection, values); } @Override @@ -398,6 +471,7 @@ Object convertStringLiteral(String str) { return new ColumnTypeConvertor() { @Override Object convertStringLiteral(String str) { + // noinspection unchecked,rawtypes return Enum.valueOf((Class) cls, str); } }; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MultiSourceFunctionalColumn.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MultiSourceFunctionalColumn.java index 9f1cb344cc9..e96fbf758e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MultiSourceFunctionalColumn.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MultiSourceFunctionalColumn.java @@ -19,15 +19,14 @@ import io.deephaven.engine.table.impl.chunkfillers.ChunkFiller; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.TrackingRowSet; -import org.apache.commons.lang3.mutable.MutableObject; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.*; -import java.util.function.BiFunction; import java.util.stream.Collectors; -// TODO: Comment the heck out of this... public class MultiSourceFunctionalColumn implements SelectColumn { + private final List sourceNames; private ColumnSource[] sourceColumns; private ColumnSource[] prevSources; @@ -37,29 +36,36 @@ public class MultiSourceFunctionalColumn implements SelectColumn { @NotNull private final Class destDataType; @NotNull - private final BiFunction[], D> function; - @NotNull + private final RowKeyAndSourcesFunction function; + @Nullable private final Class componentType; - public MultiSourceFunctionalColumn(@NotNull List sourceNames, + @FunctionalInterface + public interface RowKeyAndSourcesFunction { + D apply(long rowKey, ColumnSource[] sources); + } + + public MultiSourceFunctionalColumn( + @NotNull List sourceNames, @NotNull String destName, @NotNull Class destDataType, - @NotNull BiFunction[], D> function) { - this(sourceNames, destName, destDataType, Object.class, function); + @NotNull RowKeyAndSourcesFunction function) { + this(sourceNames, destName, destDataType, null, function); } - public MultiSourceFunctionalColumn(@NotNull List sourceNames, + public MultiSourceFunctionalColumn( + @NotNull List sourceNames, @NotNull String destName, @NotNull Class destDataType, - @NotNull Class componentType, - @NotNull BiFunction[], D> function) { + @Nullable Class componentType, + @NotNull RowKeyAndSourcesFunction function) { this.sourceNames = sourceNames.stream() .map(NameValidator::validateColumnName) .collect(Collectors.toList()); this.destName = NameValidator.validateColumnName(destName); this.destDataType = Require.neqNull(destDataType, "destDataType"); - this.componentType = Require.neqNull(componentType, "componentType"); + this.componentType = componentType; this.function = function; Require.gtZero(destName.length(), "destName.length()"); } @@ -111,23 +117,24 @@ public List getColumns() { @Override public List getColumnArrays() { - return Collections.emptyList(); + return List.of(); } @NotNull @Override public ColumnSource getDataView() { return new ViewColumnSource<>(destDataType, componentType, new Formula(null) { - @Override - public Object getPrev(long rowKey) { - return function.apply(rowKey, prevSources); - } @Override public Object get(long rowKey) { return function.apply(rowKey, sourceColumns); } + @Override + public Object getPrev(long rowKey) { + return function.apply(rowKey, prevSources); + } + @Override public ChunkType getChunkType() { return ChunkType.fromElementType(destDataType); @@ -135,7 +142,6 @@ public ChunkType getChunkType() { @Override public FillContext makeFillContext(int chunkCapacity) { - // Not sure this is right. return new FunctionalColumnFillContext(getChunkType()); } @@ -158,9 +164,10 @@ public void fillPrevChunk(@NotNull FillContext fillContext, } private static class FunctionalColumnFillContext implements Formula.FillContext { - final ChunkFiller chunkFiller; - FunctionalColumnFillContext(final ChunkType chunkType) { + private final ChunkFiller chunkFiller; + + private FunctionalColumnFillContext(final ChunkType chunkType) { chunkFiller = ChunkFiller.forChunkType(chunkType); } } @@ -168,7 +175,6 @@ private static class FunctionalColumnFillContext implements Formula.FillContext @NotNull @Override public ColumnSource getLazyView() { - // TODO: memoize return getDataView(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilter.java index ab40e561ee8..b6486527259 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilter.java @@ -8,11 +8,13 @@ import io.deephaven.engine.context.QueryCompiler; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.table.DataIndex; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.BaseTable; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.remote.ConstructSnapshot; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.FinalDefault; import io.deephaven.util.annotations.InternalUseOnly; import org.jetbrains.annotations.NotNull; @@ -97,7 +99,8 @@ interface RecomputeListener { List getColumnArrays(); /** - * Initialize this select filter given the table definition + * Initialize this filter given the table definition. If this filter has already been initialized, this should be a + * no-op, or optionally validate that the table definition is compatible with previous initialization. * * @param tableDefinition the definition of the table that will be filtered * @apiNote Any {@link io.deephaven.engine.context.QueryLibrary}, {@link io.deephaven.engine.context.QueryScope}, or @@ -105,6 +108,20 @@ interface RecomputeListener { */ void init(TableDefinition tableDefinition); + /** + * Perform any operation-level initialization necessary using the {@link Table} that will be filtered with this + * WhereFilter, e.g. gathering {@link DataIndex data indexes}. This method will always be called exactly once, + * before gathering any dependencies or filtering data. + * + * @param sourceTable The {@link Table} that will be filtered with this WhereFilter + * @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 Table sourceTable) { + return () -> { + }; + } + /** * Validate that this {@code WhereFilter} is safe to use in the context of the provided sourceTable. * @@ -131,7 +148,10 @@ default void validateSafeForRefresh(final BaseTable sourceTable) { */ @NotNull WritableRowSet filter( - @NotNull RowSet selection, @NotNull RowSet fullSet, @NotNull Table table, boolean usePrev); + @NotNull RowSet selection, + @NotNull RowSet fullSet, + @NotNull Table table, + boolean usePrev); /** * Filter selection to only non-matching rows. @@ -164,7 +184,10 @@ WritableRowSet filter( */ @NotNull default WritableRowSet filterInverse( - @NotNull RowSet selection, @NotNull RowSet fullSet, @NotNull Table table, boolean usePrev) { + @NotNull RowSet selection, + @NotNull RowSet fullSet, + @NotNull Table table, + boolean usePrev) { try (final WritableRowSet regular = filter(selection, fullSet, table, usePrev)) { return selection.minus(regular); } @@ -216,17 +239,18 @@ default boolean permitParallelization() { } /** - * Set the RecomputeListener that should be notified if results based on this filter must be recomputed. + * Set the {@link RecomputeListener} that should be notified if results based on this WhereFilter must be + * recomputed. * - * @param result the listener to notify. + * @param result The {@link RecomputeListener} to notify */ void setRecomputeListener(RecomputeListener result); /** * The database system may automatically generate a filter, for example, when applying an ACL to a table. There are * certain operations which may bypass these filters. - * - * This function returns whether or not this filter is automated. + *

+ * This function returns whether this filter is automated. * * @return true if this filter was automatically applied by the database system. False otherwise. */ @@ -235,7 +259,7 @@ default boolean permitParallelization() { /** * The database system may automatically generate a filter, for example, when applying an ACL to a table. There are * certain operations which may bypass these filters. - * + *

* This function indicates that this filter is automated. * * @param value true if this filter was automatically applied by the database system. False otherwise. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterInvertedImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterInvertedImpl.java index b4838d0827e..97f4e46c1dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterInvertedImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterInvertedImpl.java @@ -4,7 +4,6 @@ package io.deephaven.engine.table.impl.select; import io.deephaven.engine.liveness.LivenessArtifact; -import io.deephaven.engine.liveness.LivenessReferent; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.table.Table; @@ -12,6 +11,7 @@ import io.deephaven.engine.table.impl.BaseTable; import io.deephaven.engine.table.impl.DependencyStreamProvider; import io.deephaven.engine.updategraph.NotificationQueue; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.VisibleForTesting; import org.jetbrains.annotations.NotNull; @@ -61,6 +61,11 @@ public void init(TableDefinition tableDefinition) { filter.init(tableDefinition); } + @Override + public SafeCloseable beginOperation(@NotNull final Table sourceTable) { + return filter.beginOperation(sourceTable); + } + @Override public void validateSafeForRefresh(BaseTable sourceTable) { filter.validateSafeForRefresh(sourceTable); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java index 44869a7f24b..971d4a8e2c9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.java @@ -62,7 +62,7 @@ final public class SelectColumnLayer extends SelectOrViewColumnLayer { private final boolean canParallelizeThisColumn; private final boolean isSystemic; private final boolean resultTypeIsLivenessReferent; - private final boolean resultTypeIsTable; + private final boolean resultTypeIsTableOrRowSet; private UpdateCommitterEx prevUnmanager; private List> prevValueChunksToUnmanage; @@ -109,12 +109,13 @@ final public class SelectColumnLayer extends SelectOrViewColumnLayer { // We want to ensure that results are managed appropriately if they are LivenessReferents resultTypeIsLivenessReferent = LivenessReferent.class.isAssignableFrom(ws.getType()); - // We assume that formulas producing Tables are likely to + // We assume that formulas producing Tables or RowSets are likely to // 1. be expensive to evaluate and // 2. have wildly varying job size, // and so we ignore minimum size to parallelize and limit divisionSize to 1 to maximize the // effect of our parallelism. - resultTypeIsTable = Table.class.isAssignableFrom(ws.getType()); + resultTypeIsTableOrRowSet = Table.class.isAssignableFrom(ws.getType()) + || RowSet.class.isAssignableFrom(ws.getType()); } private ChunkSource getChunkSource() { @@ -163,9 +164,9 @@ public void onAllRequiredColumnsCompleted() { || updateGraph.exclusiveLock().isHeldByCurrentThread(); if (canParallelizeThisColumn && jobScheduler.threadCount() > 1 && !hasShifts && - ((resultTypeIsTable && totalSize > 0) + ((resultTypeIsTableOrRowSet && totalSize > 0) || totalSize > QueryTable.MINIMUM_PARALLEL_SELECT_ROWS)) { - final long divisionSize = resultTypeIsTable ? 1 + final long divisionSize = resultTypeIsTableOrRowSet ? 1 : Math.max(QueryTable.MINIMUM_PARALLEL_SELECT_ROWS, (totalSize + jobScheduler.threadCount() - 1) / jobScheduler.threadCount()); final List updates = new ArrayList<>(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sort/LongMegaMergeKernel.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sort/LongMegaMergeKernel.java index 46a079bf084..6d432005d12 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sort/LongMegaMergeKernel.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sort/LongMegaMergeKernel.java @@ -5,7 +5,6 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.attributes.Indices; -import io.deephaven.engine.table.Context; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.SortingOrder; import io.deephaven.engine.table.impl.sort.megamerge.*; @@ -18,7 +17,7 @@ * The intention is that you will use a Timsort kernel to sort maximally sized chunks, then merge the result into a * ColumnSource using the MegaMergeKernel only if necessary. */ -public interface LongMegaMergeKernel extends Context { +public interface LongMegaMergeKernel { /** * Creates a SortKernel for the given chunkType. * diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/AbstractDeferredGroupingColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/AbstractDeferredGroupingColumnSource.java deleted file mode 100644 index f926d87edde..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/AbstractDeferredGroupingColumnSource.java +++ /dev/null @@ -1,69 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.engine.table.impl.sources; - -import io.deephaven.base.Pair; -import io.deephaven.engine.table.impl.AbstractColumnSource; -import io.deephaven.engine.table.impl.locations.GroupingProvider; -import io.deephaven.engine.rowset.RowSet; -import org.jetbrains.annotations.Nullable; - -import java.util.Map; - -/** - * Adds deferred grouping support to {@link AbstractColumnSource}. - */ -public abstract class AbstractDeferredGroupingColumnSource extends AbstractColumnSource - implements DeferredGroupingColumnSource { - - private transient volatile GroupingProvider groupingProvider; - - protected AbstractDeferredGroupingColumnSource(Class type) { - super(type, null); - } - - protected AbstractDeferredGroupingColumnSource(Class type, Class componentType) { - super(type, componentType); - } - - @Override - public GroupingProvider getGroupingProvider() { - return groupingProvider; - } - - /** - * Set a grouping provider for use in lazily-constructing groupings. - * - * @param groupingProvider The {@link GroupingProvider} to use - */ - @Override - public final void setGroupingProvider(@Nullable GroupingProvider groupingProvider) { - this.groupingProvider = groupingProvider; - } - - @Override - public final Map getGroupToRange() { - if (groupToRange == null && groupingProvider != null) { - groupToRange = groupingProvider.getGroupToRange(); - groupingProvider = null; - } - return groupToRange; - } - - @Override - public final Map getGroupToRange(RowSet rowSet) { - if (groupToRange == null && groupingProvider != null) { - Pair, Boolean> result = groupingProvider.getGroupToRange(rowSet); - if (result == null) { - return null; - } - if (result.second) { - groupToRange = result.first; - groupingProvider = null; - } - return result.first; - } - return groupToRange; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArrayBackedColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArrayBackedColumnSource.java index afccc52cb1f..0d5fda89395 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArrayBackedColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ArrayBackedColumnSource.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.sources; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -45,7 +46,7 @@ * that the column source can be incrementally expanded without copying data from one array to another. */ public abstract class ArrayBackedColumnSource - extends AbstractDeferredGroupingColumnSource + extends AbstractColumnSource implements FillUnordered, ShiftData.ShiftCallback, WritableColumnSource, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DeferredGroupingColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DeferredGroupingColumnSource.java deleted file mode 100644 index 52a8f93e442..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DeferredGroupingColumnSource.java +++ /dev/null @@ -1,37 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.engine.table.impl.sources; - -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.locations.GroupingProvider; -import io.deephaven.engine.rowset.RowSet; -import org.jetbrains.annotations.Nullable; - -import java.util.Map; - -/** - * A column source that optionally makes available a provider for grouping metadata. - */ -public interface DeferredGroupingColumnSource extends ColumnSource { - - /** - * Set the map returned by getGroupToRange(). - * - * @param groupToRange The map - */ - void setGroupToRange(@Nullable Map groupToRange); - - /** - * @return A provider previously set by - * {@link DeferredGroupingColumnSource#setGroupingProvider(io.deephaven.engine.table.impl.locations.GroupingProvider)} - */ - GroupingProvider getGroupingProvider(); - - /** - * Supply a provider that will lazily construct the group-to-range map. - * - * @param groupingProvider The provider - */ - void setGroupingProvider(@Nullable GroupingProvider groupingProvider); -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DelegatingColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DelegatingColumnSource.java index 699d52b5a3b..0db6f47e37a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DelegatingColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DelegatingColumnSource.java @@ -10,10 +10,11 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.DataIndex; import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.impl.AbstractColumnSource; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import javax.annotation.OverridingMethodsMustInvokeSuper; @@ -27,8 +28,13 @@ public DelegatingColumnSource(Class type, Class componentType, ColumnSourc @Override public WritableRowSet match( - boolean invertMatch, boolean usePrev, boolean caseInsensitive, @NotNull RowSet mapper, Object... keys) { - return delegate.match(invertMatch, usePrev, caseInsensitive, mapper, keys); + boolean invertMatch, + boolean usePrev, + boolean caseInsensitive, + @Nullable final DataIndex dataIndex, + @NotNull RowSet mapper, + Object... keys) { + return delegate.match(invertMatch, usePrev, caseInsensitive, dataIndex, mapper, keys); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NanosBasedTimeArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NanosBasedTimeArraySource.java index 94ca377919d..0d91720ef27 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NanosBasedTimeArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NanosBasedTimeArraySource.java @@ -15,12 +15,13 @@ import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.util.ShiftData; import org.jetbrains.annotations.NotNull; import java.time.*; -public abstract class NanosBasedTimeArraySource extends AbstractDeferredGroupingColumnSource +public abstract class NanosBasedTimeArraySource extends AbstractColumnSource implements FillUnordered, ShiftData.ShiftCallback, WritableColumnSource, InMemoryColumnSource, WritableSourceWithPrepareForParallelPopulation, ConvertibleTimeSource { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NanosBasedTimeSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NanosBasedTimeSparseArraySource.java index 38927d583d2..02abc54a547 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NanosBasedTimeSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NanosBasedTimeSparseArraySource.java @@ -15,6 +15,7 @@ import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.util.ShiftData; import org.jetbrains.annotations.NotNull; @@ -27,7 +28,7 @@ /** * Array-backed ColumnSource for TIME_TYPEs. Allows reinterpret as long. */ -public abstract class NanosBasedTimeSparseArraySource extends AbstractDeferredGroupingColumnSource +public abstract class NanosBasedTimeSparseArraySource extends AbstractColumnSource implements FillUnordered, WritableColumnSource, InMemoryColumnSource, PossiblyImmutableColumnSource, WritableSourceWithPrepareForParallelPopulation, ShiftData.RowSetShiftCallback, ConvertibleTimeSource { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.java index 3f4a3d33954..f8628fda503 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.util.RowRedirection; import io.deephaven.util.BooleanUtils; import io.deephaven.engine.table.impl.join.dupexpand.DupExpandKernel; @@ -40,7 +41,7 @@ * * @param */ -public class RedirectedColumnSource extends AbstractDeferredGroupingColumnSource +public class RedirectedColumnSource extends AbstractColumnSource implements UngroupableColumnSource, ConvertibleTimeSource { /** * Redirect the innerSource if it is not agnostic to redirection. Otherwise, return the innerSource. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ReinterpretUtils.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ReinterpretUtils.java index c58f53c3e1e..baf4e22309c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ReinterpretUtils.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ReinterpretUtils.java @@ -197,6 +197,21 @@ public static ColumnSource maybeConvertToPrimitive(@NotNull final ColumnSourc return source; } + /** + * Convert each source in {@code sources} to a primitive if possible. + * + * @param sources An array of the sources to potentially convert + * @return The primitive sources for each source in {@code sources} + */ + @NotNull + public static ColumnSource[] maybeConvertToPrimitive(@NotNull final ColumnSource[] sources) { + final ColumnSource[] result = new ColumnSource[sources.length]; + for (int ii = 0; ii < sources.length; ++ii) { + result[ii] = maybeConvertToPrimitive(sources[ii]); + } + return result; + } + /** * If {@code source} is something that we prefer to handle as a primitive, do the appropriate conversion. * diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RowSetColumnSourceWrapper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RowSetColumnSourceWrapper.java new file mode 100644 index 00000000000..afcce994eb3 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RowSetColumnSourceWrapper.java @@ -0,0 +1,179 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources; + +import io.deephaven.api.agg.Aggregation; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedObjectChunk; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.TrackingRowSet; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.SharedContext; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.AbstractColumnSource; +import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; +import io.deephaven.engine.table.impl.by.AggregationProcessor; +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * This class wraps a {@link ColumnSource} of {@link RowSet} and returns {@link TrackingRowSet#prev()} when previous + * values are requested and the accessed value is {@link TrackingRowSet tracking}. This should be used when the row set + * objects are mutated instead of replaced during a cycle, e.g. when {@link Table#aggBy(Aggregation)} is used with + * {@link AggregationProcessor#forExposeGroupRowSets()}. + */ +public class RowSetColumnSourceWrapper extends AbstractColumnSource + implements MutableColumnSourceGetDefaults.ForObject { + + public static RowSetColumnSourceWrapper from(@NotNull final ColumnSource source) { + return new RowSetColumnSourceWrapper(source); + } + + private final ColumnSource source; + + private RowSetColumnSourceWrapper(@NotNull final ColumnSource source) { + super(RowSet.class); + this.source = source; + } + + private static class GetContext implements ChunkSource.GetContext { + + private final ChunkSource.GetContext sourceContext; + + private final SizedObjectChunk previousValues; + + private GetContext(@NotNull final ChunkSource.GetContext sourceContext) { + this.sourceContext = sourceContext; + previousValues = new SizedObjectChunk<>(); + } + + private WritableObjectChunk getPreviousValues(final int minCapacity) { + previousValues.ensureCapacity(minCapacity); + return previousValues.get(); + } + + @Override + public void close() { + SafeCloseable.closeAll(sourceContext, previousValues); + } + } + + private static RowSet maybeGetPrevValue(@Nullable final RowSet value) { + return value != null && value.isTracking() ? value.trackingCast().prev() : value; + } + + /** + * Copy values from {@code source} to {@code destination}, re-mapping any {@link TrackingRowSet tracking} values to + * {@link TrackingRowSet#prev()}. + * + * @param source The source chunk + * @param destination The destination chunk + * @apiNote {@code source} and {@code destination} may be the same chunk + */ + private static void maybeCopyPrevValues( + @NotNull final ObjectChunk source, + @NotNull final WritableObjectChunk destination) { + final int size = source.size(); + for (int ii = 0; ii < size; ii++) { + destination.set(ii, maybeGetPrevValue(source.get(ii))); + } + destination.setSize(size); + } + + @Override + public Class getType() { + return RowSet.class; + } + + @Override + public Class getComponentType() { + return null; + } + + @Override + public @Nullable RowSet get(final long rowKey) { + return source.get(rowKey); + } + + @Override + public @Nullable RowSet getPrev(final long rowKey) { + return maybeGetPrevValue(source.getPrev(rowKey)); + } + + @Override + public ChunkSource.GetContext makeGetContext(final int chunkCapacity, final SharedContext sharedContext) { + return new GetContext(source.makeGetContext(chunkCapacity, sharedContext)); + } + + @Override + public FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { + return source.makeFillContext(chunkCapacity, sharedContext); + } + + @Override + public Chunk getChunk( + @NotNull final ColumnSource.GetContext context, + @NotNull final RowSequence rowSequence) { + return source.getChunk(((GetContext) context).sourceContext, rowSequence); + } + + @Override + public Chunk getChunk( + @NotNull final ColumnSource.GetContext context, + final long firstKey, + final long lastKey) { + return source.getChunk(((GetContext) context).sourceContext, firstKey, lastKey); + } + + @Override + public void fillChunk( + @NotNull final FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + source.fillChunk(context, destination, rowSequence); + } + + @Override + public Chunk getPrevChunk( + @NotNull final ChunkSource.GetContext context, + @NotNull final RowSequence rowSequence) { + final GetContext typedContext = (GetContext) context; + final ObjectChunk sourceChunk = + source.getPrevChunk(typedContext.sourceContext, rowSequence).asObjectChunk(); + final WritableObjectChunk destination = + typedContext.getPreviousValues(sourceChunk.size()); + maybeCopyPrevValues(sourceChunk, destination); + return ObjectChunk.downcast(destination); + } + + @Override + public Chunk getPrevChunk( + @NotNull final ColumnSource.GetContext context, + final long firstKey, + final long lastKey) { + final GetContext typedContext = (GetContext) context; + final ObjectChunk sourceChunk = + source.getPrevChunk(typedContext.sourceContext, firstKey, lastKey).asObjectChunk(); + final WritableObjectChunk destination = + typedContext.getPreviousValues(sourceChunk.size()); + maybeCopyPrevValues(sourceChunk, destination); + return ObjectChunk.downcast(destination); + } + + @Override + public void fillPrevChunk( + @NotNull final FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + source.fillPrevChunk(context, destination, rowSequence); + maybeCopyPrevValues(ObjectChunk.downcast(destination.asObjectChunk()), destination.asWritableObjectChunk()); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SparseArrayColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SparseArrayColumnSource.java index 05d895b3eb4..d67858056f9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SparseArrayColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SparseArrayColumnSource.java @@ -7,6 +7,7 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.util.ShiftData; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.engine.rowset.chunkattributes.RowKeys; @@ -71,7 +72,7 @@ *

*/ public abstract class SparseArrayColumnSource - extends AbstractDeferredGroupingColumnSource + extends AbstractColumnSource implements FillUnordered, WritableColumnSource, InMemoryColumnSource, PossiblyImmutableColumnSource, WritableSourceWithPrepareForParallelPopulation, ShiftData.RowSetShiftCallback { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DByteArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DByteArraySource.java index 2395fbf56e9..c4d86e191e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DByteArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DByteArraySource.java @@ -16,6 +16,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -38,7 +39,7 @@ * * If your size is smaller than the maximum array size, prefer {@link ImmutableByteArraySource}. */ -public class Immutable2DByteArraySource extends AbstractDeferredGroupingColumnSource +public class Immutable2DByteArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForByte, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DCharArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DCharArraySource.java index 28e37d6b35f..222cfb286dc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DCharArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DCharArraySource.java @@ -10,6 +10,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -32,7 +33,7 @@ * * If your size is smaller than the maximum array size, prefer {@link ImmutableCharArraySource}. */ -public class Immutable2DCharArraySource extends AbstractDeferredGroupingColumnSource +public class Immutable2DCharArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForChar, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DDoubleArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DDoubleArraySource.java index 9cd96dc0a92..1f712379d2e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DDoubleArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DDoubleArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -36,7 +37,7 @@ * * If your size is smaller than the maximum array size, prefer {@link ImmutableDoubleArraySource}. */ -public class Immutable2DDoubleArraySource extends AbstractDeferredGroupingColumnSource +public class Immutable2DDoubleArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForDouble, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DFloatArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DFloatArraySource.java index 81105e99e20..55de883034a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DFloatArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DFloatArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -36,7 +37,7 @@ * * If your size is smaller than the maximum array size, prefer {@link ImmutableFloatArraySource}. */ -public class Immutable2DFloatArraySource extends AbstractDeferredGroupingColumnSource +public class Immutable2DFloatArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForFloat, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DIntArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DIntArraySource.java index 5b4b8f4fd86..ce4a5d724cb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DIntArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DIntArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -36,7 +37,7 @@ * * If your size is smaller than the maximum array size, prefer {@link ImmutableIntArraySource}. */ -public class Immutable2DIntArraySource extends AbstractDeferredGroupingColumnSource +public class Immutable2DIntArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForInt, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DLongArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DLongArraySource.java index d11fc9a95f1..b34b12e3da6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DLongArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DLongArraySource.java @@ -25,6 +25,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -47,7 +48,7 @@ * * If your size is smaller than the maximum array size, prefer {@link ImmutableLongArraySource}. */ -public class Immutable2DLongArraySource extends AbstractDeferredGroupingColumnSource +public class Immutable2DLongArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForLong, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation , ConvertibleTimeSource { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DNanosBasedTimeArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DNanosBasedTimeArraySource.java index 5226228cfc9..19f7bf59e56 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DNanosBasedTimeArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DNanosBasedTimeArraySource.java @@ -12,6 +12,7 @@ import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.sources.*; import org.jetbrains.annotations.NotNull; @@ -26,7 +27,7 @@ // endregion boxing imports public abstract class Immutable2DNanosBasedTimeArraySource - extends AbstractDeferredGroupingColumnSource + extends AbstractColumnSource implements WritableColumnSource, FillUnordered, InMemoryColumnSource, ConvertibleTimeSource, WritableSourceWithPrepareForParallelPopulation { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DObjectArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DObjectArraySource.java index 805a8a4d238..31b5e1dbffc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DObjectArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DObjectArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -35,7 +36,7 @@ * * If your size is smaller than the maximum array size, prefer {@link ImmutableObjectArraySource}. */ -public class Immutable2DObjectArraySource extends AbstractDeferredGroupingColumnSource +public class Immutable2DObjectArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForObject, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DShortArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DShortArraySource.java index 185e8844415..8fa78220636 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DShortArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/Immutable2DShortArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -36,7 +37,7 @@ * * If your size is smaller than the maximum array size, prefer {@link ImmutableShortArraySource}. */ -public class Immutable2DShortArraySource extends AbstractDeferredGroupingColumnSource +public class Immutable2DShortArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForShort, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableByteArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableByteArraySource.java index 30eaf958906..f541dd5771c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableByteArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableByteArraySource.java @@ -16,6 +16,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -38,7 +39,7 @@ * * If your size is greater than the maximum capacity of an array, prefer {@link Immutable2DByteArraySource}. */ -public class ImmutableByteArraySource extends AbstractDeferredGroupingColumnSource +public class ImmutableByteArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForByte, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableCharArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableCharArraySource.java index a652234a469..6e4408ad025 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableCharArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableCharArraySource.java @@ -10,6 +10,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -32,7 +33,7 @@ * * If your size is greater than the maximum capacity of an array, prefer {@link Immutable2DCharArraySource}. */ -public class ImmutableCharArraySource extends AbstractDeferredGroupingColumnSource +public class ImmutableCharArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForChar, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableDoubleArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableDoubleArraySource.java index 5640e6a55aa..f578e944ab4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableDoubleArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableDoubleArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -36,7 +37,7 @@ * * If your size is greater than the maximum capacity of an array, prefer {@link Immutable2DDoubleArraySource}. */ -public class ImmutableDoubleArraySource extends AbstractDeferredGroupingColumnSource +public class ImmutableDoubleArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForDouble, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableFloatArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableFloatArraySource.java index da02a3a9218..7cc692cbc4c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableFloatArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableFloatArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -36,7 +37,7 @@ * * If your size is greater than the maximum capacity of an array, prefer {@link Immutable2DFloatArraySource}. */ -public class ImmutableFloatArraySource extends AbstractDeferredGroupingColumnSource +public class ImmutableFloatArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForFloat, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableIntArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableIntArraySource.java index 6e89e495ab8..dcfea68ab59 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableIntArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableIntArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -36,7 +37,7 @@ * * If your size is greater than the maximum capacity of an array, prefer {@link Immutable2DIntArraySource}. */ -public class ImmutableIntArraySource extends AbstractDeferredGroupingColumnSource +public class ImmutableIntArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForInt, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableLongArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableLongArraySource.java index 9d42b3a477b..a3688dc4269 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableLongArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableLongArraySource.java @@ -25,6 +25,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -47,7 +48,7 @@ * * If your size is greater than the maximum capacity of an array, prefer {@link Immutable2DLongArraySource}. */ -public class ImmutableLongArraySource extends AbstractDeferredGroupingColumnSource +public class ImmutableLongArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForLong, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation , ConvertibleTimeSource { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableNanosBasedTimeArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableNanosBasedTimeArraySource.java index 3a7f447f34d..d900c59934b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableNanosBasedTimeArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableNanosBasedTimeArraySource.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; @@ -24,7 +25,7 @@ import java.time.ZonedDateTime; public abstract class ImmutableNanosBasedTimeArraySource - extends AbstractDeferredGroupingColumnSource + extends AbstractColumnSource implements WritableColumnSource, FillUnordered, InMemoryColumnSource, ConvertibleTimeSource, WritableSourceWithPrepareForParallelPopulation { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableObjectArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableObjectArraySource.java index 0e592aa40b3..c1cbd19ded6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableObjectArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableObjectArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -35,7 +36,7 @@ * * If your size is greater than the maximum capacity of an array, prefer {@link Immutable2DObjectArraySource}. */ -public class ImmutableObjectArraySource extends AbstractDeferredGroupingColumnSource +public class ImmutableObjectArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForObject, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableShortArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableShortArraySource.java index 13e8799c6c8..18ca7750e80 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableShortArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableShortArraySource.java @@ -14,6 +14,7 @@ import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.*; @@ -36,7 +37,7 @@ * * If your size is greater than the maximum capacity of an array, prefer {@link Immutable2DShortArraySource}. */ -public class ImmutableShortArraySource extends AbstractDeferredGroupingColumnSource +public class ImmutableShortArraySource extends AbstractColumnSource implements ImmutableColumnSourceGetDefaults.ForShort, WritableColumnSource, FillUnordered, InMemoryColumnSource, ChunkedBackingStoreExposedWritableSource, WritableSourceWithPrepareForParallelPopulation /* MIXIN_IMPLS */ { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/MergedDataIndex.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/MergedDataIndex.java new file mode 100644 index 00000000000..d37ca9d5d49 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/MergedDataIndex.java @@ -0,0 +1,239 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources.regioned; + +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.base.verify.Assert; +import io.deephaven.base.verify.Require; +import io.deephaven.engine.primitive.iterator.CloseableIterator; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetBuilderSequential; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.BasicDataIndex; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.PartitionedTableFactory; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.by.AggregationProcessor; +import io.deephaven.engine.table.impl.by.AggregationRowLookup; +import io.deephaven.engine.table.impl.dataindex.AbstractDataIndex; +import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; +import io.deephaven.engine.table.impl.select.FunctionalColumn; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.annotations.InternalUseOnly; +import io.deephaven.vector.ObjectVector; +import org.jetbrains.annotations.NotNull; + +import java.util.*; +import java.util.stream.IntStream; + +/** + * DataIndex that accumulates the individual per-{@link TableLocation} data indexes of a {@link Table} backed by a + * {@link RegionedColumnSourceManager}. + * + * @implNote This implementation is responsible for ensuring that the provided table accounts for the relative positions + * of individual table locations in the provided table of indices. Work to coalesce the index table is + * deferred until the first call to {@link #table()}. Refreshing inputs/indexes are not supported at this time + * due to concurrency limitations (w.r.t. the UpdateGraph) of the underlying table operations used to compute + * the merged index table, as well as a lack of use cases beyond "new static partitions are added to a live + * source table". + */ +@InternalUseOnly +class MergedDataIndex extends AbstractDataIndex { + + private static final String LOCATION_DATA_INDEX_TABLE_COLUMN_NAME = "__DataIndexTable"; + + private final List keyColumnNames; + private final RegionedColumnSourceManager columnSourceManager; + + private final Map, String> keyColumnNamesByIndexedColumn; + + /** + * The lookup function for the index table. Note that this is always set before {@link #indexTable}. + */ + private AggregationRowLookup lookupFunction; + + /** + * The index table. Note that we use this as a barrier to ensure {@link #lookupFunction} is visible. + */ + private volatile Table indexTable; + + /** + * Whether this index is known to be corrupt. + */ + private volatile boolean isCorrupt; + + /** + * Whether this index is valid. {@code null} means we don't know, yet. + */ + private volatile Boolean isValid; + + MergedDataIndex( + @NotNull final String[] keyColumnNames, + @NotNull final ColumnSource[] keySources, + @NotNull final RegionedColumnSourceManager columnSourceManager) { + + Require.eq(keyColumnNames.length, "keyColumnNames.length", keySources.length, "keySources.length"); + + this.keyColumnNames = List.of(keyColumnNames); + this.columnSourceManager = columnSourceManager; + + // Create an in-order reverse lookup map for the key column names + keyColumnNamesByIndexedColumn = Collections.unmodifiableMap(IntStream.range(0, keySources.length).sequential() + .collect(LinkedHashMap::new, (m, i) -> m.put(keySources[i], keyColumnNames[i]), Assert::neverInvoked)); + if (keyColumnNamesByIndexedColumn.size() != keySources.length) { + throw new IllegalArgumentException(String.format("Duplicate key sources found in %s for %s", + Arrays.toString(keySources), Arrays.toString(keyColumnNames))); + } + + if (columnSourceManager.locationTable().isRefreshing()) { + throw new UnsupportedOperationException("Refreshing location tables are not currently supported"); + } + + // Defer the actual index table creation until it is needed + } + + @Override + @NotNull + public List keyColumnNames() { + return keyColumnNames; + } + + @Override + @NotNull + public Map, String> keyColumnNamesByIndexedColumn() { + return keyColumnNamesByIndexedColumn; + } + + @Override + @NotNull + public Table table() { + Table localIndexTable; + if ((localIndexTable = indexTable) != null) { + return localIndexTable; + } + synchronized (this) { + if ((localIndexTable = indexTable) != null) { + return localIndexTable; + } + try { + return QueryPerformanceRecorder.withNugget( + String.format("Merge Data Indexes [%s]", String.join(", ", keyColumnNames)), + this::buildTable); + } catch (Throwable t) { + isCorrupt = true; + throw t; + } + } + } + + private Table buildTable() { + final Table locationTable = columnSourceManager.locationTable().coalesce(); + + // Perform a parallelizable update to produce coalesced location index tables with their row sets shifted by + // the appropriate region offset. + // This potentially loads many small row sets into memory, but it avoids the risk of re-materializing row set + // pages during the accumulation phase. + final String[] keyColumnNamesArray = keyColumnNames.toArray(String[]::new); + final Table locationDataIndexes = locationTable + .update(List.of(new FunctionalColumn<>( + columnSourceManager.locationColumnName(), TableLocation.class, + LOCATION_DATA_INDEX_TABLE_COLUMN_NAME, Table.class, + (final long locationRowKey, final TableLocation location) -> loadIndexTableAndShiftRowSets( + locationRowKey, location, keyColumnNamesArray)))) + .dropColumns(columnSourceManager.locationColumnName()); + + // Merge all the location index tables into a single table + final Table mergedDataIndexes = PartitionedTableFactory.of(locationDataIndexes).merge(); + + // Group the merged data indexes by the keys + final Table groupedByKeyColumns = mergedDataIndexes.groupBy(keyColumnNamesArray); + + // Combine the row sets from each group into a single row set + final Table combined = groupedByKeyColumns + .update(List.of(new FunctionalColumn<>( + ROW_SET_COLUMN_NAME, ObjectVector.class, + ROW_SET_COLUMN_NAME, RowSet.class, + this::mergeRowSets))); + Assert.assertion(combined.isFlat(), "combined.isFlat()"); + Assert.eq(groupedByKeyColumns.size(), "groupedByKeyColumns.size()", combined.size(), "combined.size()"); + + // Cleanup after ourselves + try (final CloseableIterator rowSets = mergedDataIndexes.objectColumnIterator(ROW_SET_COLUMN_NAME)) { + rowSets.forEachRemaining(SafeCloseable::close); + } + + lookupFunction = AggregationProcessor.getRowLookup(groupedByKeyColumns); + indexTable = combined; + return combined; + } + + private static Table loadIndexTableAndShiftRowSets( + final long locationRowKey, + @NotNull final TableLocation location, + @NotNull final String[] keyColumnNames) { + final BasicDataIndex dataIndex = location.getDataIndex(keyColumnNames); + if (dataIndex == null) { + throw new UncheckedDeephavenException(String.format("Failed to load data index [%s] for location %s", + String.join(", ", keyColumnNames), location)); + } + final Table indexTable = dataIndex.table(); + return indexTable.coalesce().update(List.of(new FunctionalColumn<>( + dataIndex.rowSetColumnName(), RowSet.class, + ROW_SET_COLUMN_NAME, RowSet.class, + (final RowSet rowSet) -> rowSet + .shift(RegionedColumnSource.getFirstRowKey(Math.toIntExact(locationRowKey)))))); + } + + private RowSet mergeRowSets( + @SuppressWarnings("unused") final long unusedRowKey, + @NotNull final ObjectVector keyRowSets) { + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + try (final CloseableIterator rowSets = keyRowSets.iterator()) { + rowSets.forEachRemaining(builder::appendRowSequence); + } + return builder.build(); + } + + @Override + @NotNull + public RowKeyLookup rowKeyLookup() { + table(); + return (final Object key, final boolean usePrev) -> { + // Pass the object to the aggregation lookup, then return the resulting row position (which is also the row + // key). + final int keyRowPosition = lookupFunction.get(key); + if (keyRowPosition == lookupFunction.noEntryValue()) { + return RowSequence.NULL_ROW_KEY; + } + return keyRowPosition; + }; + } + + @Override + public boolean isRefreshing() { + return false; + } + + @Override + public boolean isValid() { + if (isCorrupt) { + return false; + } + if (isValid != null) { + return isValid; + } + final String[] keyColumnNamesArray = keyColumnNames.toArray(String[]::new); + try (final CloseableIterator locations = + columnSourceManager.locationTable().objectColumnIterator(columnSourceManager.locationColumnName())) { + while (locations.hasNext()) { + if (!locations.next().hasDataIndex(keyColumnNamesArray)) { + return isValid = false; + } + } + } + return isValid = true; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java new file mode 100644 index 00000000000..750caaa1493 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/PartitioningColumnDataIndex.java @@ -0,0 +1,267 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources.regioned; + +import gnu.trove.map.hash.TObjectIntHashMap; +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetBuilderRandom; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.ModifiedColumnSet; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.TableUpdateListener; +import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.impl.BaseTable; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.TableUpdateImpl; +import io.deephaven.engine.table.impl.dataindex.AbstractDataIndex; +import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; +import io.deephaven.engine.table.impl.sources.ObjectArraySource; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.engine.table.impl.sources.RowSetColumnSourceWrapper; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.List; +import java.util.Map; + +/** + * DataIndex over a partitioning column of a {@link Table} backed by a {@link RegionedColumnSourceManager}. + */ +class PartitioningColumnDataIndex extends AbstractDataIndex { + + private static final int KEY_NOT_FOUND = (int) RowSequence.NULL_ROW_KEY; + + private final String keyColumnName; + + private final Map, String> keyColumnNamesByIndexedColumn; + + /** The table containing the index. Consists of a sorted key column and an associated RowSet column. */ + private final QueryTable indexTable; + private final WritableColumnSource indexKeySource; + private final ObjectArraySource indexRowSetSource; + + private final ColumnSource locationTableKeySource; + private final ColumnSource locationTableKeySourceReinterpreted; + private final ColumnSource locationTableRowSetSource; + + + /** Provides fast lookup from keys to positions in the index table **/ + private final TObjectIntHashMap keyPositionMap; + + private final ModifiedColumnSet upstreamLocationModified; + private final ModifiedColumnSet upstreamRowSetModified; + private final ModifiedColumnSet downstreamRowSetModified; + + /** + * Construct a new PartitioningColumnDataIndex. Note that this must be constructed by the + * {@link RegionedColumnSourceManager} at a time when there cannot be any concurrent "refresh" behavior, and so we + * can safely use the {@link RegionedColumnSourceManager#locationTable() location table} without snapshotting or + * considering previous values. + * + * @param keyColumnName The key column name + * @param keySource The key source in the indexed table + * @param columnSourceManager The column source manager that provides locations and region indexes + */ + PartitioningColumnDataIndex( + @NotNull final String keyColumnName, + @NotNull final ColumnSource keySource, + @NotNull final RegionedColumnSourceManager columnSourceManager) { + this.keyColumnName = keyColumnName; + + keyColumnNamesByIndexedColumn = Map.of(keySource, keyColumnName); + + // Build the index table and the position lookup map. + final QueryTable locationTable = (QueryTable) columnSourceManager.locationTable().coalesce(); + indexKeySource = ArrayBackedColumnSource.getMemoryColumnSource( + locationTable.size(), + keySource.getType(), + keySource.getComponentType()); + indexRowSetSource = new ObjectArraySource<>(RowSet.class); + indexTable = new QueryTable(RowSetFactory.empty().toTracking(), Map.of( + keyColumnName, indexKeySource, + ROW_SET_COLUMN_NAME, RowSetColumnSourceWrapper.from(indexRowSetSource))); + + locationTableKeySource = locationTable.getColumnSource(keyColumnName, keySource.getType()); + locationTableKeySourceReinterpreted = ReinterpretUtils.maybeConvertToPrimitive(locationTableKeySource); + locationTableRowSetSource = locationTable.getColumnSource(columnSourceManager.rowSetColumnName(), RowSet.class); + + keyPositionMap = new TObjectIntHashMap<>(locationTable.intSize(), 0.5F, KEY_NOT_FOUND); + + // Create a dummy update for the initial state. + final TableUpdate initialUpdate = new TableUpdateImpl( + locationTable.getRowSet().copy(), + RowSetFactory.empty(), + RowSetFactory.empty(), + RowSetShiftData.EMPTY, + ModifiedColumnSet.EMPTY); + try { + processUpdate(initialUpdate, true); + } finally { + initialUpdate.release(); + } + + if (locationTable.isRefreshing()) { + // No need to track previous values; we mutate the index table's RowSets in-place, and we never move a key. + indexTable.getRowSet().writableCast().initializePreviousValue(); + upstreamLocationModified = locationTable.newModifiedColumnSet(columnSourceManager.locationColumnName()); + upstreamRowSetModified = locationTable.newModifiedColumnSet(columnSourceManager.rowSetColumnName()); + downstreamRowSetModified = indexTable.newModifiedColumnSet(rowSetColumnName()); + final TableUpdateListener tableListener = new BaseTable.ListenerImpl(String.format( + "Partitioning Column Data Index - %s", keyColumnName), locationTable, indexTable) { + @Override + public void onUpdate(@NotNull final TableUpdate upstream) { + processUpdate(upstream, false); + } + }; + locationTable.addUpdateListener(tableListener); + manage(indexTable); + } else { + upstreamLocationModified = null; + upstreamRowSetModified = null; + downstreamRowSetModified = null; + } + } + + private synchronized void processUpdate( + @NotNull final TableUpdate upstream, + final boolean initializing) { + if (upstream.empty()) { + return; + } + if (upstream.removed().isNonempty()) { + throw new UnsupportedOperationException("Removed locations are not currently supported"); + } + if (upstream.shifted().nonempty()) { + throw new UnsupportedOperationException("Shifted locations are not currently supported"); + } + if (upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(upstreamLocationModified)) { + throw new UnsupportedOperationException("Modified locations are not currently supported"); + } + Assert.assertion(initializing || isRefreshing(), "initializing || isRefreshing()"); + + final int previousSize = keyPositionMap.size(); + final RowSetBuilderRandom modifiedBuilder = initializing ? null : RowSetFactory.builderRandom(); + + if (upstream.added().isNonempty()) { + upstream.added().forAllRowKeys((final long locationRowKey) -> handleKey( + locationRowKey, false, previousSize, modifiedBuilder)); + } + + if (upstream.modified().isNonempty() && upstream.modifiedColumnSet().containsAny(upstreamRowSetModified)) { + Assert.eqFalse(initializing, "initializing"); + upstream.modified().forAllRowKeys((final long locationRowKey) -> handleKey( + locationRowKey, true, previousSize, modifiedBuilder)); + } + + final int newSize = keyPositionMap.size(); + if (previousSize != newSize) { + indexTable.getRowSet().writableCast().insertRange(previousSize, newSize - 1); + } + + if (initializing) { + return; + } + + // Send the downstream updates to any listeners of the index table + final WritableRowSet modified = modifiedBuilder.build(); + if (previousSize == newSize && modified.isEmpty()) { + modified.close(); + return; + } + + final TableUpdate downstream = new TableUpdateImpl( + RowSetFactory.fromRange(previousSize, newSize - 1), + RowSetFactory.empty(), + modified, + RowSetShiftData.EMPTY, + modified.isNonempty() ? downstreamRowSetModified : ModifiedColumnSet.EMPTY); + indexTable.notifyListeners(downstream); + } + + private void handleKey( + final long locationRowKey, + final boolean isModify, + final int previousSize, + @Nullable final RowSetBuilderRandom modifiedBuilder) { + final KEY_TYPE locationKey = locationTableKeySource.get(locationRowKey); + final Object locationKeyReinterpreted = locationTableKeySourceReinterpreted.get(locationRowKey); + final RowSet regionRowSet = locationTableRowSetSource.get(locationRowKey); + if (regionRowSet == null) { + throw new IllegalStateException(String.format("Null row set found at location index %d", locationRowKey)); + } + + final long regionFirstRowKey = RegionedColumnSource.getFirstRowKey(Math.toIntExact(locationRowKey)); + // Test using the (maybe) reinterpreted key + final int pos = keyPositionMap.get(locationKeyReinterpreted); + if (pos == KEY_NOT_FOUND) { + if (isModify) { + throw new IllegalStateException(String.format("Modified partition key %s not found", locationKey)); + } + final int addedKeyPos = keyPositionMap.size(); + // Store the (maybe) reinterpreted key in the lookup hashmap. + keyPositionMap.put(locationKeyReinterpreted, addedKeyPos); + + // Use the original key for the index table output column. + indexKeySource.ensureCapacity(addedKeyPos + 1); + indexKeySource.set(addedKeyPos, locationKey); + + indexRowSetSource.ensureCapacity(addedKeyPos + 1); + indexRowSetSource.set(addedKeyPos, regionRowSet.shift(regionFirstRowKey)); + } else { + // noinspection DataFlowIssue + final WritableRowSet existingRowSet = indexRowSetSource.get(pos).writableCast(); + try (final WritableRowSet shiftedRowSet = regionRowSet.shift(regionFirstRowKey)) { + // We could assert that: + // 1. an added location is non-overlapping with the key's existing row set + // 2. a modified location's current row set is a superset of its previous row set + // 3. a modified location's previous row set is a subset of the key's existing row set + existingRowSet.insert(shiftedRowSet); + } + + if (modifiedBuilder != null && pos < previousSize) { + modifiedBuilder.addKey(pos); + } + } + } + + @Override + @NotNull + public List keyColumnNames() { + return List.of(keyColumnName); + } + + @Override + @NotNull + public Map, String> keyColumnNamesByIndexedColumn() { + return keyColumnNamesByIndexedColumn; + } + + @Override + @NotNull + public Table table() { + return indexTable; + } + + @Override + @NotNull + public RowKeyLookup rowKeyLookup() { + return (final Object key, final boolean usePrev) -> keyPositionMap.get(key); + } + + @Override + public boolean isRefreshing() { + return indexTable.isRefreshing(); + } + + @Override + public boolean isValid() { + return true; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSource.java index 8d9a2e4d0a3..c1b195038c0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSource.java @@ -4,9 +4,9 @@ package io.deephaven.engine.table.impl.sources.regioned; import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.ColumnLocation; -import io.deephaven.engine.table.impl.sources.DeferredGroupingColumnSource; import io.deephaven.engine.table.impl.ImmutableColumnSource; import io.deephaven.engine.rowset.RowSet; import io.deephaven.util.annotations.VisibleForTesting; @@ -39,7 +39,7 @@ */ @VisibleForTesting // This could be package-private, but for mock-based unit testing purposes it must be public public interface RegionedColumnSource - extends DeferredGroupingColumnSource, ImmutableColumnSource { + extends ColumnSource, ImmutableColumnSource { /** * Address bits allocated to the region index. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceBase.java index 98d3bf44595..a72c1b9b55e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceBase.java @@ -5,7 +5,7 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.impl.sources.AbstractDeferredGroupingColumnSource; +import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.util.annotations.TestUseOnly; @@ -16,7 +16,7 @@ * Partial implementation of {@link RegionedColumnSource} for array-backed and delegating implementations to extend. */ abstract class RegionedColumnSourceBase> - extends AbstractDeferredGroupingColumnSource + extends AbstractColumnSource implements RegionedPageStore, RegionedColumnSource { static final Parameters PARAMETERS; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index edcc041a676..a689c63645c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -4,21 +4,28 @@ package io.deephaven.engine.table.impl.sources.regioned; import io.deephaven.base.verify.Assert; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderSequential; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.WritableRowSet; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.impl.ColumnSourceManager; -import io.deephaven.engine.table.impl.ColumnToCodecMappings; -import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.liveness.LivenessArtifact; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.table.impl.locations.ColumnLocation; +import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.TableLocation; import io.deephaven.engine.table.impl.locations.impl.TableLocationUpdateSubscriptionBuffer; -import io.deephaven.engine.table.impl.sources.DeferredGroupingColumnSource; +import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; +import io.deephaven.engine.table.impl.sources.ObjectArraySource; +import io.deephaven.engine.table.impl.util.DelayedErrorNotifier; import io.deephaven.hash.KeyedObjectHashMap; import io.deephaven.hash.KeyedObjectKey; import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.annotations.ReferentialIntegrity; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.util.*; import java.util.stream.Collectors; @@ -27,7 +34,7 @@ /** * Manage column sources made up of regions in their own row key address space. */ -public class RegionedColumnSourceManager implements ColumnSourceManager { +public class RegionedColumnSourceManager extends LivenessArtifact implements ColumnSourceManager { private static final Logger log = LoggerFactory.getLogger(RegionedColumnSourceManager.class); @@ -49,7 +56,7 @@ public class RegionedColumnSourceManager implements ColumnSourceManager { /** * An unmodifiable view of columnSources. */ - private final Map> sharedColumnSources = + private final Map> sharedColumnSources = Collections.unmodifiableMap(columnSources); /** @@ -69,11 +76,36 @@ public class RegionedColumnSourceManager implements ColumnSourceManager { */ private final List orderedIncludedTableLocations = new ArrayList<>(); + private static final String LOCATION_COLUMN_NAME = "__TableLocation"; + private static final ColumnDefinition LOCATION_COLUMN_DEFINITION = + ColumnDefinition.fromGenericType(LOCATION_COLUMN_NAME, TableLocation.class); + private static final String ROWS_SET_COLUMN_NAME = "__RowSet"; + private static final ColumnDefinition ROWS_SET_COLUMN_DEFINITION = + ColumnDefinition.fromGenericType(ROWS_SET_COLUMN_NAME, RowSet.class); + private static final TableDefinition SIMPLE_LOCATION_TABLE_DEFINITION = TableDefinition.of( + LOCATION_COLUMN_DEFINITION, + ROWS_SET_COLUMN_DEFINITION); + + /** + * Non-empty table locations stored in a table. Rows are keyed by {@link IncludedTableLocationEntry#regionIndex + * region index}. + */ + private final QueryTable includedLocationsTable; + private final Map> partitioningColumnValueSources; + private final ObjectArraySource locationSource; + private final ObjectArraySource rowSetSource; + private final ModifiedColumnSet rowSetModifiedColumnSet; + + @SuppressWarnings("MismatchedQueryAndUpdateOfCollection") + @ReferentialIntegrity + private final Collection retainedDataIndexes = new ArrayList<>(); + /** - * Whether grouping is enabled. + * A reference to a delayed error notifier for the {@link #includedLocationsTable}, if one is pending. */ - @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private boolean isGroupingEnabled = true; // We always start out with grouping enabled. + @SuppressWarnings("unused") + @ReferentialIntegrity + private Runnable delayedErrorReference; /** * Construct a column manager with the specified component factory and definitions. @@ -82,10 +114,13 @@ public class RegionedColumnSourceManager implements ColumnSourceManager { * @param componentFactory The component factory * @param columnDefinitions The column definitions */ - RegionedColumnSourceManager(final boolean isRefreshing, + RegionedColumnSourceManager( + final boolean isRefreshing, @NotNull final RegionedTableComponentFactory componentFactory, @NotNull final ColumnToCodecMappings codecMappings, @NotNull final List> columnDefinitions) { + super(false); + this.isRefreshing = isRefreshing; this.columnDefinitions = columnDefinitions; for (final ColumnDefinition columnDefinition : columnDefinitions) { @@ -93,6 +128,43 @@ public class RegionedColumnSourceManager implements ColumnSourceManager { columnDefinition.getName(), componentFactory.createRegionedColumnSource(columnDefinition, codecMappings)); } + + // Create the table that will hold the location data + partitioningColumnValueSources = columnDefinitions.stream() + .filter(ColumnDefinition::isPartitioning) + .collect(Collectors.toMap( + ColumnDefinition::getName, + cd -> ArrayBackedColumnSource.getMemoryColumnSource(cd.getDataType(), cd.getComponentType()))); + locationSource = new ObjectArraySource<>(TableLocation.class); + rowSetSource = new ObjectArraySource<>(RowSet.class); + final LinkedHashMap> columnSourceMap = + new LinkedHashMap<>(partitioningColumnValueSources); + columnSourceMap.put(LOCATION_COLUMN_NAME, locationSource); + columnSourceMap.put(ROWS_SET_COLUMN_NAME, rowSetSource); + final TableDefinition locationTableDefinition = partitioningColumnValueSources.isEmpty() + ? SIMPLE_LOCATION_TABLE_DEFINITION + : TableDefinition.inferFrom(columnSourceMap); + + try (final SafeCloseable ignored = isRefreshing ? LivenessScopeStack.open() : null) { + includedLocationsTable = new QueryTable( + locationTableDefinition, + RowSetFactory.empty().toTracking(), + columnSourceMap, + null, // No need to pre-allocate a MCS + null // No attributes to provide (not add-only or append-only, because locations can grow) + ) { + { + setFlat(); + setRefreshing(isRefreshing); + } + }; + if (isRefreshing) { + rowSetModifiedColumnSet = includedLocationsTable.newModifiedColumnSet(ROWS_SET_COLUMN_NAME); + manage(includedLocationsTable); + } else { + rowSetModifiedColumnSet = null; + } + } } @Override @@ -141,40 +213,159 @@ public boolean removeLocationKey(@NotNull final ImmutableTableLocationKey locati return false; } + @Override + public synchronized TrackingWritableRowSet initialize() { + Assert.assertion(includedLocationsTable.isEmpty(), "includedLocationsTable.isEmpty()"); + + // Do our first pass over the locations to include as many as possible and build the initial row set + // noinspection resource + final TrackingWritableRowSet initialRowSet = update(true).toTracking(); + + // Add single-column data indexes for all partitioning columns, whether refreshing or not + columnDefinitions.stream().filter(ColumnDefinition::isPartitioning).forEach(cd -> { + try (final SafeCloseable ignored = isRefreshing ? LivenessScopeStack.open() : null) { + final DataIndex partitioningIndex = + new PartitioningColumnDataIndex<>(cd.getName(), columnSources.get(cd.getName()), this); + retainedDataIndexes.add(partitioningIndex); + if (isRefreshing) { + manage(partitioningIndex); + } + DataIndexer.of(initialRowSet).addDataIndex(partitioningIndex); + } + }); + + // If we're static, add all data indexes present in the included locations + if (!isRefreshing && initialRowSet.isNonempty()) { + // Use the first location as a proxy for the whole table; since data indexes must be complete over all + // locations, this is a valid approach. + final TableLocation firstLocation = includedTableLocations.iterator().next().location; + for (final String[] keyColumnNames : firstLocation.getDataIndexColumns()) { + // Here, we assume the data index is present on all included locations. MergedDataIndex.validate() will + // be used to test this before attempting to materialize the data index table later on. + final ColumnSource[] keySources = Arrays.stream(keyColumnNames) + .map(columnSources::get) + .toArray(ColumnSource[]::new); + final DataIndex mergedIndex = new MergedDataIndex(keyColumnNames, keySources, this); + retainedDataIndexes.add(mergedIndex); + // Not refreshing, so no need to manage mergedIndex + DataIndexer.of(initialRowSet).addDataIndex(mergedIndex); + } + } + return initialRowSet; + } + @Override public synchronized WritableRowSet refresh() { + if (!isRefreshing) { + throw new UnsupportedOperationException("Cannot refresh a static table"); + } + return update(false); + } + + @Override + public void deliverError(@NotNull final Throwable error, @Nullable final TableListener.Entry entry) { + // Notify any listeners to the locations table that we had an error + final long currentStep = includedLocationsTable.getUpdateGraph().clock().currentStep(); + if (includedLocationsTable.getLastNotificationStep() == currentStep) { + delayedErrorReference = new DelayedErrorNotifier(error, entry, includedLocationsTable); + } else { + includedLocationsTable.notifyListenersOnError(error, entry); + includedLocationsTable.forceReferenceCountToZero(); + } + } + + private WritableRowSet update(final boolean initializing) { final RowSetBuilderSequential addedRowSetBuilder = RowSetFactory.builderSequential(); + + final RowSetBuilderSequential modifiedRegionBuilder = initializing ? null : RowSetFactory.builderSequential(); + // Ordering matters, since we're using a sequential builder. for (final IncludedTableLocationEntry entry : orderedIncludedTableLocations) { - entry.pollUpdates(addedRowSetBuilder); + if (entry.pollUpdates(addedRowSetBuilder)) { + // Changes were detected, update the row set in the table and mark the row/column as modified. + /* + * Since TableLocationState.getRowSet() returns a copy(), we should consider adding an UpdateCommitter + * to close() the previous row sets for modified locations. This is not important for current + * implementations, since they always allocate new, flat RowSets. + */ + rowSetSource.set(entry.regionIndex, entry.location.getRowSet()); + if (modifiedRegionBuilder != null) { + modifiedRegionBuilder.appendKey(entry.regionIndex); + } + } } Collection entriesToInclude = null; for (final Iterator iterator = emptyTableLocations.iterator(); iterator.hasNext();) { - final EmptyTableLocationEntry nonexistentEntry = iterator.next(); - nonexistentEntry.refresh(); - final RowSet locationRowSet = nonexistentEntry.location.getRowSet(); - if (locationRowSet != null) { - if (locationRowSet.isEmpty()) { - locationRowSet.close(); - } else { - nonexistentEntry.initialRowSet = locationRowSet; - (entriesToInclude == null ? entriesToInclude = new TreeSet<>() : entriesToInclude) - .add(nonexistentEntry); - iterator.remove(); - } + final EmptyTableLocationEntry emptyEntry = iterator.next(); + emptyEntry.refresh(); + final RowSet locationRowSet = emptyEntry.location.getRowSet(); + if (locationRowSet == null) { + continue; + } + if (locationRowSet.isEmpty()) { + locationRowSet.close(); + } else { + emptyEntry.initialRowSet = locationRowSet; + (entriesToInclude == null ? entriesToInclude = new TreeSet<>() : entriesToInclude).add(emptyEntry); + iterator.remove(); } } + + final int previousNumRegions = includedTableLocations.size(); + final int newNumRegions = previousNumRegions + (entriesToInclude == null ? 0 : entriesToInclude.size()); if (entriesToInclude != null) { + partitioningColumnValueSources.values().forEach( + (final WritableColumnSource wcs) -> wcs.ensureCapacity(newNumRegions)); + locationSource.ensureCapacity(newNumRegions); + rowSetSource.ensureCapacity(newNumRegions); + for (final EmptyTableLocationEntry entryToInclude : entriesToInclude) { final IncludedTableLocationEntry entry = new IncludedTableLocationEntry(entryToInclude); includedTableLocations.add(entry); orderedIncludedTableLocations.add(entry); entry.processInitial(addedRowSetBuilder, entryToInclude.initialRowSet); + + // We have a new location, add the row set to the table and mark the row as added. + // @formatter:off + // noinspection rawtypes,unchecked + partitioningColumnValueSources.forEach( + (final String key, final WritableColumnSource wcs) -> + wcs.set(entry.regionIndex, entry.location.getKey().getPartitionValue(key))); + // @formatter:on + locationSource.set(entry.regionIndex, entry.location); + rowSetSource.set(entry.regionIndex, entry.location.getRowSet()); } } - if (!isRefreshing) { - emptyTableLocations.clear(); + + if (previousNumRegions != newNumRegions) { + includedLocationsTable.getRowSet().writableCast().insertRange(previousNumRegions, newNumRegions - 1); + } + + if (initializing) { + Assert.eqZero(previousNumRegions, "previousNumRegions"); + if (isRefreshing) { + rowSetSource.startTrackingPrevValues(); + includedLocationsTable.getRowSet().writableCast().initializePreviousValue(); + includedLocationsTable.initializeLastNotificationStep(includedLocationsTable.getUpdateGraph().clock()); + } else { + emptyTableLocations.clear(); + } + } else { + final RowSet modifiedRegions = modifiedRegionBuilder.build(); + if (previousNumRegions == newNumRegions && modifiedRegions.isEmpty()) { + modifiedRegions.close(); + } else { + final TableUpdate update = new TableUpdateImpl( + previousNumRegions == newNumRegions + ? RowSetFactory.empty() + : RowSetFactory.fromRange(previousNumRegions, newNumRegions - 1), + RowSetFactory.empty(), + modifiedRegions, + RowSetShiftData.EMPTY, + modifiedRegions.isNonempty() ? rowSetModifiedColumnSet : ModifiedColumnSet.EMPTY); + includedLocationsTable.notifyListeners(update); + } } return addedRowSetBuilder.build(); } @@ -194,28 +385,28 @@ public final synchronized Collection includedLocations() { } @Override - public final synchronized boolean isEmpty() { - return includedTableLocations.isEmpty(); + public Table locationTable() { + return includedLocationsTable; } @Override - public final Map> getColumnSources() { - return sharedColumnSources; + public String locationColumnName() { + return LOCATION_COLUMN_NAME; } @Override - public final synchronized void disableGrouping() { - if (!isGroupingEnabled) { - return; - } - isGroupingEnabled = false; - for (ColumnDefinition columnDefinition : columnDefinitions) { - if (columnDefinition.isGrouping()) { - DeferredGroupingColumnSource columnSource = getColumnSources().get(columnDefinition.getName()); - columnSource.setGroupingProvider(null); - columnSource.setGroupToRange(null); - } - } + public String rowSetColumnName() { + return ROWS_SET_COLUMN_NAME; + } + + @Override + public final synchronized boolean isEmpty() { + return includedTableLocations.isEmpty(); + } + + @Override + public final Map> getColumnSources() { + return sharedColumnSources; } /** @@ -276,7 +467,7 @@ private class IncludedTableLocationEntry implements Comparable columnLocationStates = new ArrayList<>(); + private final List> columnLocationStates = new ArrayList<>(); /** * RowSet in the region's space, not the table's space. @@ -302,35 +493,26 @@ private void processInitial(final RowSetBuilderSequential addedRowSetBuilder, fi final long regionFirstKey = RegionedColumnSource.getFirstRowKey(regionIndex); initialRowSet.forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> addedRowSetBuilder .appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey)); - RowSet addRowSetInTable = null; - try { - for (final ColumnDefinition columnDefinition : columnDefinitions) { - // noinspection unchecked - final ColumnLocationState state = new ColumnLocationState( - columnDefinition, - columnSources.get(columnDefinition.getName()), - location.getColumnLocation(columnDefinition.getName())); - columnLocationStates.add(state); - state.regionAllocated(regionIndex); - if (state.needToUpdateGrouping()) { - state.updateGrouping( - addRowSetInTable == null ? addRowSetInTable = initialRowSet.shift(regionFirstKey) - : addRowSetInTable); - } - } - } finally { - if (addRowSetInTable != null) { - addRowSetInTable.close(); - } + + for (final ColumnDefinition columnDefinition : columnDefinitions) { + // noinspection unchecked,rawtypes + final ColumnLocationState state = new ColumnLocationState( + columnDefinition, + columnSources.get(columnDefinition.getName()), + location.getColumnLocation(columnDefinition.getName())); + columnLocationStates.add(state); + state.regionAllocated(regionIndex); } + rowSetAtLastUpdate = initialRowSet; } - private void pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) { + /** Returns {@code true} if there were changes to the row set for this location. */ + private boolean pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) { Assert.neqNull(subscriptionBuffer, "subscriptionBuffer"); // Effectively, this is asserting "isRefreshing". try { if (!subscriptionBuffer.processPending()) { - return; + return false; } } catch (Exception ex) { invalidate(); @@ -357,7 +539,7 @@ private void pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) { if (rowSetAtLastUpdate.size() == updateRowSet.size()) { // Nothing to do - return; + return false; } if (updateRowSet.lastRowKey() > RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK) { throw new TableDataException(String.format( @@ -375,26 +557,13 @@ private void pollUpdates(final RowSetBuilderSequential addedRowSetBuilder) { final long regionFirstKey = RegionedColumnSource.getFirstRowKey(regionIndex); addedRowSet.forAllRowKeyRanges((subRegionFirstKey, subRegionLastKey) -> addedRowSetBuilder .appendRange(regionFirstKey + subRegionFirstKey, regionFirstKey + subRegionLastKey)); - RowSet addRowSetInTable = null; - try { - for (final ColumnLocationState state : columnLocationStates) { - if (state.needToUpdateGrouping()) { - state.updateGrouping( - addRowSetInTable == null - ? addRowSetInTable = updateRowSet.shift(regionFirstKey) - : addRowSetInTable); - } - } - } finally { - if (addRowSetInTable != null) { - addRowSetInTable.close(); - } - } } } finally { rowSetAtLastUpdate.close(); rowSetAtLastUpdate = updateRowSet; } + // There was a change to the row set. + return true; } private void invalidate() { @@ -425,7 +594,7 @@ public ImmutableTableLocationKey getKey( /** * Batches up a definition, source, and location for ease of use. Implements grouping maintenance. */ - private class ColumnLocationState { + private static class ColumnLocationState { protected final ColumnDefinition definition; protected final RegionedColumnSource source; @@ -443,44 +612,5 @@ private void regionAllocated(final int regionIndex) { Assert.eq(regionIndex, "regionIndex", source.addRegion(definition, location), "source.addRegion((definition, location)"); } - - private boolean needToUpdateGrouping() { - return (definition.isGrouping() && isGroupingEnabled) || definition.isPartitioning(); - } - - /** - * Update column groupings, if appropriate. - * - * @param locationAddedRowSetInTable The added RowSet, in the table's address space - */ - private void updateGrouping(@NotNull final RowSet locationAddedRowSetInTable) { - if (definition.isGrouping()) { - Assert.eqTrue(isGroupingEnabled, "isGroupingEnabled"); - GroupingProvider groupingProvider = source.getGroupingProvider(); - if (groupingProvider == null) { - groupingProvider = GroupingProvider.makeGroupingProvider(definition); - // noinspection unchecked - source.setGroupingProvider(groupingProvider); - } - if (groupingProvider instanceof KeyRangeGroupingProvider) { - ((KeyRangeGroupingProvider) groupingProvider).addSource(location, locationAddedRowSetInTable); - } - } else if (definition.isPartitioning()) { - final DeferredGroupingColumnSource partitioningColumnSource = source; - Map columnPartitionToRowSet = partitioningColumnSource.getGroupToRange(); - if (columnPartitionToRowSet == null) { - columnPartitionToRowSet = new LinkedHashMap<>(); - partitioningColumnSource.setGroupToRange(columnPartitionToRowSet); - } - final T columnPartitionValue = - location.getTableLocation().getKey().getPartitionValue(definition.getName()); - final RowSet current = columnPartitionToRowSet.get(columnPartitionValue); - if (current == null) { - columnPartitionToRowSet.put(columnPartitionValue, locationAddedRowSetInTable.copy()); - } else { - current.writableCast().insert(locationAddedRowSetInTable); - } - } - } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceWithDictionary.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceWithDictionary.java index bf0364f8efe..9081e790b17 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceWithDictionary.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceWithDictionary.java @@ -3,17 +3,13 @@ // package io.deephaven.engine.table.impl.sources.regioned; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.RowSequenceFactory; -import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.TableUpdateImpl; -import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; import io.deephaven.engine.table.impl.*; -import io.deephaven.engine.table.impl.locations.ColumnLocation; -import io.deephaven.engine.table.impl.ColumnSourceGetDefaults; import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; -import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.impl.locations.ColumnLocation; +import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; import io.deephaven.engine.table.impl.sources.RowKeyColumnSource; import org.apache.commons.lang3.mutable.Mutable; import org.apache.commons.lang3.mutable.MutableObject; @@ -231,6 +227,7 @@ public QueryTable getStaticSymbolTable(@NotNull RowSet sourceIndex, boolean useL RowSequenceFactory.EMPTY_ITERATOR, symbolTableIndexBuilder); } while (keysToVisit.hasNext()); } + // noinspection resource symbolTableRowSet = symbolTableIndexBuilder.build().toTracking(); } @@ -274,7 +271,7 @@ public final Table getSymbolTable(@NotNull final QueryTable sourceTable, final b private final class SymbolTableUpdateListener extends BaseTable.ListenerImpl { - private final BaseTable symbolTable; + private final BaseTable symbolTable; private final ModifiedColumnSet emptyModifiedColumns; private SymbolTableUpdateListener(@NotNull final String description, @NotNull final Table sourceTable, @@ -299,6 +296,7 @@ public void onUpdate(@NotNull final TableUpdate upstream) { } final RowSetBuilderSequential symbolTableAddedBuilder = RowSetFactory.builderSequential(); + // noinspection unchecked final RegionedColumnSourceBase> dictionaryColumn = (RegionedColumnSourceBase>) symbolTable .getColumnSource(SymbolTableSource.SYMBOL_COLUMN_NAME); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ColumnHolder.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ColumnHolder.java index f45bbbd5765..19e87a2cba1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ColumnHolder.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/ColumnHolder.java @@ -31,6 +31,8 @@ * Data to construct a data column. */ public class ColumnHolder { + + @SuppressWarnings("rawtypes") public static final ColumnHolder[] ZERO_LENGTH_COLUMN_HOLDER_ARRAY = new ColumnHolder[0]; /** The name of the column. */ @@ -39,8 +41,12 @@ public class ColumnHolder { public final Class dataType; /** The data's component type of the column. */ public final Class componentType; - /** Should the result column be grouped (used for test data only). */ - public final boolean grouped; + /** + * Should the result column be indexed? This is only supported by test utilities; non-test usages should manually + * add and manage data indexes. Only use this when enclosed by a {@link io.deephaven.engine.liveness.LivenessScope} + * that was constructed with {@code enforceStrongReachability == true}. + */ + public final boolean indexed; private final Object arrayData; private final Chunk chunkData; @@ -54,85 +60,85 @@ public class ColumnHolder { * @param arrayData column data */ @SuppressWarnings("unchecked") - public ColumnHolder(String name, Class dataType, Class componentType, boolean grouped, T... arrayData) { - this(name, grouped, dataType, componentType, arrayData); + public ColumnHolder(String name, Class dataType, Class componentType, boolean indexed, T... arrayData) { + this(name, indexed, dataType, componentType, arrayData); } /** * Construct a new set of column data. * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param arrayData column data */ - public ColumnHolder(String name, boolean grouped, long... arrayData) { - this(name, grouped, long.class, null, arrayData); + public ColumnHolder(String name, boolean indexed, long... arrayData) { + this(name, indexed, long.class, null, arrayData); } /** * Construct a new set of column data. * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param arrayData column data */ - public ColumnHolder(String name, boolean grouped, int... arrayData) { - this(name, grouped, int.class, null, arrayData); + public ColumnHolder(String name, boolean indexed, int... arrayData) { + this(name, indexed, int.class, null, arrayData); } /** * Construct a new set of column data. * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param arrayData column data */ - public ColumnHolder(String name, boolean grouped, short... arrayData) { - this(name, grouped, short.class, null, arrayData); + public ColumnHolder(String name, boolean indexed, short... arrayData) { + this(name, indexed, short.class, null, arrayData); } /** * Construct a new set of column data. * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param arrayData column data */ - public ColumnHolder(String name, boolean grouped, char... arrayData) { - this(name, grouped, char.class, null, arrayData); + public ColumnHolder(String name, boolean indexed, char... arrayData) { + this(name, indexed, char.class, null, arrayData); } /** * Construct a new set of column data. * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param arrayData column data */ - public ColumnHolder(String name, boolean grouped, byte... arrayData) { - this(name, grouped, byte.class, null, arrayData); + public ColumnHolder(String name, boolean indexed, byte... arrayData) { + this(name, indexed, byte.class, null, arrayData); } /** * Construct a new set of column data. * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param arrayData column data */ - public ColumnHolder(String name, boolean grouped, double... arrayData) { - this(name, grouped, double.class, null, arrayData); + public ColumnHolder(String name, boolean indexed, double... arrayData) { + this(name, indexed, double.class, null, arrayData); } /** * Construct a new set of column data. * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param arrayData column data */ - public ColumnHolder(String name, boolean grouped, float... arrayData) { - this(name, grouped, float.class, null, arrayData); + public ColumnHolder(String name, boolean indexed, float... arrayData) { + this(name, indexed, float.class, null, arrayData); } /** @@ -140,12 +146,12 @@ public ColumnHolder(String name, boolean grouped, float... arrayData) { * where the official data type type does not match the data. * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param dataType column data type * @param componentType column component type (for array or {@link ObjectVector >} data types) * @param arrayData column data */ - private ColumnHolder(String name, boolean grouped, Class dataType, Class componentType, Object arrayData) { + private ColumnHolder(String name, boolean indexed, Class dataType, Class componentType, Object arrayData) { if (!arrayData.getClass().isArray()) { throw new IllegalArgumentException("Data must be provided as an array"); } @@ -160,7 +166,7 @@ private ColumnHolder(String name, boolean grouped, Class dataType, Class c // noinspection unchecked this.dataType = (Class) dataType; this.componentType = componentType; - this.grouped = grouped; + this.indexed = indexed; this.arrayData = arrayData; this.chunkData = null; } @@ -171,23 +177,23 @@ private ColumnHolder(String name, boolean grouped, Class dataType, Class c * * @param name column name * @param dataType abstract data type for the column - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param chunkData column data */ protected ColumnHolder(boolean chunkSentinel, String name, Class dataType, Class componentType, - boolean grouped, Chunk chunkData) { + boolean indexed, Chunk chunkData) { // noinspection unchecked this.dataType = (Class) dataType; this.componentType = componentType; this.arrayData = null; this.chunkData = chunkData; this.name = NameValidator.validateColumnName(name); - this.grouped = grouped; + this.indexed = indexed; } - public static ColumnHolder makeForChunk(String name, Class type, Class componentType, boolean grouped, + public static ColumnHolder makeForChunk(String name, Class type, Class componentType, boolean indexed, Chunk chunkData) { - return new ColumnHolder(false, name, type, componentType, grouped, chunkData); + return new ColumnHolder(false, name, type, componentType, indexed, chunkData); } @@ -197,12 +203,12 @@ public static ColumnHolder makeForChunk(String name, Class type, Class * conditions, this will bean InstantArraySource (see {@link #getColumnSource()}). * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param data column data (long integers representing nanos since the epoch) * @returnan Instant column holder implemented with longs for storage */ - public static ColumnHolder getInstantColumnHolder(String name, boolean grouped, long... data) { - return new ColumnHolder<>(name, grouped, Instant.class, null, data); + public static ColumnHolder getInstantColumnHolder(String name, boolean indexed, long... data) { + return new ColumnHolder<>(name, indexed, Instant.class, null, data); } /** @@ -211,24 +217,24 @@ public static ColumnHolder getInstantColumnHolder(String name, boolean * conditions, this will bean InstantArraySource (see {@link #getColumnSource()}). * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param chunkData column data (long integers representing nanos since the epoch) * @returnan Instant column holder implemented with longs for storage */ - public static ColumnHolder getInstantColumnHolder(String name, boolean grouped, Chunk chunkData) { - return new ColumnHolder<>(true, name, Instant.class, null, grouped, chunkData); + public static ColumnHolder getInstantColumnHolder(String name, boolean indexed, Chunk chunkData) { + return new ColumnHolder<>(true, name, Instant.class, null, indexed, chunkData); } /** - * Create a column holder for a Boolean column where the calues are represented as bytes. The given byte array will + * Create a column holder for a Boolean column where the values are represented as bytes. The given byte array will * be converted to a Boolean array. * * @param name column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param data column data (byte values where 1 represents true, 0 represents false, and null otherwise) * @return a Boolean column holder */ - public static ColumnHolder getBooleanColumnHolder(String name, boolean grouped, byte... data) { + public static ColumnHolder getBooleanColumnHolder(String name, boolean indexed, byte... data) { final Boolean[] dbData = new Boolean[data.length]; for (int i = 0; i < data.length; i++) { if (data[i] == (byte) 0) { @@ -239,20 +245,20 @@ public static ColumnHolder getBooleanColumnHolder(String name, boolean dbData[i] = null; } } - return new ColumnHolder<>(name, Boolean.class, null, grouped, dbData); + return new ColumnHolder<>(name, Boolean.class, null, indexed, dbData); } /** * Create a column holder from an array object, inferring the data type from the given array object. * * @param name The column name - * @param grouped true if the column is grouped; false otherwise + * @param indexed true if the column is indexed; false otherwise * @param data The data array * @return a column holder with a type matching the component type of the provided array */ - public static ColumnHolder createColumnHolder(String name, boolean grouped, T... data) { + public static ColumnHolder createColumnHolder(String name, boolean indexed, T... data) { return new ColumnHolder(name, data.getClass().getComponentType(), - data.getClass().getComponentType().getComponentType(), grouped, data); + data.getClass().getComponentType().getComponentType(), indexed, data); } public String getName() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/GroupedWritableRowRedirection.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/GroupedWritableRowRedirection.java index 6065f049f48..86150135f8e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/GroupedWritableRowRedirection.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/GroupedWritableRowRedirection.java @@ -19,9 +19,9 @@ * The GroupedWritableRowRedirection is intended for situations where you have several row sets that represent * contiguous rows of your output table and a flat output RowSet. *

- * When sorting a table by its grouping column, instead of using a large contiguous WritableRowRedirection, we simply - * store the row sets for each group and the accumulated cardinality. We then binary search in the accumulated - * cardinality for a given key; and fetch the corresponding offset from that group's row set. + * When sorting a table by indexed columns, instead of using a large contiguous WritableRowRedirection, we simply store + * the row sets for each group and the accumulated cardinality. We then binary search in the accumulated cardinality for + * a given key; and fetch the corresponding offset from that group's row set. *

* This WritableRowRedirection does not support mutation. */ @@ -47,14 +47,14 @@ public class GroupedWritableRowRedirection implements WritableRowRedirection { */ private final ThreadLocal threadContext = ThreadLocal.withInitial(SavedContext::new); - public GroupedWritableRowRedirection(long size, long[] groupSizes, RowSet[] groups) { + public GroupedWritableRowRedirection(final long size, final long[] groupSizes, final RowSet[] groups) { this.size = size; this.groupSizes = groupSizes; this.groups = groups; } @Override - public long get(long outerRowKey) { + public long get(final long outerRowKey) { if (outerRowKey < 0 || outerRowKey >= size) { return RowSequence.NULL_ROW_KEY; } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/GroupingValidator.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/GroupingValidator.java deleted file mode 100644 index 1dbccf9ab38..00000000000 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/GroupingValidator.java +++ /dev/null @@ -1,191 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.engine.table.impl; - -import io.deephaven.base.verify.Assert; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.TupleSource; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.TrackingRowSet; -import junit.framework.TestCase; - -import java.util.*; - -/** - * This class listens to a table and on each update verifies that the groupings returned by the table's RowSet for a set - * of columns are still valid. It is meant to be used as part of a unit test for incremental updates, to ensure that - * stale groupings are not left between table updates. - */ -public class GroupingValidator extends InstrumentedTableUpdateListenerAdapter { - private final Table source; - private final Collection groupingColumns; - private final String context; - private int validationCount = 0; - - public GroupingValidator(String context, Table source, ArrayList> groupingColumns) { - this(context, source, convertListToArray(groupingColumns)); - } - - static private Collection convertListToArray(ArrayList> groupingColumns) { - Collection collectionOfArrays = new ArrayList<>(); - for (ArrayList columnSet : groupingColumns) { - collectionOfArrays.add(columnSet.toArray(new String[columnSet.size()])); - } - return collectionOfArrays; - } - - private GroupingValidator(String context, Table source, Collection groupingColumns) { - super("grouping validator " + context, source, false); - this.context = context; - this.source = source; - this.groupingColumns = groupingColumns; - - validateGroupings(groupingColumns, source.getRowSet()); - validatePrevGroupings(groupingColumns, source.getRowSet()); - - source.addUpdateListener(this); - } - - private void validateGroupings(Collection groupingColumns, RowSet rowSet) { - for (String[] groupingToCheck : groupingColumns) { - validateGrouping(groupingToCheck, rowSet, source, context); - } - } - - private void validatePrevGroupings(Collection groupingColumns, TrackingRowSet rowSet) { - for (String[] groupingToCheck : groupingColumns) { - validatePrevGrouping(groupingToCheck, rowSet); - } - } - - public static void validateGrouping(String[] groupingToCheck, RowSet rowSet, Table source, String context) { - final ColumnSource[] groupColumns = getColumnSources(groupingToCheck, source); - final TupleSource tupleSource = TupleSourceFactory.makeTupleSource(groupColumns); - validateGrouping(groupingToCheck, rowSet, source, context, - rowSet.isTracking() - ? RowSetIndexer.of(rowSet.trackingCast()).getGrouping(tupleSource) - : Collections.emptyMap()); - } - - public static void validateGrouping(String[] groupingToCheck, RowSet rowSet, Table source, String context, - Map grouping) { - final ColumnSource[] groupColumns = getColumnSources(groupingToCheck, source); - for (Map.Entry objectIndexEntry : grouping.entrySet()) { - for (RowSet.Iterator it = objectIndexEntry.getValue().iterator(); it.hasNext();) { - long next = it.nextLong(); - checkGroupKey(groupColumns, next, objectIndexEntry.getKey(), context); - } - } - - for (RowSet.Iterator it = rowSet.iterator(); it.hasNext();) { - long next = it.nextLong(); - Object key = getValue(groupColumns, next); - RowSet keyRowSet = grouping.get(key); - Assert.assertion(keyRowSet != null, "keyRowSet != null", next, "next", key, "key", context, "context"); - if (keyRowSet != null) { - Assert.assertion(keyRowSet.find(next) >= 0, "keyRowSet.find(next) >= 0", next, "next", key, "key", - keyRowSet, "keyRowSet", context, "context"); - } - } - } - - public static void validateRestrictedGrouping(String[] groupingToCheck, RowSet rowSet, Table source, - String context, Map grouping, Set validKeys) { - ColumnSource[] groupColumns = getColumnSources(groupingToCheck, source); - for (Map.Entry objectIndexEntry : grouping.entrySet()) { - final Object groupKey = objectIndexEntry.getKey(); - Assert.assertion(validKeys.contains(groupKey), "validKeys.contains(objectIndexEntry.getKey())", groupKey, - "groupKey", validKeys, "validKeys"); - for (RowSet.Iterator it = objectIndexEntry.getValue().iterator(); it.hasNext();) { - long next = it.nextLong(); - checkGroupKey(groupColumns, next, groupKey, context); - } - } - - for (RowSet.Iterator it = rowSet.iterator(); it.hasNext();) { - long next = it.nextLong(); - Object key = getValue(groupColumns, next); - RowSet keyRowSet = grouping.get(key); - - if (validKeys.contains(key)) { - Assert.assertion(keyRowSet != null, "keyRowSet != null", next, "next", key, "key", context, "context"); - if (keyRowSet != null) { - Assert.assertion(keyRowSet.find(next) >= 0, "keyRowSet.find(next) >= 0", next, "next", key, "key", - keyRowSet, "keyRowSet", context, "context"); - } - } else { - Assert.assertion(keyRowSet == null, "keyRowSet == null", next, "next", key, "key", context, "context"); - } - } - } - - private void validatePrevGrouping(String[] groupingToCheck, TrackingRowSet rowSet) { - final ColumnSource[] groupColumns = getColumnSources(groupingToCheck, source); - final TupleSource tupleSource = TupleSourceFactory.makeTupleSource(groupColumns); - final Map grouping = RowSetIndexer.of(rowSet).getPrevGrouping(tupleSource); - for (Map.Entry objectIndexEntry : grouping.entrySet()) { - for (RowSet.Iterator it = objectIndexEntry.getValue().iterator(); it.hasNext();) { - long next = it.nextLong(); - checkGroupPrevKey(groupColumns, next, objectIndexEntry.getKey(), context); - } - } - - for (RowSet.Iterator it = rowSet.iterator(); it.hasNext();) { - long next = it.nextLong(); - Object key = getPrevValue(groupColumns, next); - RowSet keyRowSet = grouping.get(key); - Assert.assertion(keyRowSet != null, "keyRowSet != null", next, "next", key, "key", context, "context"); - if (keyRowSet != null) { - Assert.assertion(keyRowSet.find(next) >= 0, "keyRowSet.find(next) >= 0", next, "next", key, "key", - keyRowSet, "keyRowSet", context, "context"); - } - } - } - - private static ColumnSource[] getColumnSources(String[] groupingToCheck, Table source) { - return Arrays.stream(groupingToCheck).map(source::getColumnSource).toArray(ColumnSource[]::new); - } - - static private void checkGroupKey(final ColumnSource[] groupColumns, final long next, final Object key, - final String context) { - final Object value = getValue(groupColumns, next); - Assert.assertion(Objects.equals(value, key), "value.equals(key)", value, "value", key, "key", context, - "context"); - } - - static private void checkGroupPrevKey(final ColumnSource[] groupColumns, final long next, final Object key, - final String context) { - Object value = getPrevValue(groupColumns, next); - Assert.assertion(value == key || value.equals(key), "value.equals(key)", value, "value", key, "key", context, - "context"); - } - - static private Object getValue(ColumnSource[] groupColumns, long next) { - return TupleSourceFactory.makeTupleSource(groupColumns).createTuple(next); - } - - static private Object getPrevValue(ColumnSource[] groupColumns, long next) { - return TupleSourceFactory.makeTupleSource(groupColumns).createPreviousTuple(next); - } - - @Override - public void onUpdate(final TableUpdate upstream) { - validateGroupings(groupingColumns, source.getRowSet()); - // NB: This would normally be inappropriate: we don't expect grouping support on the non-tracking row sets we - // use for updates. Forcing support by cloning and making the result tracking. - validateGroupings(groupingColumns, upstream.added().copy().toTracking()); - validateGroupings(groupingColumns, upstream.modified().copy().toTracking()); - validationCount++; - System.out.println("Validation Count for " + context + ": " + validationCount); - } - - @Override - public void onFailureInternal(Throwable originalException, Entry sourceEntry) { - originalException.printStackTrace(); - TestCase.fail("Failure for context " + context + ": " + originalException.getMessage()); - } -} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/IndexValidator.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/IndexValidator.java new file mode 100644 index 00000000000..5c9de8d9b08 --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/IndexValidator.java @@ -0,0 +1,165 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.primitive.iterator.CloseableIterator; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.dataindex.DataIndexUtils; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.table.iterators.ChunkedColumnIterator; +import io.deephaven.util.SafeCloseable; +import junit.framework.TestCase; + +import java.util.*; + +import static io.deephaven.engine.table.impl.dataindex.DataIndexUtils.lookupKeysEqual; + +/** + * This class listens to a table and on each update verifies that the indexes returned by the table's RowSet for a set + * of columns are still valid. It is meant to be used as part of a unit test for incremental updates, to ensure that + * stale indexes are not left between table updates. + */ +public class IndexValidator extends InstrumentedTableUpdateListenerAdapter { + + private final Table sourceTable; + private final List indexColumns; + private final List indexes; + private final String context; + + private int validationCount = 0; + + public IndexValidator(String context, Table sourceTable, List> indexColumns) { + this(context, sourceTable, convertListToArray(indexColumns)); + } + + private static Collection convertListToArray(List> indexColumns) { + Collection collectionOfArrays = new ArrayList<>(); + for (List columnSet : indexColumns) { + collectionOfArrays.add(columnSet.toArray(new String[0])); + } + return collectionOfArrays; + } + + private IndexValidator(String context, Table sourceTable, Collection indexColumns) { + super("index validator " + context, sourceTable, false); + + this.context = context; + this.sourceTable = sourceTable; + + try (final SafeCloseable ignored = sourceTable.isRefreshing() ? LivenessScopeStack.open() : null) { + final List foundIndexColumns = new ArrayList<>(); + final List foundIndexes = new ArrayList<>(); + for (String[] keyColumns : indexColumns) { + final DataIndex index = DataIndexer.getDataIndex(sourceTable, keyColumns); + if (index != null) { + Assert.eq(sourceTable.isRefreshing(), "sourceTable.isRefreshing()", + index.isRefreshing(), "index.isRefreshing()"); + foundIndexColumns.add(keyColumns); + foundIndexes.add(index); + if (index.isRefreshing()) { + manage(index); + } + } + } + this.indexColumns = foundIndexColumns; + indexes = foundIndexes; + } + + validateIndexes(); + + sourceTable.addUpdateListener(this); + } + + @Override + public boolean canExecute(final long step) { + return super.canExecute(step) && indexes.stream().map(DataIndex::table).allMatch(t -> t.satisfied(step)); + } + + public static void validateIndex(final Table sourceTable, final String[] keyColumns, final boolean usePrev) { + final DataIndex index = DataIndexer.getDataIndex(sourceTable, keyColumns); + if (index == null) { + return; + } + final Table indexTable = index.table(); + + final ColumnSource[] sourceKeyColumns = + Arrays.stream(keyColumns).map(sourceTable::getColumnSource).toArray(ColumnSource[]::new); + + ChunkSource sourceKeys = DataIndexUtils.makeBoxedKeySource(sourceKeyColumns); + if (usePrev) { + sourceKeys = ((ChunkSource.WithPrev) sourceKeys).getPrevSource(); + } + final RowSet sourceTableRowSet = usePrev ? sourceTable.getRowSet().prev() : sourceTable.getRowSet(); + + final ColumnSource indexRowSets = usePrev ? index.rowSetColumn().getPrevSource() : index.rowSetColumn(); + ChunkSource indexKeys = DataIndexUtils.makeBoxedKeySource(index.keyColumns(sourceKeyColumns)); + if (usePrev) { + indexKeys = ((ChunkSource.WithPrev) indexKeys).getPrevSource(); + } + final RowSet indexTableRowSet = usePrev ? indexTable.getRowSet().prev() : indexTable.getRowSet(); + final DataIndex.RowKeyLookup indexLookup = index.rowKeyLookup(); + + // This is a lot of parallel iterators over the same RowSet, but this is test/validation code and its easier to + // read this way. + try (final WritableRowSet visitedRowSet = RowSetFactory.empty(); + final CloseableIterator indexRowSetsIterator = + ChunkedColumnIterator.make(indexRowSets, indexTableRowSet); + final CloseableIterator indexKeysIterator = ChunkedColumnIterator.make(indexKeys, indexTableRowSet); + final RowSet.Iterator indexTableRowSetIterator = indexTableRowSet.iterator()) { + while (indexKeysIterator.hasNext()) { + final RowSet indexRowSet = indexRowSetsIterator.next(); // Already in prev space if usePrev + final Object indexKey = indexKeysIterator.next(); + final long indexTableRowKey = indexTableRowSetIterator.nextLong(); + + // Validate that we haven't visited any row keys in the index row set yet + Assert.assertion(!visitedRowSet.overlaps(indexRowSet), "!visitedRowSet.overlaps(indexRowSet)"); + // Validate that all row keys in the index row set are part of the source table's row set + Assert.assertion(indexRowSet.subsetOf(sourceTableRowSet), "indexRowSet.subsetOf(sourceTableRowSet)"); + visitedRowSet.insert(indexRowSet); + + // Validate that every row the index bucket claims to include has the right key + try (final CloseableIterator sourceKeysIterator = + ChunkedColumnIterator.make(sourceKeys, indexRowSet)) { + sourceKeysIterator.forEachRemaining((final Object sourceKey) -> { + Assert.assertion(lookupKeysEqual(sourceKey, indexKey), "lookupKeysEqual(sourceKey, indexKey)"); + }); + } + + // Validate that the index lookup returns the right row key + Assert.eq(indexTableRowKey, "indexTableRowKey", + indexLookup.apply(indexKey, usePrev), "indexLookup.apply(indexKey, usePrev)"); + } + // Validate that we visit every row + Assert.equals(sourceTableRowSet, "sourceTableRowSet", visitedRowSet, "visitedRowSet"); + } + } + + private void validateIndexes() { + for (String[] indexColumn : indexColumns) { + validateIndex(sourceTable, indexColumn, false); + if (sourceTable.isRefreshing()) { + validateIndex(sourceTable, indexColumn, true); + } + } + } + + @Override + public void onUpdate(final TableUpdate upstream) { + validateIndexes(); + validationCount++; + System.out.println("Validation Count for " + context + ": " + validationCount); + } + + @Override + public void onFailureInternal(Throwable originalException, Entry sourceEntry) { + originalException.printStackTrace(); + TestCase.fail("Failure for context " + context + ": " + originalException.getMessage()); + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/MultiColumnSortTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/MultiColumnSortTest.java index d4f60299898..99838d1a5c3 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/MultiColumnSortTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/MultiColumnSortTest.java @@ -3,10 +3,12 @@ // package io.deephaven.engine.table.impl; +import com.google.common.collect.Sets; import io.deephaven.api.ColumnName; import io.deephaven.api.SortColumn; import io.deephaven.benchmarking.generator.ColumnGenerator; import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.testutil.generator.*; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.test.types.SerialTest; @@ -26,6 +28,10 @@ @Category(SerialTest.class) public class MultiColumnSortTest { + private enum IndexType { + NONE, FULL, PARTIAL + } + @Rule public EngineCleanup framework = new EngineCleanup(); @@ -34,12 +40,32 @@ public void testMultiColumnSort() { for (int size = 10; size <= 100_000; size *= 10) { for (int seed = 0; seed < 1; ++seed) { System.out.println("Seed: " + seed); - testMultiColumnSort(seed, size); + testMultiColumnSort(seed, size, IndexType.NONE); + } + } + } + + @Test + public void testMultiColumnSortFullIndex() { + for (int size = 10; size <= 100_000; size *= 10) { + for (int seed = 0; seed < 1; ++seed) { + System.out.println("Seed: " + seed); + testMultiColumnSort(seed, size, IndexType.FULL); + } + } + } + + @Test + public void testMultiColumnSortPartialIndex() { + for (int size = 10; size <= 100_000; size *= 10) { + for (int seed = 0; seed < 1; ++seed) { + System.out.println("Seed: " + seed); + testMultiColumnSort(seed, size, IndexType.PARTIAL); } } } - private void testMultiColumnSort(int seed, int size) { + private void testMultiColumnSort(int seed, int size, IndexType indexType) { final Random random = new Random(seed); final Table table = getTable(size, random, @@ -60,6 +86,24 @@ private void testMultiColumnSort(int seed, int size) { final List columnNames = table.getDefinition().getColumnNames(); + Set> keyColumnPowerSet = Sets.powerSet(new HashSet<>(columnNames)); + + if (indexType == IndexType.FULL) { + // Create full indexes for every possible column subset. + for (Set keyColumnSubset : keyColumnPowerSet) { + if (keyColumnSubset.isEmpty() || keyColumnSubset.size() == columnNames.size()) { + // Won't consider the empty or full set. + continue; + } + DataIndexer.getOrCreateDataIndex(table, keyColumnSubset.toArray(String[]::new)); + } + } else if (indexType == IndexType.PARTIAL) { + // Only create single-column indexes + for (String keyColumn : columnNames) { + DataIndexer.getOrCreateDataIndex(table, keyColumn); + } + } + doMultiColumnTest(table, SortColumn.asc(ColumnName.of("boolCol")), SortColumn.desc(ColumnName.of("Sym"))); for (String outerColumn : columnNames) { @@ -181,7 +225,97 @@ public void benchmarkTest() { "Enum2", 1000, 6, 6, 0xF00DF00DL); final BenchmarkTableBuilder builder; - final int actualSize = BenchmarkTools.sizeWithSparsity(25000000, 90); + final int actualSize = BenchmarkTools.sizeWithSparsity(10_000_000, 90); + + System.out.println("Actual Size: " + actualSize); + + builder = BenchmarkTools.persistentTableBuilder("Carlos", actualSize); + + final BenchmarkTable bmTable = builder + .setSeed(0xDEADBEEF) + .addColumn(BenchmarkTools.stringCol("PartCol", 4, 5, 7, 0xFEEDBEEF)) + .addColumn(BenchmarkTools.numberCol("I1", int.class)) + .addColumn(BenchmarkTools.numberCol("D1", double.class, -10e6, 10e6)) + .addColumn(BenchmarkTools.numberCol("L1", long.class)) + .addColumn(enumStringCol1) + .addColumn(enumStringCol2) + .build(); + + + final long startGen = System.currentTimeMillis(); + System.out.println(new Date(startGen) + " Generating Table."); + final Table table = bmTable.getTable(); + final long endGen = System.currentTimeMillis(); + System.out.println(new Date(endGen) + " Completed generate in " + (endGen - startGen) + "ms"); + + final long startSort = System.currentTimeMillis(); + System.out.println(new Date(startSort) + " Starting sort."); + + final Table sorted = table.sort("Enum1", "L1"); + + final long end = System.currentTimeMillis(); + System.out.println(new Date(end) + " Completed sort in " + (end - startSort) + "ms"); + + checkSort(sorted, SortColumn.asc(ColumnName.of("Enum1")), SortColumn.asc(ColumnName.of("L1"))); + } + } + + @Test + public void benchmarkFullIndexTest() { + { + final ColumnGenerator enumStringCol1 = BenchmarkTools.stringCol( + "Enum1", 10000, 6, 6, 0xB00FB00FL); + final ColumnGenerator enumStringCol2 = BenchmarkTools.stringCol( + "Enum2", 1000, 6, 6, 0xF00DF00DL); + + final BenchmarkTableBuilder builder; + final int actualSize = BenchmarkTools.sizeWithSparsity(10_000_000, 90); + + System.out.println("Actual Size: " + actualSize); + + builder = BenchmarkTools.persistentTableBuilder("Carlos", actualSize); + + final BenchmarkTable bmTable = builder + .setSeed(0xDEADBEEF) + .addColumn(BenchmarkTools.stringCol("PartCol", 4, 5, 7, 0xFEEDBEEF)) + .addColumn(BenchmarkTools.numberCol("I1", int.class)) + .addColumn(BenchmarkTools.numberCol("D1", double.class, -10e6, 10e6)) + .addColumn(BenchmarkTools.numberCol("L1", long.class)) + .addColumn(enumStringCol1) + .addColumn(enumStringCol2) + .build(); + + + final long startGen = System.currentTimeMillis(); + System.out.println(new Date(startGen) + " Generating Table."); + final Table table = bmTable.getTable(); + DataIndexer.getOrCreateDataIndex(table, "Enum1", "L1"); + + final long endGen = System.currentTimeMillis(); + System.out.println(new Date(endGen) + " Completed generate in " + (endGen - startGen) + "ms"); + + final long startSort = System.currentTimeMillis(); + System.out.println(new Date(startSort) + " Starting sort."); + + final Table sorted = table.sort("Enum1", "L1"); + + final long end = System.currentTimeMillis(); + System.out.println(new Date(end) + " Completed sort in " + (end - startSort) + "ms"); + + checkSort(sorted, SortColumn.asc(ColumnName.of("Enum1")), SortColumn.asc(ColumnName.of("L1"))); + } + } + + @Test + public void benchmarkFirstColumnIndexTest() { + { + final ColumnGenerator enumStringCol1 = BenchmarkTools.stringCol( + "Enum1", 10000, 6, 6, 0xB00FB00FL); + final ColumnGenerator enumStringCol2 = BenchmarkTools.stringCol( + "Enum2", 1000, 6, 6, 0xF00DF00DL); + + final BenchmarkTableBuilder builder; + final int actualSize = BenchmarkTools.sizeWithSparsity(10_000_000, 90); System.out.println("Actual Size: " + actualSize); @@ -201,6 +335,8 @@ public void benchmarkTest() { final long startGen = System.currentTimeMillis(); System.out.println(new Date(startGen) + " Generating Table."); final Table table = bmTable.getTable(); + DataIndexer.getOrCreateDataIndex(table, "Enum1"); + final long endGen = System.currentTimeMillis(); System.out.println(new Date(endGen) + " Completed generate in " + (endGen - startGen) + "ms"); @@ -215,4 +351,5 @@ public void benchmarkTest() { checkSort(sorted, SortColumn.asc(ColumnName.of("Enum1")), SortColumn.asc(ColumnName.of("L1"))); } } + } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index 672bec725dc..99aedfdb176 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -19,8 +19,9 @@ import io.deephaven.engine.rowset.RowSetShiftData; import io.deephaven.engine.rowset.TrackingWritableRowSet; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.testutil.QueryTableTestBase.TableComparator; import io.deephaven.engine.table.impl.by.*; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; import io.deephaven.engine.table.impl.select.IncrementalReleaseFilter; import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.engine.table.impl.select.SelectColumnFactory; @@ -28,7 +29,6 @@ import io.deephaven.engine.table.impl.sources.UnionRedirection; import io.deephaven.engine.table.impl.util.ColumnHolder; import io.deephaven.engine.testutil.*; -import io.deephaven.engine.testutil.QueryTableTestBase.TableComparator; import io.deephaven.engine.testutil.generator.*; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.engine.testutil.sources.TestColumnSource; @@ -195,14 +195,13 @@ public int initialHashTableSize(@NotNull final Table table) { @Test public void testStaticGroupedByWithChunks() { - final Table input = emptyTable(10000).update("A=Integer.toString(i % 5)", "B=i / 5"); - // noinspection unused - final Map g1 = RowSetIndexer.of(input.getRowSet()).getGrouping(input.getColumnSource("A")); - // noinspection unused - final Map g2 = RowSetIndexer.of(input.getRowSet()).getGrouping(input.getColumnSource("B")); + final Table input1 = emptyTable(10000).update("A=Integer.toString(i % 5)", "B=i / 5"); + + DataIndexer.getOrCreateDataIndex(input1, "A"); + DataIndexer.getOrCreateDataIndex(input1, "B"); - individualStaticByTest(input, null, "A"); - individualStaticByTest(input, null, "B"); + individualStaticByTest(input1, null, "A"); + individualStaticByTest(input1, null, "B"); } @Test @@ -309,23 +308,17 @@ public int initialHashTableSize(@NotNull final Table table) { return 8; } }; - final AggregationControl controlShiftByProbing = new AggregationControl() { - @Override - public boolean shouldProbeShift(long shiftSize, int numStates) { - return true; - } - }; final EvalNugget[] ens = new EvalNugget[] { incrementalByEvalNugget(controlSize8, merged), incrementalByEvalNugget(merged), incrementalByEvalNugget(controlSize8, merged, "StrCol"), - incrementalByEvalNugget(controlShiftByProbing, merged, "StrCol"), + incrementalByEvalNugget(AggregationControl.DEFAULT, merged, "StrCol"), incrementalByEvalNugget(controlSize8, merged, "IntCol"), incrementalByEvalNugget(merged, "IntCol"), incrementalByEvalNugget(controlSize8, merged, "TimeCol"), - incrementalByEvalNugget(controlShiftByProbing, merged, "TimeCol"), + incrementalByEvalNugget(AggregationControl.DEFAULT, merged, "TimeCol"), incrementalByEvalNugget(controlSize8, (QueryTable) merged.updateView("TimeCol=isNull(TimeCol) ? NULL_LONG : epochNanos(TimeCol)"), "TimeCol"), @@ -583,7 +576,7 @@ public void testStaticBy() { table = testRefreshingTable( col("S", "e", "c", "g"), - colGrouped("X", 4, 2, 6), + colIndexed("X", 4, 2, 6), col("Y", 4, 2, 2)); TestCase.assertEquals(2, table.updateView("Z=X+Y").groupBy("Z").size()); TestCase.assertEquals(4, table.updateView("Z=X+Y").groupBy("Z").numColumns()); @@ -609,7 +602,7 @@ public void testStaticBy() { table = testRefreshingTable( col("S", "e", "c", "g"), col("X", 4, 2, 6), - colGrouped("Y", 4, 2, 2)); + colIndexed("Y", 4, 2, 2)); TestCase.assertEquals(2, table.updateView("Z=X+Y").groupBy("Z").size()); TestCase.assertEquals(4, table.updateView("Z=X+Y").groupBy("Z").numColumns()); TestCase.assertEquals(ObjectVector.class, @@ -633,8 +626,8 @@ public void testStaticBy() { table = testRefreshingTable( col("S", "e", "c", "g"), - colGrouped("X", 4, 2, 6), - colGrouped("Y", 4, 3, 2)); + colIndexed("X", 4, 2, 6), + colIndexed("Y", 4, 3, 2)); TestCase.assertEquals(2, table.updateView("Z=X+Y").groupBy("Z").size()); TestCase.assertEquals(4, table.updateView("Z=X+Y").groupBy("Z").numColumns()); TestCase.assertEquals(ObjectVector.class, @@ -1361,7 +1354,7 @@ private void testSumByStatic(int size, boolean lotsOfStrings, boolean grouped) { "bigI", "bigD" }, - Arrays.asList(grouped ? Collections.singletonList(ColumnInfo.ColAttributes.Grouped) : ea, ea, ea, ea, + Arrays.asList(grouped ? Collections.singletonList(ColumnInfo.ColAttributes.Indexed) : ea, ea, ea, ea, ea, ea, ea, ea, ea, ea, ea), lotsOfStrings ? new StringGenerator(1000000) : new SetGenerator<>("a", "b", "c", "d"), new CharGenerator('a', 'z'), @@ -1684,7 +1677,7 @@ private void doTestSumByIncremental(final int size, final int seed, boolean grou final Random random = new Random(seed); final ColumnInfo[] columnInfo; final List ea = Collections.emptyList(); - final List ga = Collections.singletonList(ColumnInfo.ColAttributes.Grouped); + final List ga = Collections.singletonList(ColumnInfo.ColAttributes.Indexed); final QueryTable queryTable = getTable(size, random, columnInfo = initColumnInfos( new String[] {"Sym", "charCol", "byteCol", "shortCol", "intCol", "longCol", "bigI", "bigD", "doubleCol", "doubleNanCol", "boolCol"}, @@ -1710,16 +1703,19 @@ private void doTestSumByIncremental(final int size, final int seed, boolean grou EvalNugget.Sorted.from(() -> queryTable.sumBy("Sym"), "Sym"), EvalNugget.Sorted.from(() -> queryTable.sort("Sym").sumBy("Sym"), "Sym"), EvalNugget.Sorted.from(() -> queryTable.dropColumns("Sym").sort("intCol").sumBy("intCol"), "intCol"), - EvalNugget.Sorted.from(() -> queryTable.sort("Sym", "intCol").sumBy("Sym", "intCol"), "Sym", "intCol"), + EvalNugget.Sorted.from(() -> queryTable.sort("Sym", "intCol").sumBy("Sym", "intCol"), "Sym", + "intCol"), EvalNugget.Sorted.from(() -> queryTable.sort("Sym").update("x=intCol+1").sumBy("Sym"), "Sym"), EvalNugget.Sorted.from(() -> queryTable.sortDescending("intCol").update("x=intCol+1").dropColumns("Sym") .sumBy("intCol"), "intCol"), EvalNugget.Sorted.from( () -> queryTable.sort("Sym", "intCol").update("x=intCol+1").sumBy("Sym", "intCol"), "Sym", "intCol"), - EvalNugget.Sorted.from(() -> queryTable.sort("Sym", "intCol").update("x=intCol+1").sumBy("Sym"), "Sym"), + EvalNugget.Sorted.from(() -> queryTable.sort("Sym", "intCol").update("x=intCol+1").sumBy("Sym"), + "Sym"), EvalNugget.Sorted.from(() -> queryTable.sort("Sym").absSumBy("Sym"), "Sym"), - EvalNugget.Sorted.from(() -> queryTable.dropColumns("Sym").sort("intCol").absSumBy("intCol"), "intCol"), + EvalNugget.Sorted.from(() -> queryTable.dropColumns("Sym").sort("intCol").absSumBy("intCol"), + "intCol"), EvalNugget.Sorted.from(() -> queryTable.sort("Sym", "intCol").absSumBy("Sym", "intCol"), "Sym", "intCol"), EvalNugget.Sorted.from(() -> queryTable.sort("Sym").update("x=intCol+1").absSumBy("Sym"), "Sym"), @@ -3514,7 +3510,7 @@ public void testFirstLastByAttributes() { public void testIds6220() { final QueryTable table = testRefreshingTable( RowSetFactory.fromRange(0, 2).toTracking(), - colGrouped("Key", "a", "b", "c"), col("I", 2, 4, 6)); + colIndexed("Key", "a", "b", "c"), col("I", 2, 4, 6)); final IncrementalReleaseFilter filter = new IncrementalReleaseFilter(0, 10); final Table byTable = table.where(filter).groupBy("Key"); TableTools.showWithRowSet(byTable); @@ -3743,14 +3739,12 @@ public void testInitialGroupsWithGrouping() { // Tests grouped addition for static tables and static initial groups final Table data = testTable(col("S", "A", "A", "B", "B"), col("I", 10, 20, 30, 40)); - final RowSetIndexer dataIndexer = RowSetIndexer.of(data.getRowSet()); - dataIndexer.getGrouping(data.getColumnSource("S")); + DataIndexer.getOrCreateDataIndex(data, "S"); final Table distinct = data.selectDistinct("S"); assertTableEquals(testTable(col("S", "A", "B")), distinct); final Table reversed = data.reverse(); - final RowSetIndexer reversedIndexer = RowSetIndexer.of(reversed.getRowSet()); - reversedIndexer.getGrouping(reversed.getColumnSource("S")); + DataIndexer.getOrCreateDataIndex(reversed, "S"); final Table initializedDistinct = data.aggBy(List.of(Count.of("C")), false, reversed, ColumnName.from("S")).dropColumns("C"); assertTableEquals(testTable(col("S", "B", "A")), initializedDistinct); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAjTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAjTest.java index a3e7e8369c7..a113c85e234 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAjTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAjTest.java @@ -10,6 +10,7 @@ import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.engine.table.PartitionedTable; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.testutil.*; import io.deephaven.engine.testutil.generator.*; import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; @@ -99,9 +100,21 @@ public void testAjStatic() { } @Test - public void testAjStaticGrouped() { + public void testAjStaticGroupedBoth() { // noinspection unchecked - testAjStatic(TstUtils::colGrouped, TstUtils::colGrouped); + testAjStatic(TstUtils::colIndexed, TstUtils::colIndexed); + } + + @Test + public void testAjStaticGroupedLeftOnly() { + // noinspection unchecked + testAjStatic(TstUtils::colIndexed, TableTools::col); + } + + @Test + public void testAjStaticGroupedRightOnly() { + // noinspection unchecked + testAjStatic(TableTools::col, TstUtils::colIndexed); } public void testAjStatic(MakeColumn leftMaker, MakeColumn rightMaker) { @@ -154,6 +167,83 @@ public void testAjStatic(MakeColumn leftMaker, MakeColumn rightMaker) { intColumn(reverseResultGt, "Sentinel")); } + @Test + public void testAjStaticIndexedBoth() { + // noinspection unchecked + testAjStaticIndexed(true, true); + } + + @Test + public void testAjStaticIndexedLeftOnly() { + // noinspection unchecked + testAjStaticIndexed(true, false); + } + + @Test + public void testAjStaticIndexedRightOnly() { + // noinspection unchecked + testAjStaticIndexed(false, true); + } + + public void testAjStaticIndexed(boolean leftIndexed, boolean rightIndexed) { + final Table left = TstUtils.testTable( + col("BucketA", "A", "B", "A", "C", "D", "A"), + col("BucketB", "A", "A", "A", "A", "A", "A"), + longCol("LeftStamp", 1L, 10L, 50L, 3L, 4L, 60L)); + final Table right = TstUtils.testTable( + col("BucketA", "A", "B", "A", "B", "A", "D", "E"), + col("BucketB", "A", "A", "A", "A", "A", "A", "A"), + longCol("RightStamp", 1L, 5L, 10L, 25L, 50L, 5L, 3L), + intCol("Sentinel", 1, 2, 3, 4, 5, 6, 7)); + + if (leftIndexed) { + DataIndexer.getOrCreateDataIndex(left, "BucketA", "BucketB"); + } + if (rightIndexed) { + DataIndexer.getOrCreateDataIndex(right, "BucketA", "BucketB"); + } + + System.out.println("Left"); + TableTools.show(left); + System.out.println("Right"); + TableTools.show(right); + + final Table result = left.aj(right, "BucketA,BucketB,LeftStamp>=RightStamp", "Sentinel"); + System.out.println("Result"); + TableTools.showWithRowSet(result); + assertEquals(Arrays.asList("BucketA", "BucketB", "LeftStamp", "RightStamp", "Sentinel"), + result.getDefinition().getColumnNames()); + + BaseArrayTestCase.assertEquals(new int[] {1, 2, 5, NULL_INT, NULL_INT, 5}, intColumn(result, "Sentinel")); + + final Table ltResult = left.aj(right, "BucketA,BucketB,LeftStamp>RightStamp", "Sentinel"); + System.out.println("LT Result"); + TableTools.showWithRowSet(ltResult); + assertEquals(Arrays.asList("BucketA", "BucketB", "LeftStamp", "RightStamp", "Sentinel"), + ltResult.getDefinition().getColumnNames()); + + BaseArrayTestCase.assertEquals(new int[] {NULL_INT, 2, 3, NULL_INT, NULL_INT, 5}, + intColumn(ltResult, "Sentinel")); + + final Table reverseResult = left.raj(right, "BucketA,BucketB,LeftStamp<=RightStamp", "Sentinel"); + System.out.println("Reverse Result"); + TableTools.showWithRowSet(reverseResult); + assertEquals(Arrays.asList("BucketA", "BucketB", "LeftStamp", "RightStamp", "Sentinel"), + reverseResult.getDefinition().getColumnNames()); + + BaseArrayTestCase.assertEquals(new int[] {1, 4, 5, NULL_INT, 6, NULL_INT}, + intColumn(reverseResult, "Sentinel")); + + final Table reverseResultGt = left.raj(right, "BucketA,BucketB,LeftStamp stampType) { + testAjRandomIncrementalWithInitial(seed, leftNodeSize, rightNodeSize, leftSize, rightSize, joinIncrement, + leftRefreshing, rightRefreshing, initialOnly, withZeroKeys, withBuckets, withReverse, false, false, + getJoinControlWithNodeSize(leftNodeSize, rightNodeSize), stampType); + } + + @Test + public void testAjBothIncrementalIndexed() { + final int tableMultiplier = 10; + final int initialTableSize = 10; + final int maximumTableSize = 100; + final int nodeMultiplier = 8; + final int initialNodeSize = 4; + final int maximumNodeSize = 256; + final int seedCount = 1; + + final long startTime = System.currentTimeMillis(); + int configurations = 0; + + for (int leftSize = initialTableSize; leftSize <= maximumTableSize; leftSize *= tableMultiplier) { + for (int rightSize = initialTableSize; rightSize <= maximumTableSize; rightSize *= tableMultiplier) { + for (int leftNodeSize = initialNodeSize; leftNodeSize <= maximumNodeSize; leftNodeSize *= + nodeMultiplier) { + if (leftNodeSize / nodeMultiplier > leftSize) { + continue; + } + + for (int rightNodeSize = initialNodeSize; rightNodeSize <= maximumNodeSize; rightNodeSize *= + nodeMultiplier) { + if (rightNodeSize / nodeMultiplier > rightSize) { + continue; + } + + for (int seed = 0; seed < seedCount; ++seed) { + for (final JoinIncrement joinIncrement : new JoinIncrement[] {base.leftRightStepShift, + base.leftRightConcurrentStepShift}) { + System.out.println((System.currentTimeMillis() - startTime) + ": Seed=" + seed + + ", leftNodeSize=" + leftNodeSize + ", rightNodeSize=" + rightNodeSize + + ", leftSize=" + leftSize + ", rightSize=" + rightSize + ", joinIncrement=" + + joinIncrement); + try (final SafeCloseable ignored = LivenessScopeStack.open()) { + testAjRandomBothIncrementalIndexed(seed, leftNodeSize, rightNodeSize, leftSize, + rightSize, + joinIncrement, int.class); + testAjRandomBothIncrementalLeftIndexed(seed, leftNodeSize, rightNodeSize, leftSize, + rightSize, + joinIncrement, int.class); + testAjRandomBothIncrementalRightIndexed(seed, leftNodeSize, rightNodeSize, leftSize, + rightSize, + joinIncrement, int.class); + } + } + configurations++; + } + } + } + } + } + + System.out.println( + "Executed " + configurations + " configurations in " + (System.currentTimeMillis() - startTime) + "ms"); + } + + private void testAjRandomBothIncrementalIndexed(int seed, int leftNodeSize, int rightNodeSize, int leftSize, + int rightSize, + JoinIncrement joinIncrement, Class stampType) { + testAjRandomIncrementalWithInitial(seed, leftNodeSize, rightNodeSize, leftSize, rightSize, joinIncrement, true, + true, false, false, true, false, true, true, + getJoinControlWithNodeSize(leftNodeSize, rightNodeSize), stampType); + } + + private void testAjRandomBothIncrementalLeftIndexed(int seed, int leftNodeSize, int rightNodeSize, int leftSize, + int rightSize, + JoinIncrement joinIncrement, Class stampType) { + testAjRandomIncrementalWithInitial(seed, leftNodeSize, rightNodeSize, leftSize, rightSize, joinIncrement, true, + true, false, false, true, false, true, false, + getJoinControlWithNodeSize(leftNodeSize, rightNodeSize), stampType); + } + + private void testAjRandomBothIncrementalRightIndexed(int seed, int leftNodeSize, int rightNodeSize, int leftSize, + int rightSize, + JoinIncrement joinIncrement, Class stampType) { + testAjRandomIncrementalWithInitial(seed, leftNodeSize, rightNodeSize, leftSize, rightSize, joinIncrement, true, + true, false, false, true, false, false, true, + getJoinControlWithNodeSize(leftNodeSize, rightNodeSize), stampType); + } + @SuppressWarnings("SameParameterValue") private void testAjRandomIncrementalWithInitial(int seed, int leftNodeSize, int rightNodeSize, int leftSize, int rightSize, JoinIncrement joinIncrement, boolean leftRefreshing, boolean rightRefreshing, - boolean initialOnly, boolean withZeroKeys, boolean withBuckets, boolean withReverse, JoinControl control, + boolean initialOnly, boolean withZeroKeys, boolean withBuckets, boolean withReverse, boolean leftIndexing, + boolean rightIndexing, JoinControl control, Class stampType) { final Random random = new Random(seed); final int maxSteps = 10; @@ -876,6 +1078,9 @@ private void testAjRandomIncrementalWithInitial(int seed, int leftNodeSize, int new SetGenerator<>(String.class, set1), leftStampGenerator, new IntGenerator(10_000_000, 10_010_000))); + if (leftIndexing) { + DataIndexer.getOrCreateDataIndex(leftTable, "Bucket"); + } final ColumnInfo[] rightColumnInfo; final QueryTable rightTable = getTable(rightRefreshing, rightSize, random, rightColumnInfo = initColumnInfos(new String[] {"Truthiness", "Bucket", "RightStamp", "RightSentinel"}, @@ -885,7 +1090,10 @@ private void testAjRandomIncrementalWithInitial(int seed, int leftNodeSize, int new IntGenerator(20_000_000, 20_010_000))); final QueryTable rightSorted = sortRight ? (QueryTable) rightTable.sort("RightStamp") : rightTable; - + if (rightIndexing) { + // Indexing doesn't currently survive sorting. + DataIndexer.getOrCreateDataIndex(rightSorted, "Bucket"); + } if (RefreshingTableTestCase.printTableUpdates) { System.out.println("Left: "); TableTools.showWithRowSet(leftTable); @@ -917,6 +1125,10 @@ private void testAjRandomIncrementalWithInitial(int seed, int leftNodeSize, int } final QueryTable rightReversed = (QueryTable) rightSorted.reverse(); + if (rightIndexing) { + // Indexing doesn't currently survive reversal. + DataIndexer.getOrCreateDataIndex(rightReversed, "Bucket"); + } final EvalNuggetInterface[] en = Stream.concat(Stream.concat(!withZeroKeys ? Stream.empty() : Stream.concat( @@ -1069,7 +1281,7 @@ public int leftChunkSize() { private void testAjRandomIncremental(JoinIncrement joinIncrement, int seed, int leftSize, int rightSize, - boolean leftRefreshing, boolean rightRefreshing) { + boolean leftRefreshing, boolean rightRefreshing, boolean leftIndexed, boolean rightIndexed) { final Random random = new Random(seed); final int maxSteps = 10; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableHugeSortTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableHugeSortTest.java index 035444abf70..14ff2e44221 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableHugeSortTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableHugeSortTest.java @@ -3,11 +3,11 @@ // package io.deephaven.engine.table.impl; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.context.QueryScope; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.util.TableTools; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.test.types.OutOfBandTest; import org.junit.Rule; @@ -15,8 +15,6 @@ import java.util.Arrays; import java.util.Comparator; -import java.util.LinkedHashMap; -import java.util.Map; import org.junit.experimental.categories.Category; import static io.deephaven.engine.util.TableTools.show; @@ -76,7 +74,7 @@ public void testHugeSort() { } @Test - public void testHugeGroupedSort() { + public void testHugeIndexedSort() { final String[] captains = new String[] {"Hornigold", "Jennings", "Vane", "Bellamy"}; final long tableSize = 1L << 24; // 16 MM (note we msut be a multiple of captains.length) @@ -84,21 +82,20 @@ public void testHugeGroupedSort() { QueryScope.addParam("captains", captains); QueryScope.addParam("segSize", segSize); - final Table grouped = + final Table indexed = TableTools.emptyTable(tableSize).updateView("Captain=captains[(int)(ii / segSize)]", "Sentinel=ii"); - final Map gtr = new LinkedHashMap<>(); - for (int ii = 0; ii < captains.length; ++ii) { - gtr.put(captains[ii], RowSetFactory.fromRange(ii * segSize, (ii + 1) * segSize - 1)); - } - System.out.println(gtr); - ((AbstractColumnSource) (grouped.getColumnSource("Captain"))).setGroupToRange(gtr); + + final ColumnSource captainSource = (ColumnSource) indexed.getColumnSource("Captain"); + + // Create the index for this table and column. + DataIndexer.getOrCreateDataIndex(indexed, "Captain"); final long sortStart = System.currentTimeMillis(); - final Table sortedGrouped = grouped.sortDescending("Captain"); + final Table sortedIndexed = indexed.sortDescending("Captain"); final long sortDuration = System.currentTimeMillis() - sortStart; System.out.println("Sort Duration: " + sortDuration + "ms"); - show(sortedGrouped); + show(sortedIndexed); final String[] sortedCaptains = Arrays.copyOf(captains, captains.length); Arrays.sort(sortedCaptains, Comparator.reverseOrder()); @@ -108,7 +105,7 @@ public void testHugeGroupedSort() { System.out.println("Comparing tables:"); final long compareStart = System.currentTimeMillis(); - assertTableEquals(sortedValues.view("Captain"), sortedGrouped.view("Captain")); + assertTableEquals(sortedValues.view("Captain"), sortedIndexed.view("Captain")); final long compareDuration = System.currentTimeMillis() - compareStart; System.out.println("Compare Duration: " + compareDuration + "ms"); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableJoinTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableJoinTest.java index e324d2ee9ad..d975556ea9c 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableJoinTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableJoinTest.java @@ -29,6 +29,8 @@ import java.util.Arrays; import java.util.Random; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -770,62 +772,22 @@ public void testRaj() { static final JoinControl SMALL_LEFT_CONTROL = new JoinControl() { @Override - boolean buildLeft(QueryTable leftTable, Table rightTable) { - return true; - } - - @Override - int tableSizeForRightBuild(Table rightTable) { - return 1 << 8; - } - - @Override - int tableSizeForLeftBuild(Table rightTable) { - return 1 << 8; + BuildParameters buildParameters( + @NotNull final Table leftTable, @Nullable Table leftDataIndexTable, + @NotNull final Table rightTable, @Nullable Table rightDataIndexTable) { + return new BuildParameters(BuildParameters.From.LeftInput, 1 << 8); } }; static final JoinControl SMALL_RIGHT_CONTROL = new JoinControl() { @Override - boolean buildLeft(QueryTable leftTable, Table rightTable) { - return false; - } - - - @Override - int tableSizeForRightBuild(Table rightTable) { - return 1 << 8; - } - - @Override - int tableSizeForLeftBuild(Table rightTable) { - return 1 << 8; + BuildParameters buildParameters( + @NotNull final Table leftTable, @Nullable Table leftDataIndexTable, + @NotNull final Table rightTable, @Nullable Table rightDataIndexTable) { + return new BuildParameters(BuildParameters.From.RightInput, 1 << 8); } }; - static final JoinControl HIGH_LOAD_FACTOR_CONTROL = new JoinControl() { - @Override - int tableSizeForRightBuild(Table rightTable) { - return 1 << 8; - } - - @Override - int tableSizeForLeftBuild(Table leftTable) { - return 1 << 8; - } - - @Override - double getMaximumLoadFactor() { - return 20.0; - } - - @Override - double getTargetLoadFactor() { - return 19.0; - } - }; - - @Test public void testAjRegression0() { final QueryTable rightQueryTable = TstUtils.testRefreshingTable( diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableNaturalJoinTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableNaturalJoinTest.java index d99b96dc675..009cd0c5e50 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableNaturalJoinTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableNaturalJoinTest.java @@ -17,7 +17,7 @@ import io.deephaven.engine.table.DataColumn; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.select.MatchPairFactory; import io.deephaven.engine.table.impl.util.ColumnHolder; import io.deephaven.engine.testutil.*; @@ -39,7 +39,6 @@ import java.time.Instant; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.Random; import java.util.function.BiConsumer; import java.util.function.Function; @@ -165,7 +164,10 @@ public void testNaturalJoinIncremental() { final int sz = 5; final int maxSteps = 10; for (JoinIncrement joinIncrement : joinIncrementorsShift) { - testNaturalJoinIncremental(false, false, sz, sz, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(false, false, sz, sz, false, false, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(false, false, sz, sz, true, false, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(false, false, sz, sz, false, true, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(false, false, sz, sz, true, true, joinIncrement, 0, maxSteps); } final int[] leftSizes = new int[] {10, 50, 100}; @@ -175,7 +177,14 @@ public void testNaturalJoinIncremental() { for (long seed = 0; seed < 5; seed++) { System.out.println("leftSize=" + leftSize + ", rightSize=" + rightSize + ", seed=" + seed); for (JoinIncrement joinIncrement : joinIncrementorsShift) { - testNaturalJoinIncremental(false, false, leftSize, rightSize, joinIncrement, seed, maxSteps); + testNaturalJoinIncremental(false, false, leftSize, rightSize, false, false, joinIncrement, seed, + maxSteps); + testNaturalJoinIncremental(false, false, leftSize, rightSize, true, false, joinIncrement, seed, + maxSteps); + testNaturalJoinIncremental(false, false, leftSize, rightSize, false, true, joinIncrement, seed, + maxSteps); + testNaturalJoinIncremental(false, false, leftSize, rightSize, true, true, joinIncrement, seed, + maxSteps); } } } @@ -186,14 +195,24 @@ public void testNaturalJoinLeftIncrementalRightStatic() { for (JoinIncrement joinIncrement : new JoinIncrement[] {leftStepShift, leftStep}) { final int sz = 5; final int maxSteps = 20; - testNaturalJoinIncremental(false, true, sz, sz, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(false, true, sz, sz, false, false, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(false, true, sz, sz, true, false, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(false, true, sz, sz, false, true, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(false, true, sz, sz, true, true, joinIncrement, 0, maxSteps); final int[] leftSizes = new int[] {50, 100}; final int[] rightSizes = new int[] {50, 100}; for (long seed = 0; seed < 1; seed++) { for (int leftSize : leftSizes) { for (int rightSize : rightSizes) { - testNaturalJoinIncremental(false, true, leftSize, rightSize, joinIncrement, seed, maxSteps); + testNaturalJoinIncremental(false, true, leftSize, rightSize, false, false, joinIncrement, seed, + maxSteps); + testNaturalJoinIncremental(false, true, leftSize, rightSize, true, false, joinIncrement, seed, + maxSteps); + testNaturalJoinIncremental(false, true, leftSize, rightSize, false, true, joinIncrement, seed, + maxSteps); + testNaturalJoinIncremental(false, true, leftSize, rightSize, true, true, joinIncrement, seed, + maxSteps); } } } @@ -204,14 +223,24 @@ public void testNaturalJoinLeftStaticRightIncremental() { for (JoinIncrement joinIncrement : new JoinIncrement[] {rightStepShift, rightStep}) { final int sz = 5; final int maxSteps = 20; - testNaturalJoinIncremental(true, false, sz, sz, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(true, false, sz, sz, false, false, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(true, false, sz, sz, true, false, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(true, false, sz, sz, false, true, joinIncrement, 0, maxSteps); + testNaturalJoinIncremental(true, false, sz, sz, true, true, joinIncrement, 0, maxSteps); final int[] leftSizes = new int[] {50, 100}; final int[] rightSizes = new int[] {50, 100}; for (long seed = 0; seed < 5; seed++) { for (int leftSize : leftSizes) { for (int rightSize : rightSizes) { - testNaturalJoinIncremental(true, false, leftSize, rightSize, joinIncrement, seed, maxSteps); + testNaturalJoinIncremental(true, false, leftSize, rightSize, false, false, joinIncrement, seed, + maxSteps); + testNaturalJoinIncremental(true, false, leftSize, rightSize, true, false, joinIncrement, seed, + maxSteps); + testNaturalJoinIncremental(true, false, leftSize, rightSize, false, true, joinIncrement, seed, + maxSteps); + testNaturalJoinIncremental(true, false, leftSize, rightSize, true, true, joinIncrement, seed, + maxSteps); } } } @@ -219,25 +248,32 @@ public void testNaturalJoinLeftStaticRightIncremental() { } private void testNaturalJoinIncremental(boolean leftStatic, boolean rightStatic, int leftSize, int rightSize, + boolean leftIndexed, boolean rightIndexed, JoinIncrement joinIncrement, long seed, long maxSteps) { - testNaturalJoinIncremental(leftStatic, rightStatic, leftSize, rightSize, joinIncrement, seed, + testNaturalJoinIncremental(leftStatic, rightStatic, leftSize, rightSize, leftIndexed, rightIndexed, + joinIncrement, seed, new MutableInt((int) maxSteps)); } private void testNaturalJoinIncremental(boolean leftStatic, boolean rightStatic, int leftSize, int rightSize, + boolean leftIndexed, boolean rightIndexed, JoinIncrement joinIncrement, long seed, MutableInt numSteps) { - testNaturalJoinIncremental(leftStatic, rightStatic, leftSize, rightSize, joinIncrement, seed, numSteps, + testNaturalJoinIncremental(leftStatic, rightStatic, leftSize, rightSize, leftIndexed, rightIndexed, + joinIncrement, seed, numSteps, new JoinControl()); } private static void testNaturalJoinIncremental(boolean leftStatic, boolean rightStatic, int leftSize, int rightSize, - JoinIncrement joinIncrement, long seed, long maxSteps, JoinControl control) { - testNaturalJoinIncremental(leftStatic, rightStatic, leftSize, rightSize, joinIncrement, seed, + boolean leftIndexed, boolean rightIndexed, JoinIncrement joinIncrement, long seed, long maxSteps, + JoinControl control) { + testNaturalJoinIncremental(leftStatic, rightStatic, leftSize, rightSize, leftIndexed, rightIndexed, + joinIncrement, seed, new MutableInt((int) maxSteps), control); } private static void testNaturalJoinIncremental(boolean leftStatic, boolean rightStatic, int leftSize, int rightSize, - JoinIncrement joinIncrement, long seed, MutableInt numSteps, JoinControl control) { + boolean leftIndexed, boolean rightIndexed, JoinIncrement joinIncrement, long seed, MutableInt numSteps, + JoinControl control) { final Random random = new Random(seed); final int maxSteps = numSteps.intValue(); @@ -259,6 +295,11 @@ private static void testNaturalJoinIncremental(boolean leftStatic, boolean right compositeGenerator, new SetGenerator<>("a", "b"), rightInt2Generator)); + if (rightIndexed) { + DataIndexer.getOrCreateDataIndex(rightTable, "I1"); + DataIndexer.getOrCreateDataIndex(rightTable, "I1", "C1"); + DataIndexer.getOrCreateDataIndex(rightTable, "I1", "C1", "C2"); + } final ColumnInfo[] leftColumnInfo; final QueryTable leftTable = getTable(!leftStatic, leftSize, random, @@ -266,19 +307,24 @@ private static void testNaturalJoinIncremental(boolean leftStatic, boolean right new FromUniqueIntGenerator(rightIntGenerator, new IntGenerator(20, 10000), 0.75), new SetGenerator<>("a", "b", "c"), new FromUniqueIntGenerator(rightInt2Generator, new IntGenerator(20, 10000), 0.75))); + if (leftIndexed) { + DataIndexer.getOrCreateDataIndex(leftTable, "I1"); + DataIndexer.getOrCreateDataIndex(leftTable, "I1", "C1"); + DataIndexer.getOrCreateDataIndex(leftTable, "I1", "C1", "C2"); + } final EvalNugget[] en = new EvalNugget[] { new EvalNugget() { public Table e() { return NaturalJoinHelper.naturalJoin(leftTable, rightTable, MatchPairFactory.getExpressions("I1"), - MatchPairFactory.getExpressions("LI1=I1", "LC1=C1", "LC2=C2"), false, control); + MatchPairFactory.getExpressions("RI1=I1", "RC1=C1", "RC2=C2"), false, control); } }, new EvalNugget() { public Table e() { return NaturalJoinHelper.naturalJoin(leftTable, rightTable, - MatchPairFactory.getExpressions("C1", "I1"), MatchPairFactory.getExpressions("LC2=C2"), + MatchPairFactory.getExpressions("C1", "I1"), MatchPairFactory.getExpressions("RC2=C2"), false, control); } }, @@ -324,7 +370,7 @@ private void testNaturalJoinMixedGroupingLeftStatic(int leftSize, int rightSize, final Random random = new Random(seed); final QueryTable leftTable = getTable(false, leftSize, random, initColumnInfos(new String[] {"I1", "C1", "C2"}, - new ColumnInfo.ColAttributes[] {ColumnInfo.ColAttributes.Grouped}, + new ColumnInfo.ColAttributes[] {ColumnInfo.ColAttributes.Indexed}, new IntGenerator(1, rightSize * 10), new SetGenerator<>("a", "b", "c", "d", "e", "f"), new IntGenerator(1, 10))); @@ -362,11 +408,9 @@ private void testNaturalJoinMixedGroupingLeftStatic(int leftSize, int rightSize, assertTableEquals(noGroupingResult, result); final Table leftFlat = leftTable.flatten(); - final ColumnSource flatGrouped = leftFlat.getColumnSource("I1"); - final TrackingRowSet flatRowSet = leftFlat.getRowSet(); - final Map grouping = RowSetIndexer.of(flatRowSet).getGrouping(flatGrouped); - // noinspection unchecked - ((AbstractColumnSource) flatGrouped).setGroupToRange(grouping); + + // Create the data index for this table and column. + DataIndexer.getOrCreateDataIndex(leftFlat, "I1"); final Table resultFlat = leftFlat.naturalJoin(rightTable, "I1", "LC1=C1,LC2=C2"); assertTableEquals(noGroupingResult, resultFlat); @@ -401,7 +445,7 @@ public void testNaturalJoinSimpleStatic() { public void testNaturalJoinGroupedStatic() { // noinspection unchecked - testNaturalJoinSimpleStatic(TstUtils::colGrouped); + testNaturalJoinSimpleStatic(TstUtils::colIndexed); } private interface MakeLeftColumn { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableSortTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableSortTest.java index eded694ef80..ca87f38ad82 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableSortTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableSortTest.java @@ -14,6 +14,7 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.testutil.*; import io.deephaven.engine.testutil.generator.*; import io.deephaven.time.DateTimeUtils; @@ -95,25 +96,25 @@ public void testSort() { assertEquals(Arrays.asList("c", "b", "a"), Arrays.asList(DataAccessHelpers.getColumn(result, "DataToSort").get(0, 3))); - final ColumnHolder c1 = TstUtils.colGrouped("Unsorted", 3, 1, 2); + final ColumnHolder c1 = TstUtils.colIndexed("Unsorted", 3, 1, 2); final Table table = newTable(c1, col("DataToSort", "c", "a", "b")); result = table.sort("DataToSort"); assertEquals(Arrays.asList(1, 2, 3), Arrays.asList(DataAccessHelpers.getColumn(result, "Unsorted").get(0, 3))); assertEquals(Arrays.asList("a", "b", "c"), Arrays.asList(DataAccessHelpers.getColumn(result, "DataToSort").get(0, 3))); - final ColumnHolder c11 = TstUtils.colGrouped("Unsorted", 3, 1, 2); + final ColumnHolder c11 = TstUtils.colIndexed("Unsorted", 3, 1, 2); result = newTable(c11, col("DataToSort", "c", "a", "b")).sortDescending("DataToSort"); assertEquals(Arrays.asList(3, 2, 1), Arrays.asList(DataAccessHelpers.getColumn(result, "Unsorted").get(0, 3))); assertEquals(Arrays.asList("c", "b", "a"), Arrays.asList(DataAccessHelpers.getColumn(result, "DataToSort").get(0, 3))); - final ColumnHolder c2 = TstUtils.colGrouped("Unsorted", '3', '1', '2'); + final ColumnHolder c2 = TstUtils.colIndexed("Unsorted", '3', '1', '2'); result = newTable(c2, col("DataToSort", "c", "a", "b")).sort("Unsorted"); assertEquals(Arrays.asList('1', '2', '3'), Arrays.asList(DataAccessHelpers.getColumn(result, "Unsorted").get(0, 3))); assertEquals(Arrays.asList("a", "b", "c"), Arrays.asList(DataAccessHelpers.getColumn(result, "DataToSort").get(0, 3))); - final ColumnHolder c22 = TstUtils.colGrouped("Unsorted", '3', '1', '2'); + final ColumnHolder c22 = TstUtils.colIndexed("Unsorted", '3', '1', '2'); result = newTable(c22, col("DataToSort", "c", "a", "b")).sortDescending("Unsorted"); assertEquals(Arrays.asList('3', '2', '1'), Arrays.asList(DataAccessHelpers.getColumn(result, "Unsorted").get(0, 3))); @@ -159,14 +160,14 @@ public void testSort() { Arrays.asList(DataAccessHelpers.getColumn(result, "Witness").get(0, 4))); - final ColumnHolder c3 = TstUtils.colGrouped("Unsorted", '3', '1', '2', null); + final ColumnHolder c3 = TstUtils.colIndexed("Unsorted", '3', '1', '2', null); result = newTable(c3, col("DataToSort", "c", "a", "b", "d")).sort("Unsorted"); show(result); assertEquals(Arrays.asList(null, '1', '2', '3'), Arrays.asList(DataAccessHelpers.getColumn(result, "Unsorted").get(0, 4))); assertEquals(Arrays.asList("d", "a", "b", "c"), Arrays.asList(DataAccessHelpers.getColumn(result, "DataToSort").get(0, 4))); - final ColumnHolder c4 = TstUtils.colGrouped("Unsorted", '3', '1', null, '2'); + final ColumnHolder c4 = TstUtils.colIndexed("Unsorted", '3', '1', null, '2'); result = newTable(c4, col("DataToSort", "c", "a", "d", "b")).sortDescending("Unsorted"); assertEquals(Arrays.asList('3', '2', '1', null), Arrays.asList(DataAccessHelpers.getColumn(result, "Unsorted").get(0, 4))); @@ -229,7 +230,7 @@ public void testSort2() { public void testGroupedSortRefreshing() { final Table table = testRefreshingTable(RowSetFactory.flat(9).toTracking(), - colGrouped("A", "Apple", "Apple", "Apple", "Banana", "Banana", "Banana", "Canteloupe", "Canteloupe", + colIndexed("A", "Apple", "Apple", "Apple", "Banana", "Banana", "Banana", "Canteloupe", "Canteloupe", "Canteloupe"), col("Secondary", "C", "A", "B", "C", "A", "B", "C", "A", "B")).update("Sentinel=i"); @@ -247,33 +248,84 @@ public void testGroupedSortRefreshing() { "Apple"), col("Secondary", "A", "B", "C", "A", "B", "C", "A", "B", "C"), col("Sentinel", 7, 8, 6, 4, 5, 3, 1, 2, 0)), sorted2); + + // Add a new index covering both columns. + DataIndexer.getOrCreateDataIndex(table, "A", "Secondary"); + + final QueryTable sorted3 = (QueryTable) table.sort(List.of( + SortColumn.desc(ColumnName.of("A")), + SortColumn.asc(ColumnName.of("Secondary")))); + show(sorted3); + assertTableEquals(newTable( + col("A", "Canteloupe", "Canteloupe", "Canteloupe", "Banana", "Banana", "Banana", "Apple", "Apple", + "Apple"), + col("Secondary", "A", "B", "C", "A", "B", "C", "A", "B", "C"), + col("Sentinel", 7, 8, 6, 4, 5, 3, 1, 2, 0)), sorted3); } - public void testGroupedSortHistorical() { - testGroupedSortHistorical(10000); - testGroupedSortHistorical(1000000); + public void testIndexedSortHistorical() { + testIndexedSortHistorical(10000); + testIndexedSortHistorical(1000000); } - private void testGroupedSortHistorical(int size) { + private void testIndexedSortHistorical(int size) { final String[] choices = new String[] {"Hornigold", "Jennings", "Vane", "Bellamy"}; + final String[] letters = new String[] {"D", "C", "A", "B"}; assertEquals(0, size % choices.length); + assertEquals(0, size % letters.length); final String[] values = new String[size]; + final String[] letterValues = new String[size]; for (int ii = 0; ii < values.length; ++ii) { values[ii] = choices[ii % choices.length]; + letterValues[ii] = letters[ii % letters.length]; } - final Table grouped = testTable(RowSetFactory.flat(values.length).toTracking(), - colGrouped("Captain", values)).update("Sentinel=i"); - final Table nogroups = testTable(RowSetFactory.flat(values.length).toTracking(), - col("Captain", values)).update("Sentinel=i"); - - final Table sortedGrouped = grouped.sortDescending("Captain"); - final Table sortedNoGroups = nogroups.sortDescending("Captain"); - show(sortedGrouped); - - assertTableEquals(sortedNoGroups, sortedGrouped); + // Single column index on "Captain" + Table indexed = testTable(RowSetFactory.flat(values.length).toTracking(), + colIndexed("Captain", values), + col("Secondary", values)).update("Sentinel=i"); + Table nogroups = testTable(RowSetFactory.flat(values.length).toTracking(), + col("Captain", values), + col("Secondary", values)).update("Sentinel=i"); + + Table sortedIndexed = indexed.sortDescending("Captain"); + Table sortedNoGroups = nogroups.sortDescending("Captain"); + show(sortedIndexed); + assertTableEquals(sortedNoGroups, sortedIndexed); + + // Single column indexes on both "Captain" and "Secondary" + indexed = testTable(RowSetFactory.flat(values.length).toTracking(), + colIndexed("Captain", values), + colIndexed("Secondary", values)).update("Sentinel=i"); + nogroups = testTable(RowSetFactory.flat(values.length).toTracking(), + col("Captain", values), + col("Secondary", values)).update("Sentinel=i"); + + sortedIndexed = indexed.sortDescending("Captain", "Secondary"); + sortedNoGroups = nogroups.sortDescending("Captain", "Secondary"); + show(sortedIndexed); + assertTableEquals(sortedNoGroups, sortedIndexed); + + sortedIndexed = indexed.sortDescending("Secondary", "Captain"); + sortedNoGroups = nogroups.sortDescending("Secondary", "Captain"); + show(sortedIndexed); + assertTableEquals(sortedNoGroups, sortedIndexed); + + // Multi-column indexes on "Captain" and "Secondary" + indexed = testTable(RowSetFactory.flat(values.length).toTracking(), + col("Captain", values), + col("Secondary", values)).update("Sentinel=i"); + DataIndexer.getOrCreateDataIndex(indexed, "Captain", "Secondary"); + nogroups = testTable(RowSetFactory.flat(values.length).toTracking(), + col("Captain", values), + col("Secondary", values)).update("Sentinel=i"); + + sortedIndexed = indexed.sortDescending("Captain", "Secondary"); + sortedNoGroups = nogroups.sortDescending("Captain", "Secondary"); + show(sortedIndexed); + assertTableEquals(sortedNoGroups, sortedIndexed); } public void testSortBool() { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableStaticNaturalJoinRandomTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableStaticNaturalJoinRandomTest.java index a9b9bb8ea97..f1b53b28416 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableStaticNaturalJoinRandomTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableStaticNaturalJoinRandomTest.java @@ -85,7 +85,7 @@ private static void testNaturalJoinRandomStatic(int seed, int leftSize, int righ rightGenerator, new IntGenerator())); final List leftKeyAttributes = - grouped ? Collections.singletonList(ColumnInfo.ColAttributes.Grouped) + grouped ? Collections.singletonList(ColumnInfo.ColAttributes.Indexed) : Collections.emptyList(); final QueryTable leftTable = getTable(false, leftSize, random, initColumnInfos(new String[] {"JoinKey", "LeftSentinel"}, diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTest.java index 0d6b68416ae..b9ee758b68f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTest.java @@ -21,14 +21,12 @@ import io.deephaven.engine.liveness.SingletonLivenessManager; import io.deephaven.engine.rowset.*; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; -import io.deephaven.engine.table.impl.locations.GroupingProvider; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.remote.ConstructSnapshot; import io.deephaven.engine.table.impl.remote.InitialSnapshotTable; import io.deephaven.engine.table.impl.select.*; import io.deephaven.engine.table.impl.select.MatchFilter.CaseSensitivity; import io.deephaven.engine.table.impl.select.MatchFilter.MatchType; -import io.deephaven.engine.table.impl.sources.DeferredGroupingColumnSource; import io.deephaven.engine.table.impl.sources.LongAsInstantColumnSource; import io.deephaven.engine.table.impl.sources.NullValueColumnSource; import io.deephaven.engine.table.impl.util.BarrageMessage; @@ -52,7 +50,6 @@ import org.apache.commons.lang3.mutable.MutableObject; import org.apache.groovy.util.Maps; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import org.junit.Assert; import org.junit.experimental.categories.Category; @@ -936,6 +933,40 @@ public void testStringContainsFilter() { } } + public void testStringMatchFilterIndexed() { + // MatchFilters (currently) only use indexes on initial creation but this incremental test will recreate + // index-enabled match filtered tables and compare them against incremental non-indexed filtered tables. + + Function filter = ConditionFilter::createConditionFilter; + final Random random = new Random(0); + + final int size = 500; + + final ColumnInfo[] columnInfo; + final QueryTable table = getTable(size, random, columnInfo = initColumnInfos(new String[] {"S1", "S2"}, + new SetGenerator<>("aa", "bb", "cc", "dd", "AA", "BB", "CC", "DD"), + new SetGenerator<>("aaa", "bbb", "ccc", "ddd", "AAA", "BBB", "CCC", "DDD"))); + + DataIndexer.getOrCreateDataIndex(table, "S1"); + DataIndexer.getOrCreateDataIndex(table, "S2"); + + final EvalNuggetInterface[] en = new EvalNuggetInterface[] { + EvalNugget.from(() -> table.where("S1 in 'aa'")), + EvalNugget.from(() -> table.where("S2 in 'bbb'")), + EvalNugget.from(() -> table.where("S2 not in 'ccc', 'dddd'")), + EvalNugget.from(() -> table.where("S1 not in 'aa', 'bb'")), + + EvalNugget.from(() -> table.where("S1 icase in 'aa'")), + EvalNugget.from(() -> table.where("S2 icase in 'bbb'")), + EvalNugget.from(() -> table.where("S2 icase not in 'ccc', 'dddd'")), + EvalNugget.from(() -> table.where("S1 icase not in 'aa', 'bb'")), + }; + + for (int i = 0; i < 500; i++) { + simulateShiftAwareStep(size, random, table, columnInfo, en); + } + } + public void testDoubleRangeFilterSimple() { final Table t = TableTools.newTable(doubleCol("DV", 1.0, 2.0, -3.0, Double.NaN, QueryConstants.NULL_DOUBLE, 6.0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 9.0)).update("IV=i+1"); @@ -3234,12 +3265,10 @@ public Thread newThread(Runnable runnable) { public void testWhereInGrouped() throws IOException { diskBackedTestHarness(t -> { - final ColumnSource symbol = t.getColumnSource("Symbol"); - // noinspection unchecked,rawtypes - final Map gtr = (Map) RowSetIndexer.of(t.getRowSet()).getGrouping(symbol); - ((AbstractColumnSource) symbol).setGroupToRange(gtr); - final Table result = - t.whereIn(t.where("Truthiness=true"), "Symbol", "Timestamp"); + // Create the data index by asking for it. + DataIndexer.getOrCreateDataIndex(t, "Symbol"); + + final Table result = t.whereIn(t.where("Truthiness=true"), "Symbol", "Timestamp"); TableTools.showWithRowSet(result); }); } @@ -3249,7 +3278,7 @@ private void diskBackedTestHarness(Consumer testFunction) throws IOExcept final TableDefinition definition = TableDefinition.of( ColumnDefinition.ofInt("Sentinel"), - ColumnDefinition.ofString("Symbol").withGrouping(), + ColumnDefinition.ofString("Symbol"), ColumnDefinition.ofTime("Timestamp"), ColumnDefinition.ofBoolean("Truthiness")); @@ -3266,6 +3295,10 @@ private void diskBackedTestHarness(Consumer
testFunction) throws IOExcept .updateView("Sentinel=i", "Symbol=syms[i % syms.length]", "Timestamp=baseTime+dateOffset[i]*3600L*1000000000L", "Truthiness=booleans[i]") .groupBy("Symbol").ungroup(); + + // Create the index for "Symbol" column. + DataIndexer.getOrCreateDataIndex(source, "Symbol"); + testDirectory.mkdirs(); final File dest = new File(testDirectory, "Table.parquet"); try { @@ -3489,34 +3522,10 @@ public void testRegressionIssue544() { private static class TestInstantGroupingSource extends LongAsInstantColumnSource - implements DeferredGroupingColumnSource { - - final GroupingProvider groupingProvider = new GroupingProvider<>() { - @Override - public Map getGroupToRange() { - return null; - } - - @Override - public Pair, Boolean> getGroupToRange(RowSet hint) { - return null; - } - }; + implements ColumnSource { TestInstantGroupingSource(ColumnSource realSource) { super(realSource); - // noinspection unchecked,rawtypes - ((DeferredGroupingColumnSource) realSource).setGroupingProvider(groupingProvider); - } - - @Override - public GroupingProvider getGroupingProvider() { - return null; - } - - @Override - public void setGroupingProvider(@Nullable GroupingProvider groupingProvider) { - throw new UnsupportedOperationException(); } } @@ -3532,12 +3541,20 @@ public void testDeferredGroupingPropagationInstantCol() { final TestInstantGroupingSource cs = new TestInstantGroupingSource(colSource(0L, 1, 2, 3)); final Map> columns = Maps.of("T", cs); final QueryTable t1 = new QueryTable(rowSet, columns); + + // Create an index for "T" + DataIndexer.getOrCreateDataIndex(t1, "T"); + final Table t2 = t1.select("T"); // noinspection rawtypes - final DeferredGroupingColumnSource result = - (DeferredGroupingColumnSource) t2.getColumnSource("T").reinterpret(long.class); - assertSame(cs.groupingProvider, result.getGroupingProvider()); + final ColumnSource result = t2.getColumnSource("T"); + final ColumnSource reinterpreted = result.reinterpret(long.class); + + assertTrue(DataIndexer.of(t1.getRowSet()).hasDataIndex(cs)); + assertTrue(DataIndexer.of(t2.getRowSet()).hasDataIndex(result)); + + assertFalse(DataIndexer.of(t2.getRowSet()).hasDataIndex(reinterpreted)); } private static void validateUpdates(final Table table) { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java index 96afabfa638..429ca7ccd37 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java @@ -13,23 +13,24 @@ import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.context.QueryScope; import io.deephaven.engine.exceptions.CancellationException; -import io.deephaven.engine.table.impl.sources.RowKeyColumnSource; import io.deephaven.engine.exceptions.TableInitializationException; -import io.deephaven.engine.util.TableTools; -import io.deephaven.engine.table.impl.verify.TableAssertions; -import io.deephaven.engine.table.impl.select.*; -import io.deephaven.engine.table.impl.chunkfilter.IntRangeComparator; -import io.deephaven.engine.table.impl.sources.UnionRedirection; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ShiftObliviousListener; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; +import io.deephaven.engine.table.impl.chunkfilter.IntRangeComparator; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.table.impl.select.*; +import io.deephaven.engine.table.impl.sources.RowKeyColumnSource; +import io.deephaven.engine.table.impl.sources.UnionRedirection; +import io.deephaven.engine.table.impl.verify.TableAssertions; import io.deephaven.engine.testutil.*; import io.deephaven.engine.testutil.QueryTableTestBase.TableComparator; import io.deephaven.engine.testutil.generator.*; import io.deephaven.engine.testutil.junit4.EngineCleanup; +import io.deephaven.engine.util.TableTools; import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; import io.deephaven.time.DateTimeUtils; @@ -231,13 +232,38 @@ public void testWhereOneOfTwo() { @Test public void testWhereInDependency() { + testWhereInDependencyInternal(false, false); + } + + @Test + public void testWhereInDependencyIndexed() { + testWhereInDependencyInternal(true, false); + testWhereInDependencyInternal(false, true); + testWhereInDependencyInternal(true, true); + } + + private void testWhereInDependencyInternal(boolean filterIndexed, boolean setIndexed) { final QueryTable tableToFilter = testRefreshingTable(i(10, 11, 12, 13, 14, 15).toTracking(), col("A", 1, 2, 3, 4, 5, 6), col("B", 2, 4, 6, 8, 10, 12), col("C", 'a', 'b', 'c', 'd', 'e', 'f')); + if (filterIndexed) { + DataIndexer.getOrCreateDataIndex(tableToFilter, "A"); + DataIndexer.getOrCreateDataIndex(tableToFilter, "B"); + } final QueryTable setTable = testRefreshingTable(i(100, 101, 102).toTracking(), col("A", 1, 2, 3), col("B", 2, 4, 6)); final Table setTable1 = setTable.where("A > 2"); final Table setTable2 = setTable.where("B > 6"); + if (setIndexed) { + DataIndexer.getOrCreateDataIndex(setTable, "A"); + DataIndexer.getOrCreateDataIndex(setTable, "B"); + + DataIndexer.getOrCreateDataIndex(setTable1, "A"); + DataIndexer.getOrCreateDataIndex(setTable1, "B"); + + DataIndexer.getOrCreateDataIndex(setTable2, "A"); + DataIndexer.getOrCreateDataIndex(setTable2, "B"); + } final DynamicWhereFilter dynamicFilter1 = new DynamicWhereFilter((QueryTable) setTable1, true, MatchPairFactory.getExpressions("A")); @@ -325,27 +351,52 @@ public void testWhereDynamicIn() { @Test public void testWhereDynamicInIncremental() { + testWhereDynamicIncrementalInternal(false, false); + } + + @Test + public void testWhereDynamicInIncrementalIndexed() { + testWhereDynamicIncrementalInternal(true, false); + testWhereDynamicIncrementalInternal(false, true); + testWhereDynamicIncrementalInternal(true, true); + } + + private static void testWhereDynamicIncrementalInternal(boolean filterIndexed, boolean setIndexed) { final ColumnInfo[] setInfo; final ColumnInfo[] filteredInfo; - final int setSize = 10; - final int filteredSize = 500; + final int setSize = 100; + final int filteredSize = 5000; final Random random = new Random(0); + final String[] columnNames = + new String[] {"Sym", "intCol", "doubleCol", "charCol", "byteCol", "floatCol", "longCol", "shortCol"}; + final QueryTable setTable = getTable(setSize, random, setInfo = initColumnInfos( - new String[] {"Sym", "intCol", "doubleCol", "charCol", "byteCol", "floatCol", "longCol", "shortCol"}, - new SetGenerator<>("aa", "bb", "bc", "cc", "dd"), - new IntGenerator(-100, 100), + columnNames, + new SetGenerator<>("aa", "bb"), + new IntGenerator(0, 10), new DoubleGenerator(0, 100), new SetGenerator<>('a', 'b', 'c', 'd', 'e', 'f'), new ByteGenerator((byte) 0, (byte) 64), new SetGenerator<>(1.0f, 2.0f, 3.3f, null), new LongGenerator(0, 1000), new ShortGenerator((short) 500, (short) 600))); + if (setIndexed) { + // Add an index on every column but "doubleCol" + for (final String columnName : columnNames) { + if (!columnName.equals("doubleCol")) { + DataIndexer.getOrCreateDataIndex(setTable, columnName); + } + } + // Add the multi-column index for "Sym", "intCol" + DataIndexer.getOrCreateDataIndex(setTable, "Sym", "intCol"); + } + final QueryTable filteredTable = getTable(filteredSize, random, filteredInfo = initColumnInfos( - new String[] {"Sym", "intCol", "doubleCol", "charCol", "byteCol", "floatCol", "longCol", "shortCol"}, - new SetGenerator<>("aa", "bb", "bc", "cc", "dd", "ee", "ff", "gg", "hh", "ii"), - new IntGenerator(-100, 100), + columnNames, + new SetGenerator<>("aa", "bb", "cc", "dd"), + new IntGenerator(0, 20), new DoubleGenerator(0, 100), new CharGenerator('a', 'z'), new ByteGenerator((byte) 0, (byte) 127), @@ -353,11 +404,22 @@ public void testWhereDynamicInIncremental() { new LongGenerator(1500, 2500), new ShortGenerator((short) 400, (short) 700))); + if (filterIndexed) { + // Add an index on every column but "doubleCol" + for (final String columnName : columnNames) { + if (!columnName.equals("doubleCol")) { + DataIndexer.getOrCreateDataIndex(filteredTable, columnName); + } + } + // Add the multi-column index for "Sym", "intCol" + DataIndexer.getOrCreateDataIndex(filteredTable, "Sym", "intCol"); + } final EvalNugget[] en = new EvalNugget[] { EvalNugget.from(() -> filteredTable.whereIn(setTable, "Sym")), EvalNugget.from(() -> filteredTable.whereNotIn(setTable, "Sym")), EvalNugget.from(() -> filteredTable.whereIn(setTable, "Sym", "intCol")), + EvalNugget.from(() -> filteredTable.whereIn(setTable, "intCol", "Sym")), EvalNugget.from(() -> filteredTable.whereIn(setTable, "charCol")), EvalNugget.from(() -> filteredTable.whereIn(setTable, "byteCol")), EvalNugget.from(() -> filteredTable.whereIn(setTable, "shortCol")), @@ -365,6 +427,7 @@ public void testWhereDynamicInIncremental() { EvalNugget.from(() -> filteredTable.whereIn(setTable, "longCol")), EvalNugget.from(() -> filteredTable.whereIn(setTable, "floatCol")), EvalNugget.from(() -> filteredTable.whereNotIn(setTable, "Sym", "intCol")), + EvalNugget.from(() -> filteredTable.whereNotIn(setTable, "intCol", "Sym")), EvalNugget.from(() -> filteredTable.whereNotIn(setTable, "charCol")), EvalNugget.from(() -> filteredTable.whereNotIn(setTable, "byteCol")), EvalNugget.from(() -> filteredTable.whereNotIn(setTable, "shortCol")), @@ -396,6 +459,73 @@ public void testWhereDynamicInIncremental() { } } + @Test + public void testWhereInDynamicPartialIndexed() { + final ColumnInfo[] setInfo; + final ColumnInfo[] filteredInfo; + + final int setSize = 100; + final int filteredSize = 5000; + final Random random = new Random(0); + + final String[] columnNames = + new String[] {"Sym", "intCol", "doubleCol", "charCol", "byteCol", "floatCol", "longCol", "shortCol"}; + + final QueryTable setTable = getTable(setSize, random, setInfo = initColumnInfos( + columnNames, + new SetGenerator<>("aa", "bb"), + new IntGenerator(0, 10), + new DoubleGenerator(0, 100), + new SetGenerator<>('a', 'b', 'c', 'd', 'e', 'f'), + new ByteGenerator((byte) 0, (byte) 64), + new SetGenerator<>(1.0f, 2.0f, 3.3f, null), + new LongGenerator(0, 1000), + new ShortGenerator((short) 500, (short) 600))); + + final QueryTable filteredTable = getTable(filteredSize, random, filteredInfo = initColumnInfos( + columnNames, + new SetGenerator<>("aa", "bb", "cc", "dd"), + new IntGenerator(0, 20), + new DoubleGenerator(0, 100), + new CharGenerator('a', 'z'), + new ByteGenerator((byte) 0, (byte) 127), + new SetGenerator<>(1.0f, 2.0f, 3.3f, null, 4.4f, 5.5f, 6.6f), + new LongGenerator(1500, 2500), + new ShortGenerator((short) 400, (short) 700))); + + DataIndexer.getOrCreateDataIndex(filteredTable, "Sym"); + DataIndexer.getOrCreateDataIndex(filteredTable, "Sym", "charCol"); + + final EvalNugget[] en = new EvalNugget[] { + EvalNugget.from(() -> filteredTable.whereIn(setTable, "Sym", "intCol")), + EvalNugget.from(() -> filteredTable.whereNotIn(setTable, "Sym", "intCol")), + EvalNugget.from(() -> filteredTable.whereIn(setTable, "Sym", "charCol", "intCol")), + EvalNugget.from(() -> filteredTable.whereNotIn(setTable, "Sym", "charCol", "intCol")), + }; + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + for (int step = 0; step < 100; step++) { + final boolean modSet = random.nextInt(10) < 1; + final boolean modFiltered = random.nextBoolean(); + + updateGraph.runWithinUnitTestCycle(() -> { + if (modSet) { + GenerateTableUpdates.generateShiftAwareTableUpdates(GenerateTableUpdates.DEFAULT_PROFILE, + setSize, random, setTable, setInfo); + } + }); + validate(en); + + updateGraph.runWithinUnitTestCycle(() -> { + if (modFiltered) { + GenerateTableUpdates.generateShiftAwareTableUpdates(GenerateTableUpdates.DEFAULT_PROFILE, + filteredSize, random, filteredTable, filteredInfo); + } + }); + validate(en); + } + } + @Test public void testWhereRefresh() { final Table t1 = TableTools.newTable(col("A", "b", "c", "d")); @@ -1065,6 +1195,25 @@ public void testBigTableInitial() { assertEquals(6_999_999L, DataAccessHelpers.getColumn(result, "A").getLong(result.size() - 1)); } + @Test + public void testBigTableIndexed() { + final Random random = new Random(0); + final int size = 100_000; + + final QueryTable source = getTable(size, random, + initColumnInfos( + new String[] {"A"}, + new LongGenerator(0, 1000, 0.01))); + DataIndexer.getOrCreateDataIndex(source, "A"); + + final Table result = source.where("A >= 600", "A < 700"); + Table sorted = result.sort("A"); + show(sorted); + + Assert.geq(DataAccessHelpers.getColumn(sorted, "A").getLong(0), "lowest value", 600, "600"); + Assert.leq(DataAccessHelpers.getColumn(sorted, "A").getLong(result.size() - 1), "highest value", 699, "699"); + } + @Test public void testFilterErrorInitial() { final QueryTable table = testRefreshingTable( diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java index f14b2a6823a..ee43dbc5ba2 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/SourcePartitionedTableTest.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl; +import io.deephaven.base.verify.AssertionFailure; import io.deephaven.configuration.Configuration; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.primitive.iterator.CloseableIterator; @@ -10,7 +11,6 @@ import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; import io.deephaven.engine.table.impl.locations.InvalidatedRegionException; -import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocationRemovedException; import io.deephaven.engine.testutil.locations.DependentRegistrar; import io.deephaven.engine.testutil.locations.TableBackedTableLocationKey; @@ -204,7 +204,7 @@ public void testRemoveAndFail() { updateGraph.markSourcesRefreshedForUnitTests(); registrar.run(); }, false), errors -> errors.size() == 1 && - FindExceptionCause.isOrCausedBy(errors.get(0), TableDataException.class).isPresent()); + FindExceptionCause.isOrCausedBy(errors.get(0), AssertionFailure.class).isPresent()); getUpdateErrors().clear(); // Then delete it for real diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestConcurrentInstantiation.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestConcurrentInstantiation.java index 28c0731b4e4..4af3c713913 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestConcurrentInstantiation.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestConcurrentInstantiation.java @@ -17,6 +17,7 @@ import io.deephaven.engine.table.hierarchical.TreeTable; import io.deephaven.engine.table.impl.hierarchical.TreeTableFilter; import io.deephaven.engine.table.impl.hierarchical.TreeTableImpl; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.remote.ConstructSnapshot; import io.deephaven.engine.table.impl.select.*; import io.deephaven.engine.table.impl.util.ColumnHolder; @@ -55,8 +56,8 @@ @Category(OutOfBandTest.class) public class TestConcurrentInstantiation extends QueryTableTestBase { - private static final int TIMEOUT_LENGTH = 5; - private static final TimeUnit TIMEOUT_UNIT = TimeUnit.MINUTES; + private static final int TIMEOUT_LENGTH = 10; + private static final TimeUnit TIMEOUT_UNIT = TimeUnit.SECONDS; private ExecutorService pool; private ExecutorService dualPool; @@ -296,8 +297,20 @@ public void testDropColumns() throws ExecutionException, InterruptedException, T } public void testWhere() throws ExecutionException, InterruptedException, TimeoutException { + testWhereInternal(false); + } + + public void testWhereIndexed() throws ExecutionException, InterruptedException, TimeoutException { + testWhereInternal(true); + } + + private void testWhereInternal(final boolean indexed) + throws ExecutionException, InterruptedException, TimeoutException { final QueryTable table = TstUtils.testRefreshingTable(i(2, 4, 6).toTracking(), col("x", 1, 2, 3), col("y", "a", "b", "c"), col("z", true, false, true)); + if (indexed) { + DataIndexer.getOrCreateDataIndex(table, "z"); + } final Table tableStart = TstUtils.testRefreshingTable(i(2, 6).toTracking(), col("x", 1, 3), col("y", "a", "c"), col("z", true, true)); @@ -374,24 +387,68 @@ public void testWhere2() throws ExecutionException, InterruptedException, Timeou } public void testWhereDynamic() throws ExecutionException, InterruptedException, TimeoutException { + testWhereDynamicInternal(false, false); + } + public void testWhereDynamicIndexedSource() throws ExecutionException, InterruptedException, TimeoutException { + testWhereDynamicInternal(true, false); + } + + public void testWhereDynamicIndexedSet() throws ExecutionException, InterruptedException, TimeoutException { + testWhereDynamicInternal(false, true); + } + + public void testWhereDynamicIndexedBoth() throws ExecutionException, InterruptedException, TimeoutException { + testWhereDynamicInternal(true, true); + } + + private void testWhereDynamicInternal(final boolean sourceIndexed, final boolean setIndexed) + throws ExecutionException, InterruptedException, TimeoutException { final QueryTable table = TstUtils.testRefreshingTable(i(2, 4, 6).toTracking(), col("x", 1, 2, 3), col("y", "a", "b", "c"), col("z", true, false, true)); + if (sourceIndexed) { + DataIndexer.getOrCreateDataIndex(table, "z"); + } final Table tableStart = TstUtils.testRefreshingTable(i(2, 6).toTracking(), col("x", 1, 3), col("y", "a", "c"), col("z", true, true)); final Table testUpdate = TstUtils.testRefreshingTable(i(3, 6).toTracking(), col("x", 4, 3), col("y", "d", "c"), col("z", true, true)); final QueryTable whereTable = TstUtils.testRefreshingTable(i(0).toTracking(), col("z", true)); + if (setIndexed) { + DataIndexer.getOrCreateDataIndex(whereTable, "z"); + } + + // This is something of a silly test, so we've "hacked" the DynamicWhereFilter instance to let us initialize + // its DataIndex ahead of the operation so that that the where can proceed without a lock. + // Normally, DynamicWhereFilter is only used from whereIn and whereNotIn, which are not concurrent operations. + final DynamicWhereFilter filter = updateGraph.sharedLock().computeLocked( + () -> { + final DynamicWhereFilter result = + new DynamicWhereFilter(whereTable, true, MatchPairFactory.getExpressions("z")) { + private boolean begun; - final DynamicWhereFilter filter = updateGraph.exclusiveLock().computeLocked( - () -> new DynamicWhereFilter(whereTable, true, MatchPairFactory.getExpressions("z"))); + @Override + public SafeCloseable beginOperation(@NotNull Table sourceTable) { + if (!begun) { + begun = true; + return super.beginOperation(sourceTable); + } + return () -> { + }; + } + }; + // noinspection resource + result.beginOperation(table); + return result; + }); updateGraph.startCycleForUnitTests(false); final Future
future1 = dualPool.submit(() -> table.where(filter)); try { future1.get(TIMEOUT_LENGTH, TIMEOUT_UNIT); - fail("Filtering should be blocked on UGP"); + fail("Filtering should be blocked on UGP because DynamicWhereFilter does not support previous filtering," + + " and so the first where will eventually try to do a locked snapshot"); } catch (TimeoutException ignored) { } TstUtils.addToTable(table, i(2, 3), col("x", 1, 4), col("y", "a", "d"), col("z", false, true)); @@ -410,8 +467,21 @@ public void testWhereDynamic() throws ExecutionException, InterruptedException, } public void testSort() throws ExecutionException, InterruptedException, TimeoutException { + testSortInternal(false); + } + + public void testSortIndexed() throws ExecutionException, InterruptedException, TimeoutException { + testSortInternal(true); + } + + private void testSortInternal(final boolean indexed) + throws ExecutionException, InterruptedException, TimeoutException { final QueryTable table = TstUtils.testRefreshingTable(i(2, 4, 6).toTracking(), col("x", 1, 2, 3), col("y", "a", "b", "c")); + if (indexed) { + DataIndexer.getOrCreateDataIndex(table, "x"); + } + final Table tableStart = TstUtils.testRefreshingTable(i(1, 2, 3).toTracking(), col("x", 3, 2, 1), col("y", "c", "b", "a")); final Table tableUpdate = TstUtils.testRefreshingTable(i(1, 2, 3, 4).toTracking(), @@ -433,6 +503,8 @@ public void testSort() throws ExecutionException, InterruptedException, TimeoutE table.notifyListeners(i(3), i(), i()); updateGraph.markSourcesRefreshedForUnitTests(); + updateGraph.flushAllNormalNotificationsForUnitTests(); + final Table sort3 = pool.submit(() -> table.sortDescending("x")).get(TIMEOUT_LENGTH, TIMEOUT_UNIT); TstUtils.assertTableEquals(tableStart, prevTable(sort1)); @@ -1228,8 +1300,7 @@ private void testByConcurrent(Function function, boolean hasKeys, } // We only care about the silent version of this table, as it's just a vessel to tick and ensure that the - // resultant table - // is computed using the appropriate version. + // resultant table is computed using the appropriate version. final Table expected1 = updateGraph.exclusiveLock().computeLocked( () -> function.apply(table.silent()).select()); final Table expected2 = updateGraph.exclusiveLock() diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestJoinControl.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestJoinControl.java index 50cb26dfd57..6881134035d 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestJoinControl.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestJoinControl.java @@ -4,6 +4,8 @@ package io.deephaven.engine.table.impl; import io.deephaven.engine.table.Table; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * A set of JoinControl objects useful for unit tests. @@ -13,15 +15,19 @@ public class TestJoinControl { static final JoinControl BUILD_LEFT_CONTROL = new JoinControl() { @Override - boolean buildLeft(QueryTable leftTable, Table rightTable) { - return true; + BuildParameters buildParameters( + @NotNull final Table leftTable, @Nullable Table leftDataIndexTable, + @NotNull final Table rightTable, @Nullable Table rightDataIndexTable) { + return new BuildParameters(BuildParameters.From.LeftInput, initialBuildSize()); } }; static final JoinControl BUILD_RIGHT_CONTROL = new JoinControl() { @Override - boolean buildLeft(QueryTable leftTable, Table rightTable) { - return false; + BuildParameters buildParameters( + @NotNull final Table leftTable, @Nullable Table leftDataIndexTable, + @NotNull final Table rightTable, @Nullable Table rightDataIndexTable) { + return new BuildParameters(BuildParameters.From.RightInput, initialBuildSize()); } }; @@ -59,8 +65,10 @@ public double getMaximumLoadFactor() { } @Override - boolean buildLeft(QueryTable leftTable, Table rightTable) { - return true; + BuildParameters buildParameters( + @NotNull final Table leftTable, @Nullable Table leftDataIndexTable, + @NotNull final Table rightTable, @Nullable Table rightDataIndexTable) { + return new BuildParameters(BuildParameters.From.LeftInput, initialBuildSize()); } }; @@ -81,8 +89,10 @@ public double getMaximumLoadFactor() { } @Override - boolean buildLeft(QueryTable leftTable, Table rightTable) { - return false; + BuildParameters buildParameters( + @NotNull final Table leftTable, @Nullable Table leftDataIndexTable, + @NotNull final Table rightTable, @Nullable Table rightDataIndexTable) { + return new BuildParameters(BuildParameters.From.RightInput, initialBuildSize()); } }; } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java index 0c9d43b5366..4800b614fbe 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTable.java @@ -19,13 +19,13 @@ import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.SimpleTableLocationKey; import io.deephaven.engine.table.impl.locations.impl.TableLocationSubscriptionBuffer; -import io.deephaven.engine.table.impl.sources.DeferredGroupingColumnSource; import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.qst.column.Column; import org.jetbrains.annotations.NotNull; import org.jmock.api.Invocation; import org.jmock.lib.action.CustomAction; @@ -33,6 +33,7 @@ import org.junit.Before; import org.junit.Test; +import java.lang.ref.WeakReference; import java.util.*; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -50,8 +51,7 @@ public class TestPartitionAwareSourceTable extends RefreshingTableTestCase { private static final ColumnDefinition PARTITIONING_COLUMN_DEFINITION = ColumnDefinition.ofString("Date").withPartitioning(); private static final ColumnDefinition BOOLEAN_COLUMN_DEFINITION = ColumnDefinition.ofBoolean("Active"); - private static final ColumnDefinition CHARACTER_COLUMN_DEFINITION = - ColumnDefinition.ofChar("Type").withGrouping(); + private static final ColumnDefinition CHARACTER_COLUMN_DEFINITION = ColumnDefinition.ofChar("Type"); private static final ColumnDefinition INTEGER_COLUMN_DEFINITION = ColumnDefinition.ofInt("Size"); private static final ColumnDefinition DOUBLE_COLUMN_DEFINITION = ColumnDefinition.ofDouble("Price"); @@ -70,7 +70,7 @@ public class TestPartitionAwareSourceTable extends RefreshingTableTestCase { private SourceTableComponentFactory componentFactory; private ColumnSourceManager columnSourceManager; - private DeferredGroupingColumnSource[] columnSources; + private ColumnSource[] columnSources; private TableLocationProvider locationProvider; private ImmutableTableLocationKey[] tableLocationKeys; @@ -94,7 +94,7 @@ public void setUp() throws Exception { componentFactory = mock(SourceTableComponentFactory.class); columnSourceManager = mock(ColumnSourceManager.class); columnSources = TABLE_DEFINITION.getColumnStream().map(cd -> { - final DeferredGroupingColumnSource mocked = mock(DeferredGroupingColumnSource.class, cd.getName()); + final ColumnSource mocked = mock(ColumnSource.class, cd.getName()); checking(new Expectations() { { allowing(mocked).getType(); @@ -106,7 +106,7 @@ public void setUp() throws Exception { } }); return mocked; - }).toArray(DeferredGroupingColumnSource[]::new); + }).toArray(ColumnSource[]::new); locationProvider = mock(TableLocationProvider.class); tableLocationKeys = IntStream.range(0, 6).mapToObj(tlki -> { final Map> partitions = new LinkedHashMap<>(); @@ -144,7 +144,11 @@ public void setUp() throws Exception { oneOf(componentFactory).createColumnSourceManager(with(true), with(ColumnToCodecMappings.EMPTY), with(equal(TABLE_DEFINITION.getColumns()))); will(returnValue(columnSourceManager)); - oneOf(columnSourceManager).disableGrouping(); + allowing(columnSourceManager).tryRetainReference(); + will(returnValue(true)); + allowing(columnSourceManager).getWeakReference(); + will(returnValue(new WeakReference<>(columnSourceManager))); + allowing(columnSourceManager).dropReference(); } }); @@ -168,7 +172,7 @@ public void tearDown() throws Exception { } } - private Map> getIncludedColumnsMap(final int... indices) { + private Map> getIncludedColumnsMap(final int... indices) { return IntStream.of(indices) .mapToObj(ci -> new Pair<>(TABLE_DEFINITION.getColumns().get(ci).getName(), columnSources[ci])) .collect(Collectors.toMap(Pair::getFirst, Pair::getSecond, Assert::neverInvoked, LinkedHashMap::new)); @@ -235,7 +239,7 @@ private void doInitializeCheck(final ImmutableTableLocationKey[] tableLocationKe final TableDataException exception = new TableDataException("test"); final RowSet toAdd = RowSetFactory.fromRange(expectedRowSet.lastRowKey() + 1, - expectedRowSet.lastRowKey() + INDEX_INCREMENT); + expectedRowSet.lastRowKey() + INDEX_INCREMENT).toTracking(); checking(new Expectations() { { @@ -248,7 +252,7 @@ public Object invoke(Invocation invocation) { return null; } }); - oneOf(columnSourceManager).refresh(); + oneOf(columnSourceManager).initialize(); if (throwException) { will(throwException(exception)); } else { @@ -378,10 +382,12 @@ private void doRefreshExceptionCheck() { will(new CustomAction("check exception") { @Override public Object invoke(Invocation invocation) { - assertEquals(exception, ((Exception) invocation.getParameter(0)).getCause()); + assertEquals(exception, invocation.getParameter(0)); return errorNotification; } }); + oneOf(columnSourceManager).deliverError(with(any(TableDataException.class)), + with(any(PerformanceEntry.class))); } }); @@ -448,7 +454,6 @@ private void doTestRedefinition() { oneOf(componentFactory).createColumnSourceManager(with(true), with(ColumnToCodecMappings.EMPTY), with(equal(includedColumns1))); will(returnValue(columnSourceManager)); - oneOf(columnSourceManager).disableGrouping(); } }); final Table dropColumnsResult1 = SUT.dropColumns(BOOLEAN_COLUMN_DEFINITION.getName()); @@ -464,8 +469,8 @@ public Object invoke(Invocation invocation) { return null; } }); - oneOf(columnSourceManager).refresh(); - will(returnValue(RowSetFactory.empty())); + oneOf(columnSourceManager).initialize(); + will(returnValue(RowSetFactory.empty().toTracking())); oneOf(columnSourceManager).getColumnSources(); will(returnValue(includedColumns1.stream() .collect(Collectors.toMap(ColumnDefinition::getName, @@ -490,7 +495,6 @@ public Object invoke(Invocation invocation) { oneOf(componentFactory).createColumnSourceManager(with(true), with(ColumnToCodecMappings.EMPTY), with(equal(includedColumns2))); will(returnValue(columnSourceManager)); - oneOf(columnSourceManager).disableGrouping(); } }); final Table dropColumnsResult2 = dropColumnsResult1.dropColumns(CHARACTER_COLUMN_DEFINITION.getName()); @@ -506,8 +510,8 @@ public Object invoke(Invocation invocation) { return null; } }); - oneOf(columnSourceManager).refresh(); - will(returnValue(RowSetFactory.empty())); + oneOf(columnSourceManager).initialize(); + will(returnValue(RowSetFactory.empty().toTracking())); oneOf(columnSourceManager).getColumnSources(); will(returnValue(includedColumns2.stream() .collect(Collectors.toMap(ColumnDefinition::getName, @@ -541,7 +545,6 @@ public Object invoke(Invocation invocation) { oneOf(componentFactory).createColumnSourceManager(with(true), with(ColumnToCodecMappings.EMPTY), with(equal(includedColumns3))); will(returnValue(columnSourceManager)); - oneOf(columnSourceManager).disableGrouping(); } }); final Table viewResult1 = dropColumnsResult2.view(INTEGER_COLUMN_DEFINITION.getName()); @@ -557,8 +560,8 @@ public Object invoke(Invocation invocation) { return null; } }); - oneOf(columnSourceManager).refresh(); - will(returnValue(RowSetFactory.empty())); + oneOf(columnSourceManager).initialize(); + will(returnValue(RowSetFactory.empty().toTracking())); oneOf(columnSourceManager).getColumnSources(); will(returnValue(includedColumns3.stream() .collect(Collectors.toMap(ColumnDefinition::getName, @@ -599,22 +602,16 @@ public void testSelectDistinctDate() { final String[] expectedDistinctDates = IntStream.of(1, 3, 5).mapToObj(li -> COLUMN_PARTITIONS[li]).distinct().toArray(String[]::new); doInitializeCheck(locationKeysSlice(1, 3, 5), passedLocations, false, true); - passedLocations.forEach(tl -> checking(new Expectations() { - { - oneOf(tl).refresh(); - oneOf(tl).getSize(); - will(returnValue(1L)); - } - })); checking(new Expectations() { { - oneOf(columnSourceManager).allLocations(); - will(returnValue(passedLocations)); + oneOf(columnSourceManager).locationTable(); + will(returnValue(TableFactory.newTable( + Column.of(PARTITIONING_COLUMN_DEFINITION.getName(), + IntStream.of(1, 3, 5).mapToObj(li -> COLUMN_PARTITIONS[li]).toArray(String[]::new))))); } }); final Table result = SUT.selectDistinct(PARTITIONING_COLUMN_DEFINITION.getName()); assertIsSatisfied(); - // noinspection unchecked final DataColumn distinctDateColumn = DataAccessHelpers.getColumn(result, PARTITIONING_COLUMN_DEFINITION.getName()); assertEquals(expectedDistinctDates.length, distinctDateColumn.size()); @@ -630,8 +627,8 @@ public void testSelectDistinctOther() { { oneOf(locationProvider).subscribe(with(any(TableLocationSubscriptionBuffer.class))); // noinspection resource - oneOf(columnSourceManager).refresh(); - will(returnValue(RowSetFactory.empty())); + oneOf(columnSourceManager).initialize(); + will(returnValue(RowSetFactory.empty().toTracking())); allowing(columnSourceManager).getColumnSources(); will(returnValue(getIncludedColumnsMap(0, 1, 2, 3, 4))); } @@ -649,7 +646,6 @@ public void testWhereDate() { oneOf(componentFactory).createColumnSourceManager(true, ColumnToCodecMappings.EMPTY, TABLE_DEFINITION.getColumns()); will(returnValue(columnSourceManager)); - oneOf(columnSourceManager).disableGrouping(); } }); assertRowSetEquals(expectedRowSet, SUT.where(PARTITIONING_COLUMN_DEFINITION.getName() + "=`D0`").getRowSet()); @@ -711,7 +707,6 @@ public void testWhereDateSize() { oneOf(componentFactory).createColumnSourceManager(true, ColumnToCodecMappings.EMPTY, TABLE_DEFINITION.getColumns()); will(returnValue(columnSourceManager)); - oneOf(columnSourceManager).disableGrouping(); allowing(columnSources[3]).getInt(with(any(long.class))); will(returnValue(1)); allowing(columnSources[3]).makeGetContext(with(any(Integer.class))); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitioningColumns.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitioningColumns.java index f7c9e2b6aaa..aee52c78e26 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitioningColumns.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitioningColumns.java @@ -3,11 +3,16 @@ // package io.deephaven.engine.table.impl; +import io.deephaven.api.SortColumn; import io.deephaven.api.filter.Filter; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.primitive.iterator.CloseableIterator; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.dataindex.DataIndexUtils; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.table.iterators.ChunkedColumnIterator; import io.deephaven.engine.testutil.TstUtils; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.time.DateTimeUtils; @@ -18,7 +23,6 @@ import io.deephaven.engine.table.impl.locations.impl.*; import io.deephaven.engine.table.impl.select.MatchFilter; import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.sources.regioned.*; import io.deephaven.engine.rowset.RowSetFactory; import org.jetbrains.annotations.NotNull; @@ -86,6 +90,39 @@ public void testEverything() { null), null); + for (String colName : partitionKeys) { + final DataIndex fullIndex = DataIndexer.getDataIndex(result, colName); + Assert.neqNull(fullIndex, "fullIndex"); + + final ColumnSource[] columns = new ColumnSource[] {result.getColumnSource(colName)}; + + final DataIndex.RowKeyLookup fullIndexRowKeyLookup = fullIndex.rowKeyLookup(columns); + final ColumnSource fullIndexRowSetColumn = fullIndex.rowSetColumn(); + + ChunkSource.WithPrev tableKeys = DataIndexUtils.makeBoxedKeySource(columns); + + // Iterate through the entire source table and verify the lookup row set is valid and contains this row. + try (final RowSet.Iterator rsIt = result.getRowSet().iterator(); + final CloseableIterator keyIt = + ChunkedColumnIterator.make(tableKeys, result.getRowSet())) { + + while (rsIt.hasNext() && keyIt.hasNext()) { + final long rowKey = rsIt.nextLong(); + final Object key = keyIt.next(); + + // Verify the row sets at the lookup keys match. + final long fullRowKey = fullIndexRowKeyLookup.apply(key, false); + Assert.geqZero(fullRowKey, "fullRowKey"); + + final RowSet fullRowSet = fullIndexRowSetColumn.get(fullRowKey); + Assert.neqNull(fullRowSet, "fullRowSet"); + + Assert.eqTrue(fullRowSet.containsRange(rowKey, rowKey), "fullRowSet.containsRange(rowKey, rowKey)"); + } + } + + } + final Table expected = input.sort(input.getDefinition().getColumnNamesArray()); TstUtils.assertTableEquals(expected, result); @@ -99,19 +136,33 @@ public void testEverything() { private static final class DummyTableLocation extends AbstractTableLocation { - protected DummyTableLocation(@NotNull final TableKey tableKey, + private DummyTableLocation(@NotNull final TableKey tableKey, @NotNull final TableLocationKey tableLocationKey) { super(tableKey, tableLocationKey, false); } @Override - public void refresh() { + public void refresh() {} + + @Override + public @NotNull List getSortedColumns() { + return List.of(); + } + + @Override + @NotNull + public List getDataIndexColumns() { + return List.of(); + } + @Override + public boolean hasDataIndex(@NotNull final String... columns) { + return false; } @NotNull @Override - protected ColumnLocation makeColumnLocation(@NotNull String name) { + protected ColumnLocation makeColumnLocation(@NotNull final String name) { return new ColumnLocation() { @NotNull @Override @@ -132,59 +183,64 @@ public boolean exists() { @Nullable @Override - public METADATA_TYPE getMetadata(@NotNull ColumnDefinition columnDefinition) { + public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition columnDefinition) { throw new UnsupportedOperationException(); } @Override public ColumnRegionChar makeColumnRegionChar( - @NotNull ColumnDefinition columnDefinition) { + @NotNull final ColumnDefinition columnDefinition) { throw new UnsupportedOperationException(); } @Override public ColumnRegionByte makeColumnRegionByte( - @NotNull ColumnDefinition columnDefinition) { + @NotNull final ColumnDefinition columnDefinition) { throw new UnsupportedOperationException(); } @Override public ColumnRegionShort makeColumnRegionShort( - @NotNull ColumnDefinition columnDefinition) { + @NotNull final ColumnDefinition columnDefinition) { throw new UnsupportedOperationException(); } @Override public ColumnRegionInt makeColumnRegionInt( - @NotNull ColumnDefinition columnDefinition) { + @NotNull final ColumnDefinition columnDefinition) { throw new UnsupportedOperationException(); } @Override public ColumnRegionLong makeColumnRegionLong( - @NotNull ColumnDefinition columnDefinition) { + @NotNull final ColumnDefinition columnDefinition) { throw new UnsupportedOperationException(); } @Override public ColumnRegionFloat makeColumnRegionFloat( - @NotNull ColumnDefinition columnDefinition) { + @NotNull final ColumnDefinition columnDefinition) { throw new UnsupportedOperationException(); } @Override public ColumnRegionDouble makeColumnRegionDouble( - @NotNull ColumnDefinition columnDefinition) { + @NotNull final ColumnDefinition columnDefinition) { throw new UnsupportedOperationException(); } @Override public ColumnRegionObject makeColumnRegionObject( - @NotNull ColumnDefinition columnDefinition) { + @NotNull final ColumnDefinition columnDefinition) { throw new UnsupportedOperationException(); } }; } + + @Override + protected @Nullable BasicDataIndex loadDataIndex(@NotNull final String... columns) { + throw new UnsupportedOperationException(); + } } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSelectPreserveGrouping.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSelectPreserveGrouping.java index 1965f18f86f..a5423ffc31a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSelectPreserveGrouping.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSelectPreserveGrouping.java @@ -7,7 +7,7 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.context.QueryScope; import io.deephaven.engine.testutil.QueryTableTestBase; import io.deephaven.engine.testutil.TstUtils; @@ -58,40 +58,38 @@ private static void cleanupPersistence(String root) { } public void testPreserveGrouping() { - final Table x = TstUtils.testTable(TstUtils.colGrouped("Sym", "AAPL", "AAPL", "BRK", "BRK", "TSLA", "TLSA"), + final Table x = TstUtils.testTable(TstUtils.colIndexed("Sym", "AAPL", "AAPL", "BRK", "BRK", "TSLA", "TLSA"), intCol("Sentinel", 1, 2, 3, 4, 5, 6)); - final RowSetIndexer xIndexer = RowSetIndexer.of(x.getRowSet()); - assertTrue(xIndexer.hasGrouping(x.getColumnSource("Sym"))); - assertFalse(xIndexer.hasGrouping(x.getColumnSource("Sentinel"))); + + assertTrue(DataIndexer.hasDataIndex(x, "Sym")); + assertFalse(DataIndexer.hasDataIndex(x, "Sentinel")); QueryScope.addParam("switchColumnValue", 1); final Table xs = x.select("Sym", "SentinelDoubled=Sentinel*2", "Foo=switchColumnValue", "Sentinel"); assertTableEquals(x, xs.view("Sym", "Sentinel")); - final RowSetIndexer xsIndexer = RowSetIndexer.of(xs.getRowSet()); - assertTrue(xsIndexer.hasGrouping(xs.getColumnSource("Sym"))); - assertFalse(xsIndexer.hasGrouping(xs.getColumnSource("SentinelDoubled"))); - assertFalse(xsIndexer.hasGrouping(xs.getColumnSource("Foo"))); - assertFalse(xsIndexer.hasGrouping(xs.getColumnSource("Sentinel"))); + assertTrue(DataIndexer.hasDataIndex(xs, "Sym")); + assertFalse(DataIndexer.hasDataIndex(xs, "SentinelDoubled")); + assertFalse(DataIndexer.hasDataIndex(xs, "Foo")); + assertFalse(DataIndexer.hasDataIndex(xs, "Sentinel")); final Table x2 = TstUtils.testTable(TstUtils.i(0, 1 << 16, 2 << 16, 3 << 16, 4 << 16, 5 << 16).toTracking(), - TstUtils.colGrouped("Sym", "AAPL", "AAPL", "BRK", "BRK", "TSLA", "TLSA"), + TstUtils.colIndexed("Sym", "AAPL", "AAPL", "BRK", "BRK", "TSLA", "TLSA"), intCol("Sentinel", 1, 2, 3, 4, 5, 6)); final Table xu = x2.update("Sym2=Sym"); assertTableEquals(x2, xu.view("Sym=Sym2", "Sentinel")); - final RowSetIndexer xuIndexer = RowSetIndexer.of(xu.getRowSet()); - assertTrue(xuIndexer.hasGrouping(xu.getColumnSource("Sym"))); - assertTrue(xuIndexer.hasGrouping(xu.getColumnSource("Sym2"))); - assertFalse(xuIndexer.hasGrouping(xu.getColumnSource("Sentinel"))); + assertTrue(DataIndexer.hasDataIndex(xu, "Sym")); + assertTrue(DataIndexer.hasDataIndex(xu, "Sym2")); + assertFalse(DataIndexer.hasDataIndex(xu, "Sentinel")); } public void testPreserveDeferredGrouping() throws IOException { final File testDirectory = Files.createTempDirectory("DeferredGroupingTest").toFile(); final File dest = new File(testDirectory, "Table.parquet"); try { - final ColumnHolder symHolder = TstUtils.colGrouped("Sym", "AAPL", "AAPL", "BRK", "BRK", "TSLA", "TLSA"); + final ColumnHolder symHolder = TstUtils.colIndexed("Sym", "AAPL", "AAPL", "BRK", "BRK", "TSLA", "TLSA"); final ColumnHolder sentinelHolder = intCol("Sentinel", 1, 2, 3, 4, 5, 6); final Map> columns = new LinkedHashMap<>(); @@ -99,11 +97,11 @@ public void testPreserveDeferredGrouping() throws IOException { columns.put("Sym", TstUtils.getTestColumnSource(rowSet, symHolder)); columns.put("Sentinel", TstUtils.getTestColumnSource(rowSet, sentinelHolder)); final TableDefinition definition = TableDefinition.of( - ColumnDefinition.ofString("Sym").withGrouping(), + ColumnDefinition.ofString("Sym"), ColumnDefinition.ofInt("Sentinel")); final Table x = new QueryTable(definition, rowSet, columns); - assertTrue(x.getDefinition().getColumn("Sym").isGrouping()); + DataIndexer.getOrCreateDataIndex(x, "Sym"); System.out.println(x.getDefinition()); ParquetTools.writeTable(x, dest); @@ -111,25 +109,23 @@ public void testPreserveDeferredGrouping() throws IOException { final Table readBack = ParquetTools.readTable(dest); TableTools.showWithRowSet(readBack); - assertTrue(RowSetIndexer.of(readBack.getRowSet()).hasGrouping(readBack.getColumnSource("Sym"))); + assertTrue(DataIndexer.hasDataIndex(readBack, "Sym")); final Table xs = x.select("Sym", "Sentinel=Sentinel*2", "Foo=Sym", "Sent2=Sentinel"); - final RowSetIndexer xsIndexer = RowSetIndexer.of(xs.getRowSet()); - assertTrue(xsIndexer.hasGrouping(xs.getColumnSource("Sym"))); - assertTrue(xsIndexer.hasGrouping(xs.getColumnSource("Foo"))); + assertTrue(DataIndexer.hasDataIndex(xs, "Sym")); + assertTrue(DataIndexer.hasDataIndex(xs, "Foo")); assertSame(xs.getColumnSource("Sym"), xs.getColumnSource("Foo")); - assertFalse(xsIndexer.hasGrouping(xs.getColumnSource("Sentinel"))); - assertFalse(xsIndexer.hasGrouping(xs.getColumnSource("Sent2"))); + assertFalse(DataIndexer.hasDataIndex(xs, "Sentinel")); + assertFalse(DataIndexer.hasDataIndex(xs, "Sent2")); final Table xs2 = x.select("Foo=Sym", "Sentinel=Sentinel*2", "Foo2=Foo", "Foo3=Sym"); - final RowSetIndexer xs2Indexer = RowSetIndexer.of(xs.getRowSet()); - assertTrue(xs2Indexer.hasGrouping(xs2.getColumnSource("Foo"))); - assertFalse(xs2Indexer.hasGrouping(xs2.getColumnSource("Sentinel"))); - assertTrue(xs2Indexer.hasGrouping(xs2.getColumnSource("Foo2"))); + assertTrue(DataIndexer.hasDataIndex(xs2, "Foo")); + assertFalse(DataIndexer.hasDataIndex(xs2, "Sentinel")); + assertTrue(DataIndexer.hasDataIndex(xs2, "Foo2")); assertSame(xs2.getColumnSource("Foo2"), xs2.getColumnSource("Foo")); - assertTrue(xs2Indexer.hasGrouping(xs2.getColumnSource("Foo3"))); + assertTrue(DataIndexer.hasDataIndex(xs2, "Foo3")); } finally { FileUtils.deleteRecursively(testDirectory); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java index 6142ced7b12..435077fe280 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestSimpleSourceTable.java @@ -7,6 +7,7 @@ import io.deephaven.base.verify.Assert; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; @@ -14,7 +15,6 @@ import io.deephaven.engine.table.impl.locations.TableLocation; import io.deephaven.engine.table.impl.locations.TableLocationProvider; import io.deephaven.engine.table.impl.locations.impl.StandaloneTableLocationKey; -import io.deephaven.engine.table.impl.sources.DeferredGroupingColumnSource; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; @@ -36,8 +36,7 @@ public class TestSimpleSourceTable extends RefreshingTableTestCase { private static final int NUM_COLUMNS = 4; private static final ColumnDefinition BOOLEAN_COLUMN_DEFINITION = ColumnDefinition.ofBoolean("Active"); - private static final ColumnDefinition CHARACTER_COLUMN_DEFINITION = - ColumnDefinition.ofChar("Type").withGrouping(); + private static final ColumnDefinition CHARACTER_COLUMN_DEFINITION = ColumnDefinition.ofChar("Type"); private static final ColumnDefinition INTEGER_COLUMN_DEFINITION = ColumnDefinition.ofInt("Size"); private static final ColumnDefinition DOUBLE_COLUMN_DEFINITION = ColumnDefinition.ofDouble("Price"); @@ -52,7 +51,7 @@ public class TestSimpleSourceTable extends RefreshingTableTestCase { private SourceTableComponentFactory componentFactory; private ColumnSourceManager columnSourceManager; - private DeferredGroupingColumnSource[] columnSources; + private ColumnSource[] columnSources; private TableLocationProvider locationProvider; private TableLocation tableLocation; @@ -68,8 +67,15 @@ public void setUp() throws Exception { componentFactory = mock(SourceTableComponentFactory.class); columnSourceManager = mock(ColumnSourceManager.class); + checking(new Expectations() { + { + allowing(columnSourceManager).allLocations(); + will(returnValue(Collections.EMPTY_SET)); + } + }); + columnSources = TABLE_DEFINITION.getColumnStream().map(cd -> { - final DeferredGroupingColumnSource mocked = mock(DeferredGroupingColumnSource.class, cd.getName()); + final ColumnSource mocked = mock(ColumnSource.class, cd.getName()); checking(new Expectations() { { allowing(mocked).getType(); @@ -81,7 +87,7 @@ public void setUp() throws Exception { } }); return mocked; - }).toArray(DeferredGroupingColumnSource[]::new); + }).toArray(ColumnSource[]::new); locationProvider = mock(TableLocationProvider.class); tableLocation = mock(TableLocation.class); checking(new Expectations() { @@ -139,7 +145,7 @@ private static String[] getExcludedColumnNames(final TableDefinition currentDef, .toArray(String[]::new); } - private Map> getIncludedColumnsMap(final int... indices) { + private Map> getIncludedColumnsMap(final int... indices) { return IntStream.of(indices) .mapToObj(ci -> new Pair<>(TABLE_DEFINITION.getColumns().get(ci).getName(), columnSources[ci])) .collect(Collectors.toMap(Pair::getFirst, Pair::getSecond, Assert::neverInvoked, LinkedHashMap::new)); @@ -155,19 +161,20 @@ public void testInitializeException() { doSingleLocationInitializeCheck(true, true); } - private void doSingleLocationInitializeCheck(final boolean throwException, + private void doSingleLocationInitializeCheck( + final boolean throwException, @SuppressWarnings("SameParameterValue") final boolean coalesce) { Assert.assertion(!(throwException && !coalesce), "!(throwException && !listen)"); final TableDataException exception = new TableDataException("test"); - final RowSet toAdd = - RowSetFactory.fromRange(expectedRowSet.lastRowKey() + 1, - expectedRowSet.lastRowKey() + INDEX_INCREMENT); + final RowSet toAdd = RowSetFactory.fromRange( + expectedRowSet.lastRowKey() + 1, + expectedRowSet.lastRowKey() + INDEX_INCREMENT).toTracking(); checking(new Expectations() { { oneOf(locationProvider).refresh(); oneOf(columnSourceManager).addLocation(tableLocation); - oneOf(columnSourceManager).refresh(); + oneOf(columnSourceManager).initialize(); if (throwException) { will(throwException(exception)); } else { @@ -189,7 +196,7 @@ private void doSingleLocationInitializeCheck(final boolean throwException, if (throwException) { return; } else { - throw exception; + throw e; } } assertRowSetEquals(expectedRowSet, rowSet); @@ -227,8 +234,8 @@ private void doTestRedefinition() { { oneOf(locationProvider).refresh(); oneOf(columnSourceManager).addLocation(tableLocation); - oneOf(columnSourceManager).refresh(); - will(returnValue(RowSetFactory.empty())); + oneOf(columnSourceManager).initialize(); + will(returnValue(RowSetFactory.empty().toTracking())); oneOf(columnSourceManager).getColumnSources(); will(returnValue(getIncludedColumnsMap(includedColumnIndices1))); } @@ -260,8 +267,8 @@ private void doTestRedefinition() { { oneOf(locationProvider).refresh(); oneOf(columnSourceManager).addLocation(tableLocation); - oneOf(columnSourceManager).refresh(); - will(returnValue(RowSetFactory.empty())); + oneOf(columnSourceManager).initialize(); + will(returnValue(RowSetFactory.empty().toTracking())); oneOf(columnSourceManager).getColumnSources(); will(returnValue(getIncludedColumnsMap(includedColumnIndices2))); } @@ -302,8 +309,8 @@ private void doTestRedefinition() { { oneOf(locationProvider).refresh(); oneOf(columnSourceManager).addLocation(tableLocation); - oneOf(columnSourceManager).refresh(); - will(returnValue(RowSetFactory.empty())); + oneOf(columnSourceManager).initialize(); + will(returnValue(RowSetFactory.empty().toTracking())); oneOf(columnSourceManager).getColumnSources(); will(returnValue(getIncludedColumnsMap(includedColumnIndices3))); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/indexer/TestDataIndexer.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/indexer/TestDataIndexer.java new file mode 100644 index 00000000000..a79ed9cb6e1 --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/indexer/TestDataIndexer.java @@ -0,0 +1,231 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.indexer; + +import com.google.common.collect.Sets; +import io.deephaven.base.verify.Require; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.testutil.ColumnInfo; +import io.deephaven.engine.testutil.ControlledUpdateGraph; +import io.deephaven.engine.testutil.TstUtils; +import io.deephaven.engine.testutil.generator.IntGenerator; +import io.deephaven.engine.testutil.generator.SetGenerator; +import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; +import io.deephaven.engine.testutil.EvalNugget; +import io.deephaven.engine.util.TableTools; +import io.deephaven.test.types.OutOfBandTest; +import junit.framework.TestCase; +import org.apache.commons.lang3.mutable.MutableInt; + +import java.util.*; + +import org.junit.experimental.categories.Category; + +import static io.deephaven.engine.testutil.GenerateTableUpdates.generateTableUpdates; + +@Category(OutOfBandTest.class) +public class TestDataIndexer extends RefreshingTableTestCase { + + @Override + public void setUp() throws Exception { + super.setUp(); + } + + private static List> powerSet(Set originalSet) { + final List> setList = new ArrayList<>(); + final Set> powerSet = Sets.powerSet(originalSet); + for (final Set set : powerSet) { + if (set.isEmpty()) { + continue; + } + setList.add(new ArrayList<>(set)); + } + return setList; + } + + public void testIndex() { + testIndex(false, new Random(0), new MutableInt(50)); + } + + public void testIndexWithImmutableColumns() { + testIndex(true, new Random(0), new MutableInt(50)); + } + + private void testIndex(final boolean immutableColumns, final Random random, final MutableInt numSteps) { + int size = 100; + + ColumnInfo[] columnInfo = new ColumnInfo[3]; + if (immutableColumns) { + columnInfo[0] = new ColumnInfo<>(new SetGenerator<>("a", "b", "c", "d", "e", "f"), "Sym", + ColumnInfo.ColAttributes.Immutable); + columnInfo[1] = new ColumnInfo<>(new IntGenerator(10, 100), "intCol", + ColumnInfo.ColAttributes.Immutable); + } else { + columnInfo[0] = new ColumnInfo<>(new SetGenerator<>("a", "b", "c", "d", "e", "f"), "Sym"); + columnInfo[1] = new ColumnInfo<>(new IntGenerator(10, 100), "intCol"); + } + columnInfo[2] = new ColumnInfo<>(new SetGenerator<>(10.1, 20.1, 30.1), "doubleCol"); + + final QueryTable queryTable = TstUtils.getTable(size, random, columnInfo); + queryTable.setRefreshing(true); + + // Create indexes for every column combination; they will be retained by our parent class's LivenessScope + for (final List set : powerSet(queryTable.getColumnSourceMap().keySet())) { + System.out.println("Creating index for " + set); + DataIndexer.getOrCreateDataIndex(queryTable, set.toArray(String[]::new)); + } + + addIndexValidator(queryTable, "queryTable"); + + final EvalNugget[] en = new EvalNugget[] { + EvalNugget.from(() -> ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.head(0))), + EvalNugget.from(() -> ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.head(1))), + EvalNugget.from(() -> ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.update("intCol2 = intCol + 1"))), + EvalNugget.from(() -> ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.update("intCol2 = intCol + 1").select())), + EvalNugget.from(() -> ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.view("Sym", "intCol2 = intCol + 1"))), + EvalNugget.from(() -> ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.avgBy("Sym").sort("Sym"))), + EvalNugget.from(() -> ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.groupBy("Sym", "intCol") + .sort("Sym", "intCol") + .view("doubleCol=max(doubleCol)"))), + EvalNugget.from(() -> ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.avgBy("Sym", "doubleCol") + .sort("Sym", "doubleCol") + .view("intCol=min(intCol)"))), + }; + + for (int ii = 0; ii < en.length; ++ii) { + addIndexValidator(en[ii].originalValue, "en[" + ii + "]"); + } + + Table by = ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.avgBy("Sym")); + addIndexValidator(by, "groupBy"); + Table avgBy = ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.avgBy("Sym")); + addIndexValidator(avgBy, "avgBy"); + Table avgBy1 = ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> queryTable.avgBy("Sym", "intCol")); + addIndexValidator(avgBy1, "avgBy1"); + + Table merged = Require.neqNull(ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( + () -> TableTools.merge(queryTable)), "TableTools.merge(queryTable)"); + addIndexValidator(merged, "merged"); + Table updated = ExecutionContext.getContext().getUpdateGraph().exclusiveLock() + .computeLocked(() -> merged.update("HiLo = intCol > 50 ? `Hi` : `Lo`")); + addIndexValidator(updated, "updated"); + + final int maxSteps = numSteps.intValue(); // 8; + + if (RefreshingTableTestCase.printTableUpdates) { + System.out.println("Initial QueryTable: "); + TableTools.showWithRowSet(queryTable); + } + + for (numSteps.setValue(0); numSteps.intValue() < maxSteps; numSteps.increment()) { + ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle(() -> { + generateTableUpdates(size, random, queryTable, columnInfo); + }); + TstUtils.validate("Table", en); + } + } + + private void addIndexValidator(final Table originalValue, final String context) { + final List> columnSets = powerSet(originalValue.getColumnSourceMap().keySet()); + // Rely on the parent class's LivenessScope to retain the index + new IndexValidator(context, originalValue, columnSets); + } + + public void testCombinedGrouping() { + Random random = new Random(0); + int size = 100; + + ColumnInfo[] columnInfo = new ColumnInfo[4]; + columnInfo[0] = new ColumnInfo<>(new SetGenerator<>("a", "b", "c", "d", "e", "f"), "Sym", + ColumnInfo.ColAttributes.Immutable, ColumnInfo.ColAttributes.Indexed); + columnInfo[1] = new ColumnInfo<>(new SetGenerator<>("q", "r", "s", "t"), "Sym2", + ColumnInfo.ColAttributes.Immutable, ColumnInfo.ColAttributes.Indexed); + columnInfo[2] = new ColumnInfo<>(new IntGenerator(10, 100), "intCol", + ColumnInfo.ColAttributes.Immutable, ColumnInfo.ColAttributes.Indexed); + columnInfo[3] = new ColumnInfo<>(new SetGenerator<>(10.1, 20.1, 30.1), "doubleCol"); + + final QueryTable nonCountingTable = TstUtils.getTable(size, random, columnInfo); + + final QueryTable countingTable = CountingTable.getCountingTable(nonCountingTable); + + final ColumnSource symColumnSource = countingTable.getColumnSource("Sym"); + final ColumnSource sym2ColumnSource = countingTable.getColumnSource("Sym2"); + final ColumnSource intColumnSource = countingTable.getColumnSource("intCol"); + final ColumnSource doubleColumnSource = countingTable.getColumnSource("doubleCol"); + + final DataIndexer indexer = DataIndexer.of(countingTable.getRowSet()); + + assertTrue(indexer.hasDataIndex(symColumnSource)); + assertTrue(indexer.hasDataIndex(sym2ColumnSource)); + assertTrue(indexer.hasDataIndex(intColumnSource)); + assertFalse(indexer.hasDataIndex(intColumnSource, symColumnSource)); + assertFalse(indexer.hasDataIndex(intColumnSource, symColumnSource, sym2ColumnSource)); + assertFalse(indexer.hasDataIndex(intColumnSource, symColumnSource, doubleColumnSource)); + + // Add the multi-column indexes; they will be retained by our parent class's LivenessScope + DataIndexer.getOrCreateDataIndex(countingTable, "intCol", "Sym"); + DataIndexer.getOrCreateDataIndex(countingTable, "intCol", "Sym", "Sym2"); + DataIndexer.getOrCreateDataIndex(countingTable, "intCol", "Sym", "doubleCol"); + DataIndexer.getOrCreateDataIndex(countingTable, "intCol", "Sym", "Sym2", "doubleCol"); + + assertTrue(indexer.hasDataIndex(intColumnSource, symColumnSource)); + assertTrue(indexer.hasDataIndex(intColumnSource, symColumnSource, sym2ColumnSource)); + assertTrue(indexer.hasDataIndex(intColumnSource, symColumnSource, doubleColumnSource)); + assertTrue(indexer.hasDataIndex(intColumnSource, symColumnSource, sym2ColumnSource, doubleColumnSource)); + + TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); + + IndexValidator.validateIndex(countingTable, new String[] {"Sym"}, false); + countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); + + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); + IndexValidator.validateIndex(countingTable, new String[] {"intCol"}, false); + countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); + + TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); + IndexValidator.validateIndex(countingTable, new String[] {"intCol", "Sym"}, false); + countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); + + TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) sym2ColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); + IndexValidator.validateIndex(countingTable, new String[] {"intCol", "Sym", "Sym2"}, false); + countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); + + TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("getDouble")); + IndexValidator.validateIndex(countingTable, new String[] {"intCol", "Sym", "doubleCol"}, false); + countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); + + TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) sym2ColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("get")); + TestCase.assertEquals(0, ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("getDouble")); + IndexValidator.validateIndex(countingTable, new String[] {"intCol", "Sym", "Sym2", "doubleCol"}, false); + countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/indexer/TestRowSetIndexer.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/indexer/TestRowSetIndexer.java deleted file mode 100644 index 10afe27c8e9..00000000000 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/indexer/TestRowSetIndexer.java +++ /dev/null @@ -1,393 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.engine.table.impl.indexer; - -import io.deephaven.base.verify.Assert; -import io.deephaven.base.verify.Require; -import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.*; -import io.deephaven.engine.testutil.ColumnInfo; -import io.deephaven.engine.testutil.TstUtils; -import io.deephaven.engine.testutil.generator.IntGenerator; -import io.deephaven.engine.testutil.generator.SetGenerator; -import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; -import io.deephaven.engine.testutil.EvalNugget; -import io.deephaven.engine.util.TableTools; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.TupleSource; -import io.deephaven.engine.table.impl.TupleSourceFactory; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.test.types.OutOfBandTest; -import io.deephaven.tuple.ArrayTuple; -import io.deephaven.tuple.generated.IntObjectDoubleTuple; -import io.deephaven.tuple.generated.IntObjectObjectTuple; -import io.deephaven.tuple.generated.IntObjectTuple; -import junit.framework.TestCase; -import org.apache.commons.lang3.mutable.MutableInt; - -import java.util.*; -import java.util.stream.Collectors; - -import org.junit.experimental.categories.Category; - -@Category(OutOfBandTest.class) -public class TestRowSetIndexer extends RefreshingTableTestCase { - - private static ArrayList> powerSet(Set originalSet) { - return powerSet(new ArrayList<>(originalSet)); - } - - private static ArrayList> powerSet(List originalSet) { - ArrayList> sets = new ArrayList<>(); - if (originalSet.isEmpty()) { - sets.add(new ArrayList<>()); - return sets; - } - ArrayList list = new ArrayList<>(originalSet); - T head = list.get(0); - for (ArrayList set : powerSet(list.subList(1, list.size()))) { - ArrayList newSet = new ArrayList<>(list.size()); - newSet.add(head); - newSet.addAll(set); - sets.add(newSet); - sets.add(set); - } - Assert.eq(sets.size(), "sets.size()", 1 << originalSet.size(), "1<[] columnInfo = new ColumnInfo[3]; - if (immutableColumns) { - columnInfo[0] = new ColumnInfo<>(new SetGenerator<>("a", "b", "c", "d", "e", "f"), "Sym", - ColumnInfo.ColAttributes.Immutable); - columnInfo[1] = new ColumnInfo<>(new IntGenerator(10, 100), "intCol", - ColumnInfo.ColAttributes.Immutable); - } else { - columnInfo[0] = new ColumnInfo<>(new SetGenerator<>("a", "b", "c", "d", "e", "f"), "Sym"); - columnInfo[1] = new ColumnInfo<>(new IntGenerator(10, 100), "intCol"); - } - columnInfo[2] = new ColumnInfo<>(new SetGenerator<>(10.1, 20.1, 30.1), "doubleCol"); - - final QueryTable queryTable = TstUtils.getTable(size, random, columnInfo); - addGroupingValidator(queryTable, "queryTable"); - - final EvalNugget[] en = new EvalNugget[] { - EvalNugget.from(() -> { - return ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.head(0)); - }), - EvalNugget.from(() -> { - return ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.head(1)); - }), - EvalNugget.from(() -> { - return ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.update("intCol2 = intCol + 1")); - }), - EvalNugget.from(() -> { - return ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.update("intCol2 = intCol + 1").select()); - }), - EvalNugget.from(() -> { - return ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.view("Sym", "intCol2 = intCol + 1")); - }), - EvalNugget.from(() -> { - return ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.avgBy("Sym").sort("Sym")); - }), - EvalNugget.from(() -> { - return ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.groupBy("Sym", "intCol") - .sort("Sym", "intCol") - .view("doubleCol=max(doubleCol)")); - }), - EvalNugget.from(() -> { - return ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.avgBy("Sym", "doubleCol") - .sort("Sym", "doubleCol") - .view("intCol=min(intCol)")); - }), - }; - - for (int ii = 0; ii < en.length; ++ii) { - addGroupingValidator(en[ii].originalValue, "en[" + ii + "]"); - } - - Table by = ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.avgBy("Sym")); - addGroupingValidator(by, "groupBy"); - Table avgBy = ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.avgBy("Sym")); - addGroupingValidator(avgBy, "avgBy"); - Table avgBy1 = ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> queryTable.avgBy("Sym", "intCol")); - addGroupingValidator(avgBy1, "avgBy1"); - - Table merged = Require.neqNull(ExecutionContext.getContext().getUpdateGraph().exclusiveLock().computeLocked( - () -> TableTools.merge(queryTable)), "TableTools.merge(queryTable)"); - addGroupingValidator(merged, "merged"); - Table updated = ExecutionContext.getContext().getUpdateGraph().exclusiveLock() - .computeLocked(() -> merged.update("HiLo = intCol > 50 ? `Hi` : `Lo`")); - addGroupingValidator(updated, "updated"); - - final int maxSteps = numSteps.intValue(); // 8; - - if (RefreshingTableTestCase.printTableUpdates) { - System.out.println("Initial QueryTable: "); - TableTools.showWithRowSet(queryTable); - } - for (numSteps.setValue(0); numSteps.intValue() < maxSteps; numSteps.increment()) { - RefreshingTableTestCase.simulateShiftAwareStep("step == " + numSteps.intValue(), size, random, queryTable, - columnInfo, en); - } - - // we don't need them after this test is done - groupingValidators.clear(); - } - - // we don't ever need to look at the grouping validators, just make sure they don't go away - @SuppressWarnings("MismatchedQueryAndUpdateOfCollection") - private final ArrayList groupingValidators = new ArrayList<>(); - - private void addGroupingValidator(Table originalValue, String context) { - ArrayList> columnSets2 = powerSet(originalValue.getDefinition().getColumnNameSet()); - ArrayList columnNames = new ArrayList<>(originalValue.getDefinition().getColumnNameSet()); - columnSets2.add(columnNames); - groupingValidators.add(new GroupingValidator(context, originalValue, columnSets2)); - } - - public void testCombinedGrouping() { - Random random = new Random(0); - int size = 100; - - ColumnInfo[] columnInfo = new ColumnInfo[4]; - columnInfo[0] = new ColumnInfo<>(new SetGenerator<>("a", "b", "c", "d", "e", "f"), "Sym", - ColumnInfo.ColAttributes.Immutable, ColumnInfo.ColAttributes.Grouped); - columnInfo[1] = new ColumnInfo<>(new SetGenerator<>("q", "r", "s", "t"), "Sym2", - ColumnInfo.ColAttributes.Immutable, ColumnInfo.ColAttributes.Grouped); - columnInfo[2] = new ColumnInfo<>(new IntGenerator(10, 100), "intCol", - ColumnInfo.ColAttributes.Immutable, ColumnInfo.ColAttributes.Grouped); - columnInfo[3] = new ColumnInfo<>(new SetGenerator<>(10.1, 20.1, 30.1), "doubleCol"); - - final QueryTable nonCountingTable = TstUtils.getTable(size, random, columnInfo); - - final QueryTable countingTable = CountingTable.getCountingTable(nonCountingTable); - - final ColumnSource symColumnSource = countingTable.getColumnSource("Sym"); - final ColumnSource sym2ColumnSource = countingTable.getColumnSource("Sym2"); - final ColumnSource intColumnSource = countingTable.getColumnSource("intCol"); - final ColumnSource doubleColumnSource = countingTable.getColumnSource("doubleCol"); - - final RowSetIndexer indexer = RowSetIndexer.of(countingTable.getRowSet()); - - assertTrue(indexer.hasGrouping(symColumnSource)); - assertTrue(indexer.hasGrouping(sym2ColumnSource)); - assertTrue(indexer.hasGrouping(intColumnSource)); - assertFalse(indexer.hasGrouping(intColumnSource, symColumnSource)); - assertFalse(indexer.hasGrouping(intColumnSource, symColumnSource, sym2ColumnSource)); - assertFalse(indexer.hasGrouping(intColumnSource, symColumnSource, doubleColumnSource)); - - Map symGrouping = indexer.getGrouping(symColumnSource); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); - - GroupingValidator.validateGrouping(new String[] {"Sym"}, countingTable.getRowSet(), countingTable, "sym", - symGrouping); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - Map intGrouping = indexer.getGrouping(intColumnSource); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); - GroupingValidator.validateGrouping(new String[] {"intCol"}, countingTable.getRowSet(), countingTable, "intCol", - intGrouping); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - final TupleSource intSymTupleSource = TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource); - - Map intSymGrouping = indexer.getGrouping(intSymTupleSource); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); - GroupingValidator.validateGrouping(new String[] {"intCol", "Sym"}, countingTable.getRowSet(), countingTable, - "intCol+sym", intSymGrouping); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - final TupleSource intSymSym2TupleSource = - TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource, sym2ColumnSource); - Map intSymSym2Grouping = indexer.getGrouping(intSymSym2TupleSource); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) sym2ColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); - GroupingValidator.validateGrouping(new String[] {"intCol", "Sym", "Sym2"}, countingTable.getRowSet(), - countingTable, "intCol+sym+sym2", intSymSym2Grouping); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - final TupleSource intSymDoubleTupleSource = - TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource, doubleColumnSource); - Map intSymDoubleGrouping = indexer.getGrouping(intSymDoubleTupleSource); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); - TestCase.assertEquals(countingTable.size(), - ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("getDouble")); - GroupingValidator.validateGrouping(new String[] {"intCol", "Sym", "doubleCol"}, countingTable.getRowSet(), - countingTable, "intCol+sym+doubleCol", intSymDoubleGrouping); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - final TupleSource intSymSym2DoubleTupleSource = TupleSourceFactory.makeTupleSource(intColumnSource, - symColumnSource, sym2ColumnSource, doubleColumnSource); - Map intSymSym2DoubleGrouping = - indexer.getGrouping(intSymSym2DoubleTupleSource); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) sym2ColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); - TestCase.assertEquals(countingTable.size(), - ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("getDouble")); - GroupingValidator.validateGrouping(new String[] {"intCol", "Sym", "Sym2", "doubleCol"}, - countingTable.getRowSet(), countingTable, "intCol+sym+sym2+doubleCol", intSymSym2DoubleGrouping); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - } - - public void testRestrictedGrouping() { - Random random = new Random(0); - int size = 100; - - ColumnInfo[] columnInfo = new ColumnInfo[4]; - columnInfo[0] = new ColumnInfo<>(new SetGenerator<>("a", "b", "c", "d", "e", "f"), "Sym", - ColumnInfo.ColAttributes.Immutable, ColumnInfo.ColAttributes.Grouped); - columnInfo[1] = new ColumnInfo<>(new SetGenerator<>("q", "r", "s", "t", "u", "v"), "Sym2", - ColumnInfo.ColAttributes.Immutable, ColumnInfo.ColAttributes.Grouped); - columnInfo[2] = new ColumnInfo<>(new IntGenerator(10, 100), "intCol", - ColumnInfo.ColAttributes.Immutable, ColumnInfo.ColAttributes.Grouped); - columnInfo[3] = new ColumnInfo<>(new SetGenerator<>(10.1, 20.1, 30.1), "doubleCol"); - - final QueryTable nonCountingTable = TstUtils.getTable(size, random, columnInfo); - - final QueryTable countingTable = CountingTable.getCountingTable(nonCountingTable); - - final ColumnSource symColumnSource = countingTable.getColumnSource("Sym"); - final ColumnSource sym2ColumnSource = countingTable.getColumnSource("Sym2"); - final ColumnSource intColumnSource = countingTable.getColumnSource("intCol"); - final ColumnSource doubleColumnSource = countingTable.getColumnSource("doubleCol"); - - final RowSetIndexer indexer = RowSetIndexer.of(countingTable.getRowSet()); - - assertTrue(indexer.hasGrouping(symColumnSource)); - assertTrue(indexer.hasGrouping(sym2ColumnSource)); - assertTrue(indexer.hasGrouping(intColumnSource)); - assertFalse(indexer.hasGrouping(intColumnSource, symColumnSource)); - assertFalse(indexer.hasGrouping(intColumnSource, symColumnSource, sym2ColumnSource)); - assertFalse(indexer.hasGrouping(intColumnSource, symColumnSource, sym2ColumnSource, - doubleColumnSource)); - assertFalse(indexer.hasGrouping(intColumnSource, symColumnSource, doubleColumnSource)); - - final TreeSet keySet = new TreeSet<>(Arrays.asList("a", "b")); - final Map symGrouping = indexer.getGroupingForKeySet(keySet, symColumnSource); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); - - GroupingValidator.validateRestrictedGrouping(new String[] {"Sym"}, countingTable.getRowSet(), countingTable, - "sym", symGrouping, keySet); - ((CountingTable.MethodCounter) symColumnSource).clear(); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - keySet.clear(); - keySet.addAll(Arrays.asList(10, 20, 30, 40, 50, 60, 70, 80, 90)); - final Map intGrouping = indexer.getGroupingForKeySet(keySet, intColumnSource); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); - GroupingValidator.validateRestrictedGrouping(new String[] {"intCol"}, countingTable.getRowSet(), countingTable, - "intCol", intGrouping, keySet); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - keySet.clear(); - final TupleSource intSymFactory = TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource); - TstUtils.selectSubIndexSet(5, countingTable.getRowSet(), random) - .forAllRowKeys(row -> keySet.add(intSymFactory.createTuple(row))); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - final Map intSymGrouping = indexer.getGroupingForKeySet(keySet, - TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource)); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); - GroupingValidator.validateRestrictedGrouping(new String[] {"intCol", "Sym"}, countingTable.getRowSet(), - countingTable, "intCol+sym", intSymGrouping, keySet); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - keySet.clear(); - final TupleSource intSymDoubleFactory = - TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource, doubleColumnSource); - TstUtils.selectSubIndexSet(5, countingTable.getRowSet(), random) - .forAllRowKeys(row -> keySet.add(intSymDoubleFactory.createTuple(row))); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - long intSymGroupingSize = indexer.getGroupingForKeySet( - keySet.stream().map(k -> { - final IntObjectDoubleTuple kTuple = (IntObjectDoubleTuple) k; - return new IntObjectTuple(kTuple.getFirstElement(), kTuple.getSecondElement()); - }).collect(Collectors.toSet()), - TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource)) - .values().stream().mapToLong(RowSet::size).sum(); - final Map intSymDoubleGrouping = indexer.getGroupingForKeySet(keySet, - TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource, doubleColumnSource)); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); - TestCase.assertEquals(intSymGroupingSize, - ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("getDouble")); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - GroupingValidator.validateRestrictedGrouping(new String[] {"intCol", "Sym", "doubleCol"}, - countingTable.getRowSet(), countingTable, "intCol+sym+doubleCol", intSymDoubleGrouping, keySet); - - keySet.clear(); - final TupleSource intSymSym2DoubleFactory = - TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource, - sym2ColumnSource, doubleColumnSource); - TstUtils.selectSubIndexSet(5, countingTable.getRowSet(), random) - .forAllRowKeys(row -> keySet.add(intSymSym2DoubleFactory.createTuple(row))); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - long intSymSym2GroupingSize = indexer.getGroupingForKeySet( - keySet.stream().map(k -> { - final ArrayTuple kTuple = (ArrayTuple) k; - return new IntObjectObjectTuple( - kTuple.getElement(0), kTuple.getElement(1), kTuple.getElement(2)); - }).collect(Collectors.toSet()), - TupleSourceFactory.makeTupleSource(intColumnSource, symColumnSource, sym2ColumnSource)) - .values().stream().mapToLong(RowSet::size).sum(); - final Map intSymSym2DoubleGrouping = - indexer.getGroupingForKeySet(keySet, TupleSourceFactory - .makeTupleSource(intColumnSource, symColumnSource, sym2ColumnSource, doubleColumnSource)); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) symColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) sym2ColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) intColumnSource).getMethodCount("getInt")); - TestCase.assertEquals(intSymSym2GroupingSize, - ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("get")); - TestCase.assertEquals(0, ((CountingTable.MethodCounter) doubleColumnSource).getMethodCount("getDouble")); - countingTable.getColumnSources().forEach(x -> ((CountingTable.MethodCounter) x).clear()); - - GroupingValidator.validateRestrictedGrouping(new String[] {"intCol", "Sym", "Sym2", "doubleCol"}, - countingTable.getRowSet(), countingTable, "intCol+sym+sym2+doubleCol", intSymSym2DoubleGrouping, - keySet); - } -} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/indexer/TestTransformedDataIndex.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/indexer/TestTransformedDataIndex.java new file mode 100644 index 00000000000..da3b735ad19 --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/indexer/TestTransformedDataIndex.java @@ -0,0 +1,509 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.indexer; + +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.primitive.iterator.CloseableIterator; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.dataindex.DataIndexUtils; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.iterators.ChunkedColumnIterator; +import io.deephaven.engine.testutil.ColumnInfo; +import io.deephaven.engine.testutil.ControlledUpdateGraph; +import io.deephaven.engine.testutil.GenerateTableUpdates; +import io.deephaven.engine.testutil.TstUtils; +import io.deephaven.engine.testutil.generator.IntGenerator; +import io.deephaven.engine.testutil.generator.SetGenerator; +import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; +import io.deephaven.test.types.OutOfBandTest; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.*; +import java.util.function.Function; +import java.util.function.ToLongFunction; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; + +@Category(OutOfBandTest.class) +public class TestTransformedDataIndex extends RefreshingTableTestCase { + private static final int INITIAL_SIZE = 100; + private static final int STEP_SIZE = 100; + private static final int MAX_STEPS = 100; + private ColumnInfo[] columnInfo; + private QueryTable testTable; + private List dataIndexes; + + @Override + public void setUp() throws Exception { + super.setUp(); + columnInfo = new ColumnInfo[4]; + columnInfo[0] = new ColumnInfo<>(new SetGenerator<>("a", "b", "c", "d", "e", "f"), "Sym"); + columnInfo[1] = new ColumnInfo<>(new SetGenerator<>("q", "r", "s", "t"), "Sym2"); + columnInfo[2] = new ColumnInfo<>(new IntGenerator(10, 100), "intCol"); + columnInfo[3] = new ColumnInfo<>(new SetGenerator<>(10.1, 20.1, 30.1), "doubleCol"); + testTable = TstUtils.getTable(INITIAL_SIZE, new Random(0), columnInfo); + testTable.setRefreshing(true); + + // Add some data indexes; they will be retained by our parent class's LivenessScope + dataIndexes = new ArrayList<>(); + dataIndexes.add(DataIndexer.getOrCreateDataIndex(testTable, "Sym")); + dataIndexes.add(DataIndexer.getOrCreateDataIndex(testTable, "Sym2")); + dataIndexes.add(DataIndexer.getOrCreateDataIndex(testTable, "Sym", "Sym2")); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + dataIndexes = null; + } + + @Test + public void testFullIndexLookup() { + final Random random = new Random(0); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + final ColumnSource[] columns = dataIndex.keyColumnNames().stream() + .map(testTable::getColumnSource) + .toArray(ColumnSource[]::new); + assertLookupFromTable(testTable, dataIndex, columns); + } + + // Transform and validate + for (int ii = 0; ii < MAX_STEPS; ++ii) { + ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(STEP_SIZE, random, testTable, columnInfo)); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + final ColumnSource[] columns = dataIndex.keyColumnNames().stream() + .map(testTable::getColumnSource) + .toArray(ColumnSource[]::new); + assertLookupFromTable(testTable, dataIndex, columns); + } + } + } + + @Test + public void testRemappedDataIndexAllColumns() { + for (final DataIndex dataIndex : dataIndexes) { + // Map dummy columns to the key columns + final Map, ColumnSource> firstRemap = new HashMap<>(); + dataIndex.keyColumnNames().forEach(name -> firstRemap.put(testTable.getColumnSource(name), + InMemoryColumnSource.makeImmutableSource(Integer.class, null))); + final DataIndex firstRemappedIndex = dataIndex.remapKeyColumns(firstRemap); + // Verify that the original and remapped indexes point to the same index table columns + assertEquals(dataIndex.keyColumns(), firstRemappedIndex.keyColumns()); + + // Map new dummy columns to the old dummy columns (second-level) + final Map, ColumnSource> secondRemap = new HashMap<>(); + firstRemap.forEach((oldColumn, dummy) -> secondRemap.put(dummy, + InMemoryColumnSource.makeImmutableSource(Integer.class, null))); + final DataIndex secondRemappedIndex = firstRemappedIndex.remapKeyColumns(secondRemap); + // Verify that the original and remapped indexes point to the same index table columns + assertEquals(dataIndex.keyColumns(), secondRemappedIndex.keyColumns()); + + // Map even newer dummy columns to the old dummy columns (third-level) + final Map, ColumnSource> thirdRemap = new HashMap<>(); + secondRemap.forEach((oldColumn, dummy) -> thirdRemap.put(dummy, + InMemoryColumnSource.makeImmutableSource(Integer.class, null))); + final DataIndex thirdRemappedIndex = secondRemappedIndex.remapKeyColumns(secondRemap); + // Verify that the original and remapped indexes point to the same index table columns + assertEquals(dataIndex.keyColumns(), thirdRemappedIndex.keyColumns()); + } + } + + @Test + public void testRemappedDataIndexOnlyFirstColumns() { + for (final DataIndex dataIndex : dataIndexes) { + final ColumnSource firstDummy = InMemoryColumnSource.makeImmutableSource(Integer.class, null); + final Map, ColumnSource> firstRemap = + Map.of(testTable.getColumnSource(dataIndex.keyColumnNames().get(0)), firstDummy); + final DataIndex firstRemappedIndex = dataIndex.remapKeyColumns(firstRemap); + // Verify that the original and remapped indexes point to the same index table columns + assertEquals(dataIndex.keyColumns(), firstRemappedIndex.keyColumns()); + + final ColumnSource secondDummy = InMemoryColumnSource.makeImmutableSource(Integer.class, null); + final Map, ColumnSource> secondRemap = Map.of(firstDummy, secondDummy); + final DataIndex secondRemappedIndex = firstRemappedIndex.remapKeyColumns(secondRemap); + // Verify that the original and remapped indexes point to the same index table columns + assertEquals(dataIndex.keyColumns(), firstRemappedIndex.keyColumns()); + + final ColumnSource thirdDummy = InMemoryColumnSource.makeImmutableSource(Integer.class, null); + final Map, ColumnSource> thirdRemap = Map.of(secondDummy, thirdDummy); + final DataIndex thirdRemappedIndex = secondRemappedIndex.remapKeyColumns(thirdRemap); + // Verify that the original and remapped indexes point to the same index table columns + assertEquals(dataIndex.keyColumns(), thirdRemappedIndex.keyColumns()); + } + } + + + @Test + public void testMultiColumnOutOfOrderLookup() { + final Random random = new Random(0); + final DataIndexer dataIndexer = DataIndexer.of(testTable.getRowSet()); + + final DataIndex dataIndex = DataIndexer.getDataIndex(testTable, "Sym", "Sym2"); + assertNotNull(dataIndex); + + // Make sure the index is found with the re-ordered column names. + assertTrue(DataIndexer.hasDataIndex(testTable, "Sym2", "Sym")); + final DataIndex tmp1 = DataIndexer.getDataIndex(testTable, "Sym2", "Sym"); + assertEquals(dataIndex, tmp1); + + final ColumnSource[] columnsReordered = new ColumnSource[] { + testTable.getColumnSource("Sym2"), + testTable.getColumnSource("Sym") + }; + // Make sure the index is found with re-ordered columns. + assertTrue(dataIndexer.hasDataIndex(columnsReordered)); + final DataIndex tmp2 = dataIndexer.getDataIndex(columnsReordered); + assertEquals(dataIndex, tmp2); + + final ColumnSource[] columns = Arrays.stream(new String[] {"Sym", "Sym2"}) + .map(testTable::getColumnSource) + .toArray(ColumnSource[]::new); + final ColumnSource[] reorderedColumns = Arrays.stream(new String[] {"Sym2", "Sym"}) + .map(testTable::getColumnSource) + .toArray(ColumnSource[]::new); + + assertRefreshing(dataIndex); + assertLookupFromTable(testTable, dataIndex, columns); + assertLookupFromTable(testTable, dataIndex, reorderedColumns); + + for (int ii = 0; ii < MAX_STEPS; ++ii) { + ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(STEP_SIZE, random, testTable, columnInfo)); + + assertRefreshing(dataIndex); + assertLookupFromTable(testTable, dataIndex, columns); + assertLookupFromTable(testTable, dataIndex, reorderedColumns); + } + } + + @Test + public void testIntersectAndInvert() { + final Random random = new Random(0); + + try (final RowSet reduced = testTable.getRowSet().subSetByPositionRange(0, testTable.getRowSet().size() / 4)) { + final RowSet tableRowSet = testTable.getRowSet(); + final DataIndexTransformer intersectTransformer = + DataIndexTransformer.builder().intersectRowSet(reduced).build(); + final Function intersectMutator = (rs) -> rs.intersect(reduced); + final DataIndexTransformer invertTransformer = + DataIndexTransformer.builder().invertRowSet(tableRowSet).build(); + final Function invertMutator = tableRowSet::invert; + final DataIndexTransformer intersectInvertTransformer = DataIndexTransformer.builder() + .intersectRowSet(reduced) + .invertRowSet(tableRowSet) + .build(); + final Function intersectInvertMutator = (rs) -> tableRowSet.invert(rs.intersect(reduced)); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + // Test each transform individually. + BasicDataIndex subIndex = dataIndex.transform(intersectTransformer); + assertStatic(subIndex); + assertLookupMutator(dataIndex, subIndex, intersectMutator); + + subIndex = dataIndex.transform(invertTransformer); + assertStatic(subIndex); + assertLookupMutator(dataIndex, subIndex, invertMutator); + + subIndex = dataIndex.transform(intersectInvertTransformer); + assertStatic(subIndex); + assertLookupMutator(dataIndex, subIndex, intersectInvertMutator); + } + } + + // Transform and validate + for (int ii = 0; ii < MAX_STEPS; ++ii) { + ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(STEP_SIZE, random, testTable, columnInfo)); + + try (final RowSet reduced = + testTable.getRowSet().subSetByPositionRange(0, testTable.getRowSet().size() / 4)) { + final RowSet tableRowSet = testTable.getRowSet(); + final DataIndexTransformer intersectTransformer = + DataIndexTransformer.builder().intersectRowSet(reduced).build(); + final Function intersectMutator = (rs) -> rs.intersect(reduced); + final DataIndexTransformer invertTransformer = + DataIndexTransformer.builder().invertRowSet(tableRowSet).build(); + final Function invertMutator = tableRowSet::invert; + final DataIndexTransformer intersectInvertTransformer = DataIndexTransformer.builder() + .intersectRowSet(reduced) + .invertRowSet(tableRowSet) + .build(); + final Function intersectInvertMutator = + (rs) -> tableRowSet.invert(rs.intersect(reduced)); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + // Test each transform individually. + BasicDataIndex subIndex = dataIndex.transform(intersectTransformer); + assertStatic(subIndex); + assertLookupMutator(dataIndex, subIndex, intersectMutator); + + subIndex = dataIndex.transform(invertTransformer); + assertStatic(subIndex); + assertLookupMutator(dataIndex, subIndex, invertMutator); + + subIndex = dataIndex.transform(intersectInvertTransformer); + assertStatic(subIndex); + assertLookupMutator(dataIndex, subIndex, intersectInvertMutator); + } + } + } + } + + @Test + public void testSortByFirstRowKey() { + final Random random = new Random(0); + + final DataIndexTransformer transformer = DataIndexTransformer.builder().sortByFirstRowKey(true).build(); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + BasicDataIndex subIndex = dataIndex.transform(transformer); + assertRefreshing(subIndex); + assertSortedByFirstRowKey(subIndex); + assertLookup(dataIndex, subIndex); + } + + // Transform and validate + for (int ii = 0; ii < MAX_STEPS; ++ii) { + ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(STEP_SIZE, random, testTable, columnInfo)); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + BasicDataIndex subIndex = dataIndex.transform(transformer); + assertRefreshing(subIndex); + assertSortedByFirstRowKey(subIndex); + assertLookup(dataIndex, subIndex); + } + } + } + + @Test + public void testSortIntersect() { + final Random random = new Random(0); + + try (final RowSet reduced = testTable.getRowSet().subSetByPositionRange(0, testTable.getRowSet().size() / 4)) { + final DataIndexTransformer transformer = DataIndexTransformer.builder() + .intersectRowSet(reduced) + .sortByFirstRowKey(true) + .build(); + final Function mutator = (rs) -> rs.intersect(reduced); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + BasicDataIndex subIndex = dataIndex.transform(transformer); + assertStatic(subIndex); + assertSortedByFirstRowKey(subIndex); + assertLookupMutator(dataIndex, subIndex, mutator); + } + } + + // Transform and validate + for (int ii = 0; ii < MAX_STEPS; ++ii) { + ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(STEP_SIZE, random, testTable, columnInfo)); + + try (final RowSet reduced = + testTable.getRowSet().subSetByPositionRange(0, testTable.getRowSet().size() / 4)) { + final DataIndexTransformer transformer = DataIndexTransformer.builder() + .intersectRowSet(reduced) + .sortByFirstRowKey(true) + .build(); + final Function mutator = (rs) -> rs.intersect(reduced); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + BasicDataIndex subIndex = dataIndex.transform(transformer); + assertStatic(subIndex); + assertSortedByFirstRowKey(subIndex); + assertLookupMutator(dataIndex, subIndex, mutator); + } + } + } + } + + @Test + public void testSortIntersectInvert() { + final Random random = new Random(0); + + try (final RowSet reduced = testTable.getRowSet().subSetByPositionRange(0, testTable.getRowSet().size() / 4)) { + final RowSet tableRowSet = testTable.getRowSet(); + final DataIndexTransformer transformer = DataIndexTransformer.builder() + .intersectRowSet(reduced) + .invertRowSet(tableRowSet) + .sortByFirstRowKey(true) + .build(); + final Function mutator = (rs) -> tableRowSet.invert(rs.intersect(reduced)); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + BasicDataIndex subIndex = dataIndex.transform(transformer); + assertStatic(subIndex); + assertSortedByFirstRowKey(subIndex); + assertLookupMutator(dataIndex, subIndex, mutator); + } + } + + // Transform and validate + for (int ii = 0; ii < MAX_STEPS; ++ii) { + ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle( + () -> GenerateTableUpdates.generateTableUpdates(STEP_SIZE, random, testTable, columnInfo)); + + try (final RowSet reduced = + testTable.getRowSet().subSetByPositionRange(0, testTable.getRowSet().size() / 4)) { + final RowSet tableRowSet = testTable.getRowSet(); + final DataIndexTransformer transformer = DataIndexTransformer.builder() + .intersectRowSet(reduced) + .invertRowSet(tableRowSet) + .sortByFirstRowKey(true) + .build(); + final Function mutator = (rs) -> tableRowSet.invert(rs.intersect(reduced)); + + for (final DataIndex dataIndex : dataIndexes) { + assertRefreshing(dataIndex); + + BasicDataIndex subIndex = dataIndex.transform(transformer); + assertStatic(subIndex); + assertSortedByFirstRowKey(subIndex); + assertLookupMutator(dataIndex, subIndex, mutator); + } + } + } + } + + + private void assertStatic(final BasicDataIndex subIndex) { + assertFalse(subIndex.isRefreshing()); + assertFalse(subIndex.table().isRefreshing()); + } + + private void assertRefreshing(final BasicDataIndex subIndex) { + assertTrue(subIndex.isRefreshing()); + assertTrue(subIndex.table().isRefreshing()); + } + + private void assertLookup(final DataIndex fullIndex, + final BasicDataIndex subIndex) { + final Table subIndexTable = subIndex.table(); + + final DataIndex.RowKeyLookup fullIndexRowKeyLookup = fullIndex.rowKeyLookup(); + final ToLongFunction subIndexRowKeyMappingFunction = DataIndexUtils.buildRowKeyMappingFunction( + subIndex.table(), subIndex.keyColumnNames().toArray(String[]::new)); + + ChunkSource.WithPrev subKeys = DataIndexUtils.makeBoxedKeySource(subIndex.keyColumns()); + + try (final CloseableIterator subKeyIt = + ChunkedColumnIterator.make(subKeys, subIndexTable.getRowSet())) { + + while (subKeyIt.hasNext()) { + final Object subKey = subKeyIt.next(); + + // Verify the row sets at the lookup keys match. + final long subRowKey = subIndexRowKeyMappingFunction.applyAsLong(subKey); + final long fullRowKey = fullIndexRowKeyLookup.apply(subKey, false); + + assertEquals(subIndex.rowSetColumn().get(subRowKey), fullIndex.rowSetColumn().get(fullRowKey)); + } + } + } + + private void assertLookupMutator( + final DataIndex fullIndex, + final BasicDataIndex subIndex, + final Function mutator) { + final Table fullIndexTable = fullIndex.table(); + final ToLongFunction subIndexRowKeyMappingFunction = DataIndexUtils.buildRowKeyMappingFunction( + subIndex.table(), subIndex.keyColumnNames().toArray(String[]::new)); + + ChunkSource.WithPrev fullKeys = DataIndexUtils.makeBoxedKeySource(fullIndex.keyColumns()); + + try (final CloseableIterator fullKeyIt = + ChunkedColumnIterator.make(fullKeys, fullIndexTable.getRowSet()); + final CloseableIterator fullRowSetIt = + ChunkedColumnIterator.make(fullIndex.rowSetColumn(), fullIndexTable.getRowSet())) { + + while (fullKeyIt.hasNext()) { + final Object fullKey = fullKeyIt.next(); + final RowSet fullRowSet = fullRowSetIt.next(); + + // Is the key in the sub-index? + final long subRowKey = subIndexRowKeyMappingFunction.applyAsLong(fullKey); + if (subRowKey == NULL_ROW_KEY) { + // Verify applying the mutator to the full row set results in an empty row set. + assertTrue(mutator.apply(fullRowSet).isEmpty()); + } else { + // The row set from the lookup must match the computed row set. + assertEquals(subIndex.rowSetColumn().get(subRowKey), mutator.apply(fullRowSet)); + } + } + } + } + + private void assertLookupFromTable( + final Table sourceTable, + final DataIndex fullIndex, + final ColumnSource[] columns) { + final DataIndex.RowKeyLookup fullIndexRowKeyLookup = fullIndex.rowKeyLookup(columns); + final ColumnSource fullIndexRowSetColumn = fullIndex.rowSetColumn(); + + ChunkSource.WithPrev tableKeys = DataIndexUtils.makeBoxedKeySource(columns); + + // Iterate through the entire source table and verify the lookup row set is valid and contains this row. + try (final RowSet.Iterator rsIt = sourceTable.getRowSet().iterator(); + final CloseableIterator keyIt = + ChunkedColumnIterator.make(tableKeys, sourceTable.getRowSet())) { + + while (rsIt.hasNext() && keyIt.hasNext()) { + final long rowKey = rsIt.nextLong(); + final Object key = keyIt.next(); + + // Verify the row sets at the lookup keys match. + final long fullRowKey = fullIndexRowKeyLookup.apply(key, false); + Assert.geqZero(fullRowKey, "fullRowKey"); + + final RowSet fullRowSet = fullIndexRowSetColumn.get(fullRowKey); + assertNotNull(fullRowSet); + + assertTrue(fullRowSet.containsRange(rowKey, rowKey)); + } + } + } + + private void assertSortedByFirstRowKey(final BasicDataIndex subIndex) { + final Table subIndexTable = subIndex.table(); + + try (final CloseableIterator subRowSetIt = + ChunkedColumnIterator.make(subIndex.rowSetColumn(), subIndexTable.getRowSet())) { + + long lastKey = -1; + while (subRowSetIt.hasNext()) { + final RowSet subRowSet = subRowSetIt.next(); + + // Assert sorted-ness of the sub-index. + Assert.gt(subRowSet.firstRowKey(), "subRowSet.firstRowKey()", lastKey, "lastKey"); + lastKey = subRowSet.firstRowKey(); + } + } + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestGroupingProviders.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestSourceTableDataIndexes.java similarity index 84% rename from engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestGroupingProviders.java rename to engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestSourceTableDataIndexes.java index e4db2b85ed2..a774ba297ed 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestGroupingProviders.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/locations/impl/TestSourceTableDataIndexes.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.parquet.table.ParquetTools; import io.deephaven.engine.util.TableTools; import io.deephaven.engine.util.file.TrackedFileHandleFactory; @@ -32,9 +33,9 @@ import static io.deephaven.parquet.table.layout.DeephavenNestedPartitionLayout.PARQUET_FILE_NAME; /** - * Unit tests for {@link ParallelDeferredGroupingProvider}. + * Unit tests for {@code PartitioningColumnDataIndex} and {@code MergedDataIndex}. */ -public class TestGroupingProviders { +public class TestSourceTableDataIndexes { @Rule public final EngineCleanup base = new EngineCleanup(); @@ -72,39 +73,37 @@ public void testParallelOrdering() { } @Test - public void testParallelMissingGroups() { + public void testParallelMissingIndexes() { doTest(true); } - private void doTest(final boolean missingGroups) { + private void doTest(final boolean missingIndexes) { final Table raw = TableTools.emptyTable(26 * 10 * 1000).update("Part=String.format(`%04d`, (long)(ii/1000))", "Sym=(char)('A' + ii % 26)", "Other=ii"); final Table[] partitions = raw.partitionBy("Part") .transform(null, rp -> rp.groupBy("Sym").ungroup(), false) .constituents(); - if (!missingGroups) { - // Create a pair of partitions without the grouping column + // Create a local index for each partition + for (final Table t : partitions) { + DataIndexer.getOrCreateDataIndex(t, "Sym"); + } + + if (missingIndexes) { + // Create a pair of partitions without the indexing column partitions[2] = partitions[2].dropColumns("Sym"); partitions[3] = partitions[3].dropColumns("Sym"); } final TableDefinition partitionedDataDefinition = TableDefinition.of( ColumnDefinition.ofString("Part").withPartitioning(), - ColumnDefinition.ofChar("Sym").withGrouping(), + ColumnDefinition.ofChar("Sym"), ColumnDefinition.ofLong("Other")); - final TableDefinition partitionedMissingDataDefinition; - if (missingGroups) { - partitionedMissingDataDefinition = TableDefinition.of( - ColumnDefinition.ofString("Part").withPartitioning(), - ColumnDefinition.ofChar("Sym"), - ColumnDefinition.ofLong("Other")); - } else { - partitionedMissingDataDefinition = TableDefinition.of( - ColumnDefinition.ofString("Part").withPartitioning(), - ColumnDefinition.ofLong("Other")); - } + final TableDefinition partitionedMissingDataDefinition = TableDefinition.of( + ColumnDefinition.ofString("Part").withPartitioning(), + ColumnDefinition.ofChar("Sym"), + ColumnDefinition.ofLong("Other")); final String tableName = "TestTable"; @@ -125,13 +124,13 @@ private void doTest(final boolean missingGroups) { new File(dataDirectory, "IP" + File.separator + "0002" + File.separator + tableName + File.separator + PARQUET_FILE_NAME), - partitionedMissingDataDefinition); + missingIndexes ? partitionedMissingDataDefinition : partitionedDataDefinition); ParquetTools.writeTable( partitions[3], new File(dataDirectory, "IP" + File.separator + "0003" + File.separator + tableName + File.separator + PARQUET_FILE_NAME), - partitionedMissingDataDefinition); + missingIndexes ? partitionedMissingDataDefinition : partitionedDataDefinition); ParquetTools.writeTables( Arrays.copyOfRange(partitions, 4, partitions.length), partitionedDataDefinition, @@ -148,15 +147,13 @@ private void doTest(final boolean missingGroups) { + PARQUET_FILE_NAME), partitionedDataDefinition); - if (!missingGroups) { + if (missingIndexes) { // Put Sym back on for the partitions that dropped it. partitions[2] = partitions[2].updateView("Sym = NULL_CHAR"); partitions[3] = partitions[3].updateView("Sym = NULL_CHAR"); } - final Table expected = TableTools.merge(partitions).view("Part", "Sym", "Other"); // Column ordering was changed - // by groupBy()/ungroup() - // above, - // restore it here. + // Column ordering was changed by groupBy()/ungroup() above, restore it here. + final Table expected = TableTools.merge(partitions).view("Part", "Sym", "Other"); final Table actual = ParquetTools.readPartitionedTable( DeephavenNestedPartitionLayout.forParquet(dataDirectory, tableName, "Part", ipn -> ipn.equals("IP"), @@ -166,7 +163,12 @@ private void doTest(final boolean missingGroups) { TstUtils.assertTableEquals(expected, actual); - TestCase.assertEquals(missingGroups, actual.getColumnSource("Sym").getGroupToRange() == null); + // Make sure we have the partitioning column index + Assert.eqTrue(DataIndexer.hasDataIndex(actual, "Part"), + "DataIndexer.hasDataIndex(actual, \"Part\")"); + + // Without + TestCase.assertEquals(!missingIndexes, DataIndexer.hasDataIndex(actual, "Sym")); TstUtils.assertTableEquals(expected.groupBy("Sym").ungroup(), actual.groupBy("Sym").ungroup()); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/WhereFilterFactoryTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/WhereFilterFactoryTest.java index dcc27dbbb6f..cc41c935bb8 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/WhereFilterFactoryTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/WhereFilterFactoryTest.java @@ -165,7 +165,7 @@ public void testIcase() { t = TstUtils.testRefreshingTable( TableTools.col("Opra", "opra1", "opra2", "opra3", "Opra1", "Opra2", "Opra3", "Opra4", null, "OpRa5"), - TstUtils.colGrouped("Food", "Apple", "Orange", "bacon", "laffa", "pOtato", "carroT", "WafflE", null, + TstUtils.colIndexed("Food", "Apple", "Orange", "bacon", "laffa", "pOtato", "carroT", "WafflE", null, "Apple")); f = WhereFilterFactory.getExpression("Food icase in `apple`, `orange`, `bacon`,`LAFFA`"); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestChunkedRegionedOperations.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestChunkedRegionedOperations.java index 644740f8d3b..e3fd4cb145f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestChunkedRegionedOperations.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestChunkedRegionedOperations.java @@ -239,7 +239,7 @@ public void setUp() throws Exception { partitionedDataDefinition.getWritable(), parquetInstructions, partitionedInputDestinations, - CollectionUtil.ZERO_LENGTH_STRING_ARRAY); + CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY); final PartitionedTable partitionedInputMissingData = inputMissingData.view("PC", "II").partitionBy("PC"); final File[] partitionedInputMissingDestinations; @@ -255,7 +255,7 @@ public void setUp() throws Exception { partitionedMissingDataDefinition.getWritable(), parquetInstructions, partitionedInputMissingDestinations, - CollectionUtil.ZERO_LENGTH_STRING_ARRAY); + CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY); expected = TableTools .merge( diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java index 5e83c445ad1..55ffd9960b5 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TestRegionedColumnSourceManager.java @@ -3,20 +3,28 @@ // package io.deephaven.engine.table.impl.sources.regioned; +import gnu.trove.map.TIntIntMap; +import gnu.trove.map.hash.TIntIntHashMap; import io.deephaven.base.verify.AssertionFailure; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.ReferenceCountedLivenessNode; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.WritableRowSet; -import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.ColumnToCodecMappings; -import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; -import io.deephaven.engine.table.impl.locations.*; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.table.impl.locations.ColumnLocation; +import io.deephaven.engine.table.impl.locations.ImmutableTableLocationKey; +import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.locations.TableLocation; import io.deephaven.engine.table.impl.locations.impl.SimpleTableLocationKey; import io.deephaven.engine.table.impl.locations.impl.TableLocationUpdateSubscriptionBuffer; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetFactory; -import gnu.trove.map.TIntIntMap; -import gnu.trove.map.hash.TIntIntHashMap; +import io.deephaven.engine.testutil.ControlledUpdateGraph; +import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; +import io.deephaven.qst.column.Column; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import org.jmock.api.Invocation; import org.jmock.lib.action.CustomAction; import org.junit.Before; @@ -26,9 +34,9 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -import static io.deephaven.engine.testutil.TstUtils.assertRowSetEquals; import static io.deephaven.engine.table.impl.locations.TableLocationState.NULL_SIZE; import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.REGION_CAPACITY_IN_ELEMENTS; +import static io.deephaven.engine.testutil.TstUtils.assertRowSetEquals; /** * Tests for {@link RegionedColumnSourceManager}. @@ -45,7 +53,9 @@ public class TestRegionedColumnSourceManager extends RefreshingTableTestCase { private static final int NORMAL_INDEX = 2; @SuppressWarnings("FieldCanBeLocal") - private static boolean PRINT_STACK_TRACES = false; + private final static boolean PRINT_STACK_TRACES = false; + + private static final String ROW_SET_COLUMN_NAME = "RowSet"; private RegionedTableComponentFactory componentFactory; @@ -54,10 +64,10 @@ public class TestRegionedColumnSourceManager extends RefreshingTableTestCase { private ColumnDefinition groupingColumnDefinition; private ColumnDefinition normalColumnDefinition; - private RegionedColumnSource[] columnSources; - private RegionedColumnSource partitioningColumnSource; - private RegionedColumnSource groupingColumnSource; - private RegionedColumnSource normalColumnSource; + private RegionedColumnSource[] columnSources; + private RegionedColumnSource partitioningColumnSource; + private RegionedColumnSource groupingColumnSource; + private RegionedColumnSource normalColumnSource; private ColumnLocation[][] columnLocations; @@ -69,16 +79,20 @@ public class TestRegionedColumnSourceManager extends RefreshingTableTestCase { private TableLocation duplicateTableLocation0A; - private Map partitioningColumnGrouping; - private KeyRangeGroupingProvider groupingColumnGroupingProvider; + private RowSet capturedRowSet; + private DataIndex capturedPartitioningColumnIndex; + private DataIndex capturedGroupingColumnIndex; private TableLocationUpdateSubscriptionBuffer[] subscriptionBuffers; private long[] lastSizes; private int regionCount; private TIntIntMap locationIndexToRegionIndex; - private RowSet expectedRowSet; + private WritableRowSet expectedRowSet; private RowSet expectedAddedRowSet; - private Map expectedPartitioningColumnGrouping; + private Map expectedPartitioningColumnIndex; + private Map expectedGroupingColumnIndex; + + private ControlledUpdateGraph updateGraph; private RegionedColumnSourceManager SUT; @@ -89,7 +103,7 @@ public void setUp() throws Exception { componentFactory = mock(RegionedTableComponentFactory.class); partitioningColumnDefinition = ColumnDefinition.ofString("RCS_0").withPartitioning(); - groupingColumnDefinition = ColumnDefinition.ofString("RCS_1").withGrouping(); + groupingColumnDefinition = ColumnDefinition.ofString("RCS_1"); normalColumnDefinition = ColumnDefinition.ofString("RCS_2"); columnDefinitions = List.of(partitioningColumnDefinition, groupingColumnDefinition, normalColumnDefinition); @@ -106,6 +120,10 @@ public void setUp() throws Exception { oneOf(componentFactory).createRegionedColumnSource(with(same(partitioningColumnDefinition)), with(ColumnToCodecMappings.EMPTY)); will(returnValue(partitioningColumnSource)); + allowing(partitioningColumnSource).getType(); + will(returnValue(partitioningColumnDefinition.getDataType())); + allowing(partitioningColumnSource).getComponentType(); + will(returnValue(partitioningColumnDefinition.getComponentType())); oneOf(componentFactory).createRegionedColumnSource(with(same(groupingColumnDefinition)), with(ColumnToCodecMappings.EMPTY)); will(returnValue(groupingColumnSource)); @@ -140,9 +158,12 @@ public void setUp() throws Exception { Arrays.fill(lastSizes, -1); // Not null size regionCount = 0; locationIndexToRegionIndex = new TIntIntHashMap(4, 0.5f, -1, -1); - expectedRowSet = RowSetFactory.empty(); + expectedRowSet = RowSetFactory.empty().toTracking(); expectedAddedRowSet = RowSetFactory.empty(); - expectedPartitioningColumnGrouping = new LinkedHashMap<>(); + expectedPartitioningColumnIndex = new LinkedHashMap<>(); + expectedGroupingColumnIndex = new LinkedHashMap<>(); + + updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); } private ImmutableTableLocationKey makeTableKey(@NotNull final String internalPartitionValue, @@ -178,6 +199,10 @@ public Object invoke(Invocation invocation) { return RowSetFactory.flat(lastSizes[li]); } }); + allowing(tl).getDataIndexColumns(); + will(returnValue(Collections.singletonList((new String[] {groupingColumnDefinition.getName()})))); + allowing(tl).hasDataIndex(groupingColumnDefinition.getName()); + will(returnValue(true)); } }); IntStream.range(0, NUM_COLUMNS).forEach(ci -> { @@ -194,34 +219,22 @@ public Object invoke(Invocation invocation) { return tl; } - private Map makeColumnSourceMap() { - final Map result = new LinkedHashMap<>(); + private Map> makeColumnSourceMap() { + final Map> result = new LinkedHashMap<>(); IntStream.range(0, columnDefinitions.size()) .forEachOrdered(ci -> result.put(columnDefinitions.get(ci).getName(), columnSources[ci])); return result; } - private void expectPartitioningColumnInitialGrouping() { - partitioningColumnGrouping = null; - checking(new Expectations() { - { - allowing(partitioningColumnSource).getGroupToRange(); - will(new CustomAction("Return previously set partitioning column grouping") { - @Override - public Object invoke(Invocation invocation) { - return partitioningColumnGrouping; - } - }); - oneOf(partitioningColumnSource).setGroupToRange(with(any(Map.class))); - will(new CustomAction("Capture partitioning column grouping") { - @Override - public Object invoke(Invocation invocation) { - partitioningColumnGrouping = (Map) invocation.getParameter(0); - return null; - } - }); - } - }); + private void captureIndexes(@NotNull final RowSet rowSet) { + capturedRowSet = rowSet; + if (rowSet.isTracking()) { + final DataIndexer dataIndexer = DataIndexer.existingOf(rowSet.trackingCast()); + capturedPartitioningColumnIndex = + dataIndexer == null ? null : dataIndexer.getDataIndex(partitioningColumnSource); + capturedGroupingColumnIndex = + dataIndexer == null ? null : dataIndexer.getDataIndex(groupingColumnSource); + } } private void expectPoison() { @@ -234,32 +247,9 @@ private void expectPoison() { }); } - private void expectGroupingColumnInitialGrouping() { - groupingColumnGroupingProvider = null; - checking(new Expectations() { - { - allowing(groupingColumnSource).getGroupingProvider(); - will(new CustomAction("Return previously set grouping column grouping provider") { - @Override - public Object invoke(Invocation invocation) { - return groupingColumnGroupingProvider; - } - }); - oneOf(groupingColumnSource).setGroupingProvider(with(any(GroupingProvider.class))); - will(new CustomAction("Capture grouping column grouping provider") { - @Override - public Object invoke(Invocation invocation) { - groupingColumnGroupingProvider = (KeyRangeGroupingProvider) invocation.getParameter(0); - return null; - } - }); - } - }); - } - - private void setSizeExpectations(final boolean refreshing, final long... sizes) { - final WritableRowSet newExpectedRowSet = RowSetFactory.empty(); - expectedPartitioningColumnGrouping = new LinkedHashMap<>(); + private void setSizeExpectations(final boolean refreshing, final boolean success, final long... sizes) { + final WritableRowSet newExpectedRowSet = RowSetFactory.empty().toTracking(); + final Map newExpectedPartitioningColumnIndex = new LinkedHashMap<>(); IntStream.range(0, sizes.length).forEachOrdered(li -> { final long size = sizes[li]; final long lastSize = lastSizes[li]; @@ -334,29 +324,54 @@ public Object invoke(Invocation invocation) { newExpectedRowSet.insertRange( RegionedColumnSource.getFirstRowKey(regionIndex), RegionedColumnSource.getFirstRowKey(regionIndex) + size - 1); - expectedPartitioningColumnGrouping.computeIfAbsent(cp, cpk -> RowSetFactory.empty()) - .insertRange( - RegionedColumnSource.getFirstRowKey(regionIndex), - RegionedColumnSource.getFirstRowKey(regionIndex) + size - 1); + if (success) { + // noinspection resource + newExpectedPartitioningColumnIndex.computeIfAbsent(cp, cpk -> RowSetFactory.empty()) + .insertRange( + RegionedColumnSource.getFirstRowKey(regionIndex), + RegionedColumnSource.getFirstRowKey(regionIndex) + size - 1); + } } }); - expectedAddedRowSet = newExpectedRowSet.minus(expectedRowSet); - expectedRowSet = newExpectedRowSet; + if (success) { + expectedAddedRowSet = newExpectedRowSet.minus(expectedRowSet); + expectedRowSet.clear(); + expectedRowSet.insert(newExpectedRowSet); + expectedPartitioningColumnIndex = newExpectedPartitioningColumnIndex; + } else { + expectedAddedRowSet = null; + } } - private void checkIndexes(@NotNull final RowSet addedRowSet) { + private void checkIndexes() { assertIsSatisfied(); - assertRowSetEquals(expectedAddedRowSet, addedRowSet); - if (partitioningColumnGrouping == null) { - assertTrue(expectedPartitioningColumnGrouping.isEmpty()); + if (capturedRowSet == null) { + assertNull(expectedAddedRowSet); } else { - assertEquals(expectedPartitioningColumnGrouping.keySet(), partitioningColumnGrouping.keySet()); - expectedPartitioningColumnGrouping - .forEach((final String columnPartition, final RowSet expectedGrouping) -> { - final RowSet grouping = partitioningColumnGrouping.get(columnPartition); - assertRowSetEquals(expectedGrouping, grouping); - }); + assertRowSetEquals(expectedAddedRowSet, capturedRowSet); } + checkIndex(expectedPartitioningColumnIndex, capturedPartitioningColumnIndex); + checkIndex(expectedGroupingColumnIndex, capturedGroupingColumnIndex); + capturedRowSet = null; + } + + private static void checkIndex( + @NotNull final Map expected, + @Nullable final DataIndex index) { + if (index == null) { + assertTrue(expected.isEmpty()); + return; + } + final Table indexTable = index.table(); + final DataIndex.RowKeyLookup rowKeyLookup = index.rowKeyLookup(); + final ColumnSource rowSets = indexTable.getColumnSource(index.rowSetColumnName(), RowSet.class); + assertEquals(expected.size(), indexTable.size()); + expected.forEach((final String expectedKey, final RowSet expectedRows) -> { + final long indexRowKey = rowKeyLookup.apply(expectedKey, false); + final RowSet indexRows = rowSets.get(indexRowKey); + assertNotNull(indexRows); + assertRowSetEquals(expectedRows, indexRows); + }); } @Test @@ -399,14 +414,72 @@ public void testStaticBasics() { assertTrue(SUT.includedLocations().isEmpty()); // Test run - expectPartitioningColumnInitialGrouping(); - expectGroupingColumnInitialGrouping(); - setSizeExpectations(false, NULL_SIZE, 100, 0, REGION_CAPACITY_IN_ELEMENTS); + setSizeExpectations(false, true, NULL_SIZE, 100, 0, REGION_CAPACITY_IN_ELEMENTS); - checkIndexes(SUT.refresh()); + try (final RowSet first = RowSetFactory.fromRange(0, 49); + final RowSet second = RowSetFactory.fromRange(50, 99); + final RowSet third = RowSetFactory.fromRange(50, REGION_CAPACITY_IN_ELEMENTS)) { + checking(new Expectations() { + { + oneOf(tableLocation1A).getDataIndex(groupingColumnDefinition.getName()); + will(returnValue(new DataIndexImpl(TableFactory.newTable( + Column.of(groupingColumnDefinition.getName(), "ABC", "DEF"), + Column.of(ROW_SET_COLUMN_NAME, RowSet.class, first.copy(), second.copy()))))); + oneOf(tableLocation1B).getDataIndex(groupingColumnDefinition.getName()); + will(returnValue(new DataIndexImpl(TableFactory.newTable( + Column.of(groupingColumnDefinition.getName(), "DEF", "XYZ"), + Column.of(ROW_SET_COLUMN_NAME, RowSet.class, first.copy(), third.copy()))))); + } + }); + + expectedGroupingColumnIndex.put("ABC", first.copy()); + expectedGroupingColumnIndex.put("DEF", second.copy()); + expectedGroupingColumnIndex.get("DEF").insertWithShift(RegionedColumnSource.getFirstRowKey(1), first); + expectedGroupingColumnIndex.put("XYZ", third.shift(RegionedColumnSource.getFirstRowKey(1))); + } + + captureIndexes(SUT.initialize()); + capturedGroupingColumnIndex.table(); // Force us to build the merged index *before* we check satisfaction + + checkIndexes(); assertEquals(Arrays.asList(tableLocation1A, tableLocation1B), SUT.includedLocations()); } + private final class DataIndexImpl extends ReferenceCountedLivenessNode implements BasicDataIndex { + + private final Table table; + + private DataIndexImpl(@NotNull final Table table) { + super(false); + this.table = table; + } + + @Override + public @NotNull List keyColumnNames() { + return List.of(groupingColumnDefinition.getName()); + } + + @Override + public @NotNull Map, String> keyColumnNamesByIndexedColumn() { + return Map.of(groupingColumnSource, groupingColumnDefinition.getName()); + } + + @Override + public @NotNull String rowSetColumnName() { + return ROW_SET_COLUMN_NAME; + } + + @Override + public @NotNull Table table() { + return table; + } + + @Override + public boolean isRefreshing() { + return false; + } + } + @Test public void testStaticOverflow() { SUT = new RegionedColumnSourceManager(false, componentFactory, ColumnToCodecMappings.EMPTY, columnDefinitions); @@ -424,7 +497,8 @@ public void testStaticOverflow() { } }); try { - SUT.refresh(); + // noinspection resource + SUT.initialize(); fail("Expected exception"); } catch (TableDataException expected) { maybePrintStackTrace(expected); @@ -440,22 +514,9 @@ public void testRefreshing() { assertTrue(SUT.allLocations().isEmpty()); assertTrue(SUT.includedLocations().isEmpty()); - // Disable grouping, as we don't maintain it for refreshing instances - checking(new Expectations() { - { - oneOf(groupingColumnSource).setGroupingProvider(null); - oneOf(groupingColumnSource).setGroupToRange(null); - } - }); - SUT.disableGrouping(); - assertIsSatisfied(); - - // Do it a second time, to test that it's a no-op - SUT.disableGrouping(); - assertIsSatisfied(); - // Check run with no locations - checkIndexes(SUT.refresh()); + captureIndexes(SUT.initialize()); + checkIndexes(); // Add a few locations Arrays.stream(tableLocations).limit(2).forEach(SUT::addLocation); @@ -463,19 +524,21 @@ public void testRefreshing() { assertTrue(SUT.includedLocations().isEmpty()); // Refresh them - expectPartitioningColumnInitialGrouping(); - setSizeExpectations(true, 5, 1000); - checkIndexes(SUT.refresh()); + setSizeExpectations(true, true, 5, 1000); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations()); // Refresh them with no change - setSizeExpectations(true, 5, 1000); - checkIndexes(SUT.refresh()); + setSizeExpectations(true, true, 5, 1000); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations()); // Refresh them with a change for the subscription-supporting one - setSizeExpectations(true, 5, 1001); - checkIndexes(SUT.refresh()); + setSizeExpectations(true, true, 5, 1001); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations()); // Try adding a duplicate @@ -504,71 +567,87 @@ public void testRefreshing() { assertEquals(Arrays.asList(tableLocation0A, tableLocation1A), SUT.includedLocations()); // Test run with new locations included - setSizeExpectations(true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE); - checkIndexes(SUT.refresh()); + setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B), SUT.includedLocations()); // Test no-op run - setSizeExpectations(true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE); - checkIndexes(SUT.refresh()); + setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B), SUT.includedLocations()); // Test run with a location updated from null to not - setSizeExpectations(true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 2); - checkIndexes(SUT.refresh()); + setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 2); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B), SUT.includedLocations()); // Test run with a location updated - setSizeExpectations(true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 10000002); - checkIndexes(SUT.refresh()); + setSizeExpectations(true, true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 10000002); + updateGraph.runWithinUnitTestCycle(() -> captureIndexes(SUT.refresh())); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B), SUT.includedLocations()); // Test run with a size decrease - setSizeExpectations(true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 2); + setSizeExpectations(true, false, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, 2); expectPoison(); - try { - checkIndexes(SUT.refresh()); - fail("Expected exception"); - } catch (AssertionFailure expected) { - maybePrintStackTrace(expected); - } + updateGraph.runWithinUnitTestCycle(() -> { + try { + SUT.refresh(); + fail("Expected exception"); + } catch (AssertionFailure expected) { + maybePrintStackTrace(expected); + } + }); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B), SUT.includedLocations()); // Test run with a location truncated - setSizeExpectations(true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE); + setSizeExpectations(true, false, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, NULL_SIZE); expectPoison(); - try { - checkIndexes(SUT.refresh()); - fail("Expected exception"); - } catch (TableDataException expected) { - maybePrintStackTrace(expected); - } + updateGraph.runWithinUnitTestCycle(() -> { + try { + SUT.refresh(); + fail("Expected exception"); + } catch (TableDataException expected) { + maybePrintStackTrace(expected); + } + }); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B), SUT.includedLocations()); // Test run with an overflow - setSizeExpectations(true, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, REGION_CAPACITY_IN_ELEMENTS + 1); - try { - checkIndexes(SUT.refresh()); - fail("Expected exception"); - } catch (TableDataException expected) { - maybePrintStackTrace(expected); - } + setSizeExpectations(true, false, 5, REGION_CAPACITY_IN_ELEMENTS, 5003, REGION_CAPACITY_IN_ELEMENTS + 1); + updateGraph.runWithinUnitTestCycle(() -> { + try { + SUT.refresh(); + fail("Expected exception"); + } catch (TableDataException expected) { + maybePrintStackTrace(expected); + } + }); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B), SUT.includedLocations()); // Test run with an exception subscriptionBuffers[3].handleException(new TableDataException("TEST")); expectPoison(); - try { - checkIndexes(SUT.refresh()); - fail("Expected exception"); - } catch (TableDataException expected) { - assertEquals("TEST", expected.getCause().getMessage()); - } + updateGraph.runWithinUnitTestCycle(() -> { + try { + SUT.refresh(); + fail("Expected exception"); + } catch (TableDataException expected) { + assertEquals("TEST", expected.getCause().getMessage()); + } + }); + checkIndexes(); assertEquals(Arrays.asList(tableLocation0A, tableLocation1A, tableLocation0B, tableLocation1B), SUT.includedLocations()); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TstRegionedColumnSourcePrimitive.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TstRegionedColumnSourcePrimitive.java index 39142645b1f..0c0a32aeb91 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TstRegionedColumnSourcePrimitive.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/regioned/TstRegionedColumnSourcePrimitive.java @@ -6,16 +6,12 @@ import io.deephaven.base.testing.BaseCachedJMockTestCase; import io.deephaven.base.verify.RequirementFailure; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.impl.locations.GroupingProvider; -import io.deephaven.engine.rowset.RowSet; import junit.framework.TestCase; import org.junit.After; import org.junit.Before; import org.junit.Test; import java.lang.reflect.Array; -import java.util.Collections; -import java.util.Map; import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.getFirstRowKey; import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.getLastRowKey; @@ -133,47 +129,6 @@ public void testAddRegions() { } } - @Test - public void testDeferredGrouping() { - TestCase.assertNull(SUT.getGroupToRange()); - - final Map dummyGrouping = Collections.emptyMap(); - SUT.setGroupToRange(dummyGrouping); - TestCase.assertEquals(dummyGrouping, SUT.getGroupToRange()); - SUT.setGroupToRange(null); - TestCase.assertNull(SUT.getGroupToRange()); - - // noinspection unchecked - final GroupingProvider groupingProvider = mock(GroupingProvider.class); - - SUT.setGroupingProvider(groupingProvider); - checking(new Expectations() { - { - oneOf(groupingProvider).getGroupToRange(); - will(returnValue(null)); - } - }); - TestCase.assertNull(SUT.getGroupToRange()); - assertIsSatisfied(); - TestCase.assertNull(SUT.getGroupToRange()); - assertIsSatisfied(); - - SUT.setGroupingProvider(groupingProvider); - checking(new Expectations() { - { - oneOf(groupingProvider).getGroupToRange(); - will(returnValue(dummyGrouping)); - } - }); - TestCase.assertEquals(dummyGrouping, SUT.getGroupToRange()); - assertIsSatisfied(); - TestCase.assertEquals(dummyGrouping, SUT.getGroupToRange()); - assertIsSatisfied(); - SUT.setGroupToRange(null); - TestCase.assertNull(SUT.getGroupToRange()); - assertIsSatisfied(); - } - @Test public void testGetNegativeOne() { TestCase.assertNull(SUT.get(-1)); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/BaseUpdateByTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/BaseUpdateByTest.java index dfe41243949..8f8054f8925 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/BaseUpdateByTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/BaseUpdateByTest.java @@ -4,17 +4,18 @@ package io.deephaven.engine.table.impl.updateby; import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.testutil.ColumnInfo; import io.deephaven.engine.testutil.generator.*; - import io.deephaven.engine.testutil.junit4.EngineCleanup; import org.junit.Rule; import java.math.BigInteger; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; import static io.deephaven.engine.testutil.TstUtils.getTable; import static io.deephaven.engine.testutil.TstUtils.initColumnInfos; @@ -80,16 +81,12 @@ static CreateResult createTestTable(int tableSize, final Random random = new Random(seed); final ColumnInfo[] columnInfos = initColumnInfos(colsList.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY), generators.toArray(new TestDataGenerator[0])); - final QueryTable t = getTable(tableSize, random, columnInfos); + final QueryTable t = getTable(isRefreshing, tableSize, random, columnInfos); if (!isRefreshing && includeGroups) { - final AbstractColumnSource groupingSource = (AbstractColumnSource) t.getColumnSource("Sym"); - final Map gtr = groupingSource.getValuesMapping(t.getRowSet()); - groupingSource.setGroupToRange(gtr); + DataIndexer.getOrCreateDataIndex(t, "Sym"); } - t.setRefreshing(isRefreshing); - return new CreateResult(t, columnInfos, random); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingProduct.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingProduct.java index 97fb7e0ba49..9c321168d39 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingProduct.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingProduct.java @@ -10,12 +10,10 @@ import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.context.QueryScope; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.AbstractColumnSource; import io.deephaven.engine.table.impl.DataAccessHelpers; import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.testutil.*; import io.deephaven.engine.testutil.generator.*; import io.deephaven.engine.util.TableDiff; @@ -149,9 +147,7 @@ static CreateResult createSmallTestTable(int tableSize, final QueryTable t = getTable(tableSize, random, columnInfos); if (!isRefreshing && includeGroups) { - final AbstractColumnSource groupingSource = (AbstractColumnSource) t.getColumnSource("Sym"); - final Map gtr = groupingSource.getValuesMapping(t.getRowSet()); - groupingSource.setGroupToRange(gtr); + DataIndexer.getOrCreateDataIndex(t, "Sym"); } t.setRefreshing(isRefreshing); diff --git a/engine/table/src/test/java/io/deephaven/engine/updategraph/impl/TestEventDrivenUpdateGraph.java b/engine/table/src/test/java/io/deephaven/engine/updategraph/impl/TestEventDrivenUpdateGraph.java index 647a1b02cc0..206b6c73f16 100644 --- a/engine/table/src/test/java/io/deephaven/engine/updategraph/impl/TestEventDrivenUpdateGraph.java +++ b/engine/table/src/test/java/io/deephaven/engine/updategraph/impl/TestEventDrivenUpdateGraph.java @@ -168,7 +168,6 @@ public void testSimpleModify() { public void testUpdatePerformanceTracker() { final Table upt = UpdatePerformanceTracker.getQueryTable(); - final EventDrivenUpdateGraph eventDrivenUpdateGraph1 = EventDrivenUpdateGraph.newBuilder("TestEDUG1").build(); final EventDrivenUpdateGraph eventDrivenUpdateGraph2 = EventDrivenUpdateGraph.newBuilder("TestEDUG2").build(); @@ -176,8 +175,6 @@ public void testUpdatePerformanceTracker() { eventDrivenUpdateGraph1.requestRefresh(); eventDrivenUpdateGraph2.requestRefresh(); - final long start = System.currentTimeMillis(); - final int count1 = 10; final int count2 = 20; final int time1 = 10; @@ -211,7 +208,7 @@ public void testUpdatePerformanceTracker() { inRange = defaultUpdateGraph.sharedLock().computeLocked(() -> uptAgged.update( "EIUExpectedMillis = UpdateGraph==`TestEDUG1` ? " + time1 + " : " + time2, "TotalExpectedTime=InvocationCount * EIUExpectedMillis * 1_000_000L", - "InRange=(UsageNanos > 0.9 * TotalExpectedTime) && (UsageNanos < 1.5 * TotalExpectedTime)")); + "InRange=(UsageNanos > 0.9 * TotalExpectedTime) && (UsageNanos < 2.5 * TotalExpectedTime)")); } TableTools.show(inRange); diff --git a/engine/table/src/test/java/io/deephaven/engine/util/TimeTableTest.java b/engine/table/src/test/java/io/deephaven/engine/util/TimeTableTest.java index 2e630943ce7..d0d17ae6291 100644 --- a/engine/table/src/test/java/io/deephaven/engine/util/TimeTableTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/util/TimeTableTest.java @@ -8,25 +8,33 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LivenessScope; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.primitive.iterator.CloseableIterator; +import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfLong; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.testutil.ControlledUpdateGraph; -import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; import io.deephaven.engine.table.impl.TableUpdateValidator; import io.deephaven.engine.table.impl.TimeTable; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.table.impl.select.ReinterpretedColumn; import io.deephaven.engine.table.impl.sources.FillUnordered; +import io.deephaven.engine.table.iterators.ChunkedLongColumnIterator; +import io.deephaven.engine.testutil.ControlledUpdateGraph; +import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; import io.deephaven.engine.updategraph.UpdateSourceCombiner; import io.deephaven.time.DateTimeUtils; +import io.deephaven.util.SafeCloseable; import org.junit.Assert; import org.junit.Test; import java.time.Instant; import java.util.Arrays; -import java.util.Map; +import java.util.List; +import java.util.Objects; public class TimeTableTest extends RefreshingTableTestCase { @@ -219,45 +227,88 @@ public void testColumnSourceMatch() { .map(l -> l == null ? null : DateTimeUtils.epochNanosToInstant(l)) .toArray(Instant[]::new); try (final RowSet match = - dtColumn.match(false, false, false, RowSetFactory.fromRange(100, 110), (Object[]) keys)) { + dtColumn.match(false, false, false, null, RowSetFactory.fromRange(100, 110), + (Object[]) keys)) { Assert.assertEquals(match, RowSetFactory.fromKeys(100, 105, 110)); } try (final RowSet match = - column.match(false, false, false, RowSetFactory.fromRange(100, 110), (Object[]) longKeys)) { + column.match(false, false, false, null, RowSetFactory.fromRange(100, 110), + (Object[]) longKeys)) { Assert.assertEquals(match, RowSetFactory.fromKeys(100, 105, 110)); } // inverted try (final RowSet match = - dtColumn.match(true, false, false, RowSetFactory.fromRange(100, 110), (Object[]) keys)) { + dtColumn.match(true, false, false, null, RowSetFactory.fromRange(100, 110), + (Object[]) keys)) { Assert.assertEquals(match, RowSetFactory.fromKeys(101, 102, 103, 104, 106, 107, 108, 109)); } try (final RowSet match = - column.match(true, false, false, RowSetFactory.fromRange(100, 110), (Object[]) longKeys)) { + column.match(true, false, false, null, RowSetFactory.fromRange(100, 110), + (Object[]) longKeys)) { Assert.assertEquals(match, RowSetFactory.fromKeys(101, 102, 103, 104, 106, 107, 108, 109)); } } @Test - public void testGetValuesMapping() { + public void testIndexing() { build(TimeTable.newBuilder().period(10)); final ColumnSource dtColumn = timeTable.getColumnSource("Timestamp"); tick(0); tick(2000); Assert.assertEquals(timeTable.size(), 201); - final Map dtMap = dtColumn.getValuesMapping(RowSetFactory.fromRange(100, 109)); - Assert.assertEquals(dtMap.size(), 10); - dtMap.forEach((tm, rows) -> { - Assert.assertEquals(rows.size(), 1); - Assert.assertEquals(dtColumn.get(rows.firstRowKey()), tm); - }); - - Map longMap = column.getValuesMapping(RowSetFactory.fromRange(100, 109)); - Assert.assertEquals(longMap.size(), 10); - longMap.forEach((tm, rows) -> { - Assert.assertEquals(rows.size(), 1); - Assert.assertEquals(column.get(rows.firstRowKey()), tm); - }); + final DataIndexer dataIndexer = DataIndexer.of(timeTable.getRowSet()); + + // Create the index for this table and column. + try (final SafeCloseable ignored = LivenessScopeStack.open(new LivenessScope(true), true); + final RowSet intersectRowSet = RowSetFactory.fromRange(100, 109)) { + DataIndexer.getOrCreateDataIndex(timeTable, "Timestamp"); + + final BasicDataIndex dataIndex = + dataIndexer.getDataIndex(dtColumn).transform( + DataIndexTransformer.builder() + .intersectRowSet(intersectRowSet) + .build()); + final Table indexTable = dataIndex.table(); + + Assert.assertEquals(indexTable.size(), 10); + try (final CloseableIterator keyIt = indexTable.columnIterator(dataIndex.keyColumnNames().get(0)); + final CloseableIterator rsIt = indexTable.columnIterator(dataIndex.rowSetColumnName())) { + while (keyIt.hasNext()) { + final Instant key = keyIt.next(); + final RowSet rs = rsIt.next(); + Assert.assertEquals(rs.size(), 1); + Assert.assertEquals(dtColumn.get(rs.firstRowKey()), key); + } + } + + final Table riTable = timeTable.updateView(List.of( + new ReinterpretedColumn<>("Timestamp", Instant.class, "longTimestamp", long.class))); + + // Create the index for this table and column. + DataIndexer.getOrCreateDataIndex(riTable, "longTimestamp"); + + final BasicDataIndex longDataIndex = + Objects.requireNonNull(DataIndexer.getDataIndex(riTable, "longTimestamp")).transform( + DataIndexTransformer.builder() + .intersectRowSet(intersectRowSet) + .build()); + final Table longIndexTable = longDataIndex.table(); + + Assert.assertEquals(longIndexTable.size(), 10); + try (final CloseablePrimitiveIteratorOfLong keyIt = new ChunkedLongColumnIterator( + longIndexTable.getColumnSource(longDataIndex.keyColumnNames().get(0), long.class), + longIndexTable.getRowSet()); + final CloseableIterator rsIt = + longIndexTable.columnIterator(longDataIndex.rowSetColumnName())) { + while (keyIt.hasNext()) { + final long key = keyIt.nextLong(); + final RowSet rs = rsIt.next(); + Assert.assertEquals(rs.size(), 1); + Assert.assertEquals(column.getLong(rs.firstRowKey()), key); + } + } + } } @Test diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/ColumnInfo.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/ColumnInfo.java index 76843e16a2c..c7710bffbaf 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/ColumnInfo.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/ColumnInfo.java @@ -22,12 +22,26 @@ public class ColumnInfo { final TestDataGenerator generator; final String name; final boolean immutable; - final boolean grouped; + final boolean indexed; final static ColAttributes[] ZERO_LENGTH_COLUMN_ATTRIBUTES_ARRAY = new ColAttributes[0]; public enum ColAttributes { - None, Immutable, Grouped + /** + * This attribute has no effect. + */ + None, + /** + * This attribute indicates that the column is immutable. + */ + Immutable, + /** + * This attribute indicates that the column should be indexed. Only use this when enclosed by a + * {@link io.deephaven.engine.liveness.LivenessScope} that was constructed with + * {@code enforceStrongReachability == true}. + */ + Indexed + } public ColumnInfo(TestDataGenerator generator, String name, ColAttributes... colAttributes) { @@ -37,7 +51,7 @@ public ColumnInfo(TestDataGenerator generator, String name, ColAttributes. this.generator = generator; this.name = name; this.immutable = Arrays.asList(colAttributes).contains(ColAttributes.Immutable); - this.grouped = Arrays.asList(colAttributes).contains(ColAttributes.Grouped); + this.indexed = Arrays.asList(colAttributes).contains(ColAttributes.Indexed); } public ColumnHolder generateInitialColumn(RowSet rowSet, Random random) { @@ -45,14 +59,14 @@ public ColumnHolder generateInitialColumn(RowSet rowSet, Random random) { if (dataType == Long.class && type == Instant.class) { Require.eqFalse(immutable, "immutable"); - Require.eqFalse(grouped, "grouped"); + Require.eqFalse(indexed, "indexed"); return ColumnHolder.getInstantColumnHolder(name, false, initialData); } if (immutable) { - return new ImmutableColumnHolder<>(name, type, componentType, grouped, initialData); - } else if (grouped) { - return TstUtils.groupedColumnHolderForChunk(name, type, componentType, initialData); + return new ImmutableColumnHolder<>(name, type, componentType, indexed, initialData); + } else if (indexed) { + return TstUtils.indexedColumnHolderForChunk(name, type, componentType, initialData); } else { return TstUtils.columnHolderForChunk(name, type, componentType, initialData); } @@ -68,8 +82,8 @@ public void shift(long start, long end, long delta) { public ColumnHolder generateUpdateColumnHolder(RowSet keysToModify, Random random) { final Chunk chunk = generator.populateChunk(keysToModify, random); - if (grouped) { - return TstUtils.groupedColumnHolderForChunk(name, type, componentType, chunk); + if (indexed) { + return TstUtils.indexedColumnHolderForChunk(name, type, componentType, chunk); } else { return TstUtils.columnHolderForChunk(name, type, componentType, chunk); } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/TstUtils.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/TstUtils.java index 1e3d8656258..ee01c2f408f 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/TstUtils.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/TstUtils.java @@ -24,6 +24,7 @@ import io.deephaven.engine.table.impl.NoSuchColumnException.Type; import io.deephaven.engine.table.impl.PrevColumnSource; import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.select.Formula; import io.deephaven.engine.table.impl.sources.RedirectedColumnSource; import io.deephaven.engine.table.impl.sources.ViewColumnSource; @@ -111,15 +112,15 @@ public static ColumnHolder columnHolderForChunk( } /** - * Create a grouped column holder from the given chunk. + * Create an indexed column holder from the given chunk. * * @param name the name of the column * @param type the type of the column * @param componentType the component type of the column if applicable * @param chunkData the data in an chunk for this column - * @return the new ColumnHolder with the grouping attribute set + * @return the new ColumnHolder with the indexed attribute set */ - public static ColumnHolder groupedColumnHolderForChunk( + public static ColumnHolder indexedColumnHolderForChunk( String name, Class type, Class componentType, Chunk chunkData) { return ColumnHolder.makeForChunk(name, type, componentType, true, chunkData); } @@ -209,7 +210,7 @@ public static void removeRows(Table table, RowSet rowSet) { } @SafeVarargs - public static ColumnHolder colGrouped(String name, T... data) { + public static ColumnHolder colIndexed(String name, T... data) { return ColumnHolder.createColumnHolder(name, true, data); } @@ -551,31 +552,82 @@ public static QueryTable getTable(int size, Random random, ColumnInfo[] co return getTable(true, size, random, columnInfos); } - public static QueryTable getTable(boolean refreshing, int size, Random random, ColumnInfo[] columnInfos) { + public static QueryTable getTable( + final boolean refreshing, + final int size, + @NotNull final Random random, + @NotNull final ColumnInfo[] columnInfos) { final TrackingWritableRowSet rowSet = getInitialIndex(size, random).toTracking(); - final ColumnHolder[] sources = new ColumnHolder[columnInfos.length]; + final ColumnHolder[] columnHolders = new ColumnHolder[columnInfos.length]; for (int i = 0; i < columnInfos.length; i++) { - sources[i] = columnInfos[i].generateInitialColumn(rowSet, random); + columnHolders[i] = columnInfos[i].generateInitialColumn(rowSet, random); } if (refreshing) { - return testRefreshingTable(rowSet, sources); + return testRefreshingTable(rowSet, columnHolders); } else { - return testTable(rowSet, sources); + return testTable(rowSet, columnHolders); } } - public static QueryTable testTable(ColumnHolder... columnHolders) { - final WritableRowSet rowSet = RowSetFactory.flat(columnHolders[0].size()); - return testTable(rowSet.toTracking(), columnHolders); - } - - public static QueryTable testTable(TrackingRowSet rowSet, ColumnHolder... columnHolders) { + private static QueryTable testTable( + final boolean refreshing, + final boolean flat, + @NotNull final TrackingRowSet rowSet, + @NotNull final ColumnHolder... columnHolders) { final Map> columns = getColumnSourcesFromHolders(rowSet, columnHolders); - QueryTable queryTable = new QueryTable(rowSet, columns); + final QueryTable queryTable = new QueryTable(rowSet, columns); queryTable.setAttribute(BaseTable.TEST_SOURCE_TABLE_ATTRIBUTE, true); + if (refreshing) { + queryTable.setRefreshing(true); + } + if (flat) { + Assert.assertion(rowSet.isFlat(), "rowSet.isFlat()"); + queryTable.setFlat(); + } + + // Add indexes for the indexed columns. + for (ColumnHolder columnHolder : columnHolders) { + if (columnHolder.indexed) { + // This mechanism is only safe in a reachability/liveness sense if we're enclosed in a LivenessScope + // that enforces strong reachability. + DataIndexer.getOrCreateDataIndex(queryTable, columnHolder.name); + } + } + return queryTable; } + public static QueryTable testTable( + @NotNull final TrackingRowSet rowSet, + @NotNull final ColumnHolder... columnHolders) { + return testTable(false, false, rowSet, columnHolders); + } + + public static QueryTable testRefreshingTable( + @NotNull final TrackingRowSet rowSet, + @NotNull final ColumnHolder... columnHolders) { + return testTable(true, false, rowSet, columnHolders); + } + + public static QueryTable testFlatRefreshingTable( + @NotNull final TrackingRowSet rowSet, + @NotNull final ColumnHolder... columnHolders) { + return testTable(true, true, rowSet, columnHolders); + } + + private static QueryTable testTable(final boolean refreshing, @NotNull final ColumnHolder... columnHolders) { + final WritableRowSet rowSet = RowSetFactory.flat(columnHolders[0].size()); + return testTable(refreshing, false, rowSet.toTracking(), columnHolders); + } + + public static QueryTable testTable(@NotNull final ColumnHolder... columnHolders) { + return testTable(false, columnHolders); + } + + public static QueryTable testRefreshingTable(@NotNull final ColumnHolder... columnHolders) { + return testTable(true, columnHolders); + } + @NotNull private static Map> getColumnSourcesFromHolders( TrackingRowSet rowSet, ColumnHolder[] columnHolders) { @@ -586,29 +638,6 @@ private static Map> getColumnSourcesFromHolders( return columns; } - public static QueryTable testRefreshingTable(TrackingRowSet rowSet, ColumnHolder... columnHolders) { - final QueryTable queryTable = testTable(rowSet, columnHolders); - queryTable.setRefreshing(true); - queryTable.setAttribute(BaseTable.TEST_SOURCE_TABLE_ATTRIBUTE, true); - return queryTable; - } - - public static QueryTable testFlatRefreshingTable(TrackingRowSet rowSet, ColumnHolder... columnHolders) { - Assert.assertion(rowSet.isFlat(), "rowSet.isFlat()", rowSet, "rowSet"); - return new QueryTable(rowSet, getColumnSourcesFromHolders(rowSet, columnHolders)) { - { - setRefreshing(true); - setFlat(); - } - }; - } - - public static QueryTable testRefreshingTable(ColumnHolder... columnHolders) { - final QueryTable queryTable = testTable(columnHolders); - queryTable.setRefreshing(true); - return queryTable; - } - public static ColumnSource getTestColumnSource(RowSet rowSet, ColumnHolder columnHolder) { return getTestColumnSourceFromChunk(rowSet, columnHolder, columnHolder.getChunk()); } @@ -675,9 +704,6 @@ private static ColumnSource getTestColumnSourceFromChunk( result = new ObjectTestSource<>(columnHolder.dataType, rowSet, chunkData); } } - if (columnHolder.grouped) { - result.setGroupToRange(result.getValuesMapping(rowSet)); - } return result; } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocation.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocation.java index c1aeee74411..0111fa62dbf 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocation.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/locations/TableBackedTableLocation.java @@ -3,6 +3,8 @@ // package io.deephaven.engine.testutil.locations; +import io.deephaven.api.SortColumn; +import io.deephaven.engine.table.BasicDataIndex; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.locations.ColumnLocation; import io.deephaven.engine.table.impl.locations.TableDataException; @@ -10,6 +12,9 @@ import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; import io.deephaven.engine.updategraph.UpdateSourceRegistrar; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.util.List; public final class TableBackedTableLocation extends AbstractTableLocation { @@ -60,7 +65,31 @@ public void refresh() { } @Override - protected @NotNull ColumnLocation makeColumnLocation(@NotNull final String name) { + @NotNull + public List getSortedColumns() { + return List.of(); + } + + @Override + @NotNull + public List getDataIndexColumns() { + return List.of(); + } + + @Override + public boolean hasDataIndex(@NotNull final String... columns) { + return false; + } + + @Override + @NotNull + protected ColumnLocation makeColumnLocation(@NotNull final String name) { return new TableBackedColumnLocation(this, name); } + + @Override + @Nullable + protected BasicDataIndex loadDataIndex(@NotNull final String... columns) { + return null; + } } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ByteTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ByteTestSource.java index 5f23d5e8ad5..0e6abdc9da8 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ByteTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ByteTestSource.java @@ -77,8 +77,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add @Override public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } @@ -126,8 +124,6 @@ private void maybeInitializePrevForStep() { @Override public synchronized void remove(RowSet rowSet) { - setGroupToRange(null); - maybeInitializePrevForStep(); rowSet.forAllRowKeys(data::remove); } @@ -135,7 +131,6 @@ public synchronized void remove(RowSet rowSet) { @Override public synchronized void shift(long startKeyInclusive, long endKeyInclusive, long shiftDelta) { maybeInitializePrevForStep(); - setGroupToRange(null); // Note: moving to the right, we need to start with rightmost data first. final long dir = shiftDelta > 0 ? -1 : 1; diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/CharTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/CharTestSource.java index 017c147f004..73755c88bc8 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/CharTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/CharTestSource.java @@ -73,8 +73,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add @Override public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } @@ -122,8 +120,6 @@ private void maybeInitializePrevForStep() { @Override public synchronized void remove(RowSet rowSet) { - setGroupToRange(null); - maybeInitializePrevForStep(); rowSet.forAllRowKeys(data::remove); } @@ -131,7 +127,6 @@ public synchronized void remove(RowSet rowSet) { @Override public synchronized void shift(long startKeyInclusive, long endKeyInclusive, long shiftDelta) { maybeInitializePrevForStep(); - setGroupToRange(null); // Note: moving to the right, we need to start with rightmost data first. final long dir = shiftDelta > 0 ? -1 : 1; diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/DoubleTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/DoubleTestSource.java index c3ee8ed9a0b..da3e3b508ac 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/DoubleTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/DoubleTestSource.java @@ -77,8 +77,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add @Override public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } @@ -126,8 +124,6 @@ private void maybeInitializePrevForStep() { @Override public synchronized void remove(RowSet rowSet) { - setGroupToRange(null); - maybeInitializePrevForStep(); rowSet.forAllRowKeys(data::remove); } @@ -135,7 +131,6 @@ public synchronized void remove(RowSet rowSet) { @Override public synchronized void shift(long startKeyInclusive, long endKeyInclusive, long shiftDelta) { maybeInitializePrevForStep(); - setGroupToRange(null); // Note: moving to the right, we need to start with rightmost data first. final long dir = shiftDelta > 0 ? -1 : 1; diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/FloatTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/FloatTestSource.java index b78a758e087..6e53da52681 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/FloatTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/FloatTestSource.java @@ -77,8 +77,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add @Override public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } @@ -126,8 +124,6 @@ private void maybeInitializePrevForStep() { @Override public synchronized void remove(RowSet rowSet) { - setGroupToRange(null); - maybeInitializePrevForStep(); rowSet.forAllRowKeys(data::remove); } @@ -135,7 +131,6 @@ public synchronized void remove(RowSet rowSet) { @Override public synchronized void shift(long startKeyInclusive, long endKeyInclusive, long shiftDelta) { maybeInitializePrevForStep(); - setGroupToRange(null); // Note: moving to the right, we need to start with rightmost data first. final long dir = shiftDelta > 0 ? -1 : 1; diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableByteTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableByteTestSource.java index cef8c11e0ee..744a652b3ae 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableByteTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableByteTestSource.java @@ -64,8 +64,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableCharTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableCharTestSource.java index 81d2a129be6..5e8f5d4cf59 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableCharTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableCharTestSource.java @@ -60,8 +60,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableDoubleTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableDoubleTestSource.java index 792c5873b34..2e7798cb65e 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableDoubleTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableDoubleTestSource.java @@ -64,8 +64,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableFloatTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableFloatTestSource.java index cff16b6e58d..57c3071707b 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableFloatTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableFloatTestSource.java @@ -64,8 +64,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableIntTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableIntTestSource.java index bebb1c3c09f..915ff419874 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableIntTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableIntTestSource.java @@ -64,8 +64,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableLongTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableLongTestSource.java index ad5831be83c..bb6358185cc 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableLongTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableLongTestSource.java @@ -64,8 +64,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableObjectTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableObjectTestSource.java index cc1d442181d..0cffff58f2a 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableObjectTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableObjectTestSource.java @@ -65,8 +65,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("rowSet=" + rowSet + ", data size=" + vs.size()); } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableShortTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableShortTestSource.java index c6d53ce08d5..eaa908e191b 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableShortTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ImmutableShortTestSource.java @@ -64,8 +64,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/IntTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/IntTestSource.java index 838dafbe162..8720f109e7a 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/IntTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/IntTestSource.java @@ -77,8 +77,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add @Override public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } @@ -126,8 +124,6 @@ private void maybeInitializePrevForStep() { @Override public synchronized void remove(RowSet rowSet) { - setGroupToRange(null); - maybeInitializePrevForStep(); rowSet.forAllRowKeys(data::remove); } @@ -135,7 +131,6 @@ public synchronized void remove(RowSet rowSet) { @Override public synchronized void shift(long startKeyInclusive, long endKeyInclusive, long shiftDelta) { maybeInitializePrevForStep(); - setGroupToRange(null); // Note: moving to the right, we need to start with rightmost data first. final long dir = shiftDelta > 0 ? -1 : 1; diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/LongTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/LongTestSource.java index 9469bec4740..a5db115bcbf 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/LongTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/LongTestSource.java @@ -77,8 +77,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add @Override public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } @@ -126,8 +124,6 @@ private void maybeInitializePrevForStep() { @Override public synchronized void remove(RowSet rowSet) { - setGroupToRange(null); - maybeInitializePrevForStep(); rowSet.forAllRowKeys(data::remove); } @@ -135,7 +131,6 @@ public synchronized void remove(RowSet rowSet) { @Override public synchronized void shift(long startKeyInclusive, long endKeyInclusive, long shiftDelta) { maybeInitializePrevForStep(); - setGroupToRange(null); // Note: moving to the right, we need to start with rightmost data first. final long dir = shiftDelta > 0 ? -1 : 1; diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ObjectTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ObjectTestSource.java index da770fc7dc9..0d6e1a46651 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ObjectTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ObjectTestSource.java @@ -76,8 +76,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("rowSet=" + rowSet + ", data size=" + vs.size()); } @@ -110,8 +108,6 @@ private void maybeInitializePrevForStep() { @Override public synchronized void remove(RowSet rowSet) { - setGroupToRange(null); - maybeInitializePrevForStep(); rowSet.forAllRowKeys(data::remove); } @@ -119,7 +115,6 @@ public synchronized void remove(RowSet rowSet) { @Override public synchronized void shift(long startKeyInclusive, long endKeyInclusive, long shiftDelta) { maybeInitializePrevForStep(); - setGroupToRange(null); // Note: moving to the right, we need to start with rightmost data first. final long dir = shiftDelta > 0 ? -1 : 1; diff --git a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ShortTestSource.java b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ShortTestSource.java index a8412007dce..379e7bd75bd 100644 --- a/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ShortTestSource.java +++ b/engine/test-utils/src/main/java/io/deephaven/engine/testutil/sources/ShortTestSource.java @@ -77,8 +77,6 @@ public synchronized void checkIndex(RowSet rowSet) { // region chunk add @Override public synchronized void add(final RowSet rowSet, Chunk vs) { - setGroupToRange(null); - if (rowSet.size() != vs.size()) { throw new IllegalArgumentException("Index=" + rowSet + ", data size=" + vs.size()); } @@ -126,8 +124,6 @@ private void maybeInitializePrevForStep() { @Override public synchronized void remove(RowSet rowSet) { - setGroupToRange(null); - maybeInitializePrevForStep(); rowSet.forAllRowKeys(data::remove); } @@ -135,7 +131,6 @@ public synchronized void remove(RowSet rowSet) { @Override public synchronized void shift(long startKeyInclusive, long endKeyInclusive, long shiftDelta) { maybeInitializePrevForStep(); - setGroupToRange(null); // Note: moving to the right, we need to start with rightmost data first. final long dir = shiftDelta > 0 ? -1 : 1; diff --git a/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/AbstractTupleSource.java b/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/AbstractTupleSource.java index 93a616f0b4c..8e5da475e50 100644 --- a/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/AbstractTupleSource.java +++ b/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/AbstractTupleSource.java @@ -15,23 +15,15 @@ import io.deephaven.engine.table.impl.DefaultChunkSource; import org.jetbrains.annotations.NotNull; -import java.util.List; import java.util.stream.Stream; public abstract class AbstractTupleSource implements TupleSource, DefaultChunkSource.WithPrev { private final ColumnSource[] columnSources; - private final List> listColumnSources; public AbstractTupleSource(ColumnSource... columnSources) { this.columnSources = columnSources; - this.listColumnSources = List.of(columnSources); - } - - @Override - public final List> getColumnSources() { - return listColumnSources; } @Override diff --git a/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/EmptyTupleSource.java b/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/EmptyTupleSource.java index ffd011c1577..df3afce0428 100644 --- a/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/EmptyTupleSource.java +++ b/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/EmptyTupleSource.java @@ -7,16 +7,12 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.TupleSource; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.DefaultChunkSource; import io.deephaven.tuple.EmptyTuple; import org.jetbrains.annotations.NotNull; -import java.util.Collections; -import java.util.List; - /** *

* {@link TupleSource} that produces only the {@link EmptyTuple}. @@ -25,11 +21,6 @@ enum EmptyTupleSource implements TupleSource, DefaultChunkSource.Wit INSTANCE; - @Override - public List> getColumnSources() { - return Collections.emptyList(); - } - @Override public EmptyTuple createTuple(final long rowKey) { return EmptyTuple.INSTANCE; diff --git a/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/MultiColumnTupleSource.java b/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/MultiColumnTupleSource.java index e5edebf16c9..3a86081cd6d 100644 --- a/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/MultiColumnTupleSource.java +++ b/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/MultiColumnTupleSource.java @@ -14,7 +14,6 @@ import org.jetbrains.annotations.NotNull; import java.util.Arrays; -import java.util.List; /** *

@@ -24,8 +23,6 @@ final class MultiColumnTupleSource implements TupleSource, DefaultCh private final ColumnSource[] columnSources; - private final List> columnSourceList; - /** * Construct a new tuple source backed by the supplied column sources. The column sources array should not be * changed after this call. @@ -34,12 +31,6 @@ final class MultiColumnTupleSource implements TupleSource, DefaultCh */ MultiColumnTupleSource(@NotNull final ColumnSource... columnSources) { this.columnSources = columnSources; - columnSourceList = List.of(columnSources); - } - - @Override - public List> getColumnSources() { - return columnSourceList; } @Override diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java index 6178a0bae7e..28c073433a5 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetSchemaReader.java @@ -39,7 +39,7 @@ public interface ColumnDefinitionConsumer { } public static final class ParquetMessageDefinition { - /** Yes you guessed right. This is the column name. */ + /** The column name. */ public String name; /** The parquet type. */ public Class baseType; @@ -52,13 +52,11 @@ public static final class ParquetMessageDefinition { /** * Parquet 1.0 did not support logical types; if we encounter a type like this is true. For example, in parquet * 1.0 binary columns with no annotation are used to represent strings. They are also used to represent other - * things that are not strings. Good luck, may the force be with you. + * things that are not strings. */ public boolean noLogicalType; - /** Your guess is good here */ + /** Whether this column is an array. */ public boolean isArray; - /** Your guess is good here. */ - public boolean isGrouping; /** * When codec metadata is present (which will be returned as modified read instructions below for actual codec * name and args), we expect codec type and component type to be present. When they are present, codecType and @@ -69,16 +67,16 @@ public static final class ParquetMessageDefinition { public String codecComponentType; /** - * We reuse the guts of this poor object between calls to avoid allocating. Like prometheus nailed to a - * mountain, this poor object has to suffer his guts being eaten forever. Or not forever but at least for one - * stack frame activation of readParquetSchema and as many columns that function finds in the file. + * We reuse this object between calls to avoid allocating. */ void reset() { name = null; baseType = null; dhSpecialType = null; - noLogicalType = isArray = isGrouping = false; - codecType = codecComponentType = null; + noLogicalType = false; + isArray = false; + codecType = null; + codecComponentType = null; } } @@ -140,8 +138,6 @@ public static ParquetInstructions readParquetSchema( final MutableObject errorString = new MutableObject<>(); final MutableObject currentColumn = new MutableObject<>(); final Optional tableInfo = parseMetadata(keyValueMetadata); - final Set groupingColumnNames = - tableInfo.map(TableInfo::groupingColumnNames).orElse(Collections.emptySet()); final Map nonDefaultTypeColumns = tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); final LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> visitor = @@ -199,7 +195,6 @@ public static ParquetInstructions readParquetSchema( colDef.name = colName; colDef.dhSpecialType = columnTypeInfo.flatMap(ColumnTypeInfo::specialType).orElse(null); - colDef.isGrouping = groupingColumnNames.contains(colName); final Optional codecInfo = columnTypeInfo.flatMap(ColumnTypeInfo::codec); String codecName = codecInfo.map(CodecInfo::codecName).orElse(null); String codecArgs = codecInfo.flatMap(CodecInfo::codecArg).orElse(null); diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java index 66f93a7073b..e6b2291221d 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTableWriter.java @@ -3,16 +3,11 @@ // package io.deephaven.parquet.table; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.api.ColumnName; -import io.deephaven.api.RawString; -import io.deephaven.api.Selectable; -import io.deephaven.api.agg.Aggregation; import io.deephaven.engine.liveness.LivenessScopeStack; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.select.FormulaColumn; import io.deephaven.engine.table.impl.select.NullSelectColumn; import io.deephaven.engine.table.impl.select.SelectColumn; @@ -20,16 +15,18 @@ import io.deephaven.parquet.base.ColumnWriter; import io.deephaven.parquet.base.ParquetFileWriter; import io.deephaven.parquet.base.RowGroupWriter; -import io.deephaven.util.channel.SeekableChannelsProviderLoader; import io.deephaven.parquet.table.metadata.CodecInfo; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; -import io.deephaven.parquet.table.metadata.GroupingColumnInfo; +import io.deephaven.parquet.table.metadata.DataIndexInfo; import io.deephaven.parquet.table.metadata.TableInfo; -import io.deephaven.parquet.table.transfer.*; +import io.deephaven.parquet.table.transfer.ArrayAndVectorTransfer; +import io.deephaven.parquet.table.transfer.StringDictionary; +import io.deephaven.parquet.table.transfer.TransferObject; import io.deephaven.stringset.StringSet; import io.deephaven.util.QueryConstants; import io.deephaven.util.SafeCloseable; import io.deephaven.util.annotations.VisibleForTesting; +import io.deephaven.util.channel.SeekableChannelsProviderLoader; import io.deephaven.vector.Vector; import org.apache.commons.lang3.tuple.Pair; import org.apache.parquet.bytes.HeapByteBufferAllocator; @@ -37,10 +34,11 @@ import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Types; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.File; import java.io.IOException; -import java.nio.*; +import java.nio.IntBuffer; import java.nio.file.Path; import java.nio.file.Paths; import java.util.*; @@ -51,34 +49,48 @@ * API for writing DH tables in parquet format */ public class ParquetTableWriter { + public static final String METADATA_KEY = "deephaven"; - public static final String BEGIN_POS = "dh_begin_pos"; - public static final String END_POS = "dh_end_pos"; - public static final String GROUPING_KEY = "dh_key"; + + public static final String GROUPING_KEY_COLUMN_NAME = "dh_key"; + public static final String GROUPING_BEGIN_POS_COLUMN_NAME = "dh_begin_pos"; + public static final String GROUPING_END_POS_COLUMN_NAME = "dh_end_pos"; + + public static final String INDEX_ROW_SET_COLUMN_NAME = "dh_row_set"; + public static final String PARQUET_FILE_EXTENSION = ".parquet"; /** - * Helper struct used to pass information about where to write the grouping files for each grouping column + * Helper struct used to pass information about where to write the index files */ - static class GroupingColumnWritingInfo { + static class IndexWritingInfo { + /** + * Names of the indexing key columns + */ + final String[] indexColumnNames; /** - * Parquet name of this grouping column + * Parquet names of the indexing key columns */ - final String parquetColumnName; + final String[] parquetColumnNames; /** - * File path to be added in the grouping metadata of main parquet file + * File path to be added in the index metadata of main parquet file */ final File metadataFilePath; /** - * Destination path for writing the grouping file. The two filenames can differ because we write grouping files - * to shadow file paths first and then place them at the final path once the write is complete. But the metadata - * should always hold the accurate path. + * Destination path for writing the index file. The two filenames can differ because we write index files to + * shadow file paths first and then place them at the final path once the write is complete. The metadata should + * always hold the accurate path. */ final File destFile; - GroupingColumnWritingInfo(final String parquetColumnName, final File metadataFilePath, final File destFile) { - this.parquetColumnName = parquetColumnName; + IndexWritingInfo( + final String[] indexColumnNames, + final String[] parquetColumnNames, + final File metadataFilePath, + final File destFile) { + this.indexColumnNames = indexColumnNames; + this.parquetColumnNames = parquetColumnNames; this.metadataFilePath = metadataFilePath; this.destFile = destFile; } @@ -92,6 +104,8 @@ static class GroupingColumnWritingInfo { * @param writeInstructions Write instructions for customizations while writing * @param destPathName The destination path * @param incomingMeta A map of metadata values to be stores in the file footer + * @param indexInfoList Arrays containing the column names for indexes to persist as sidecar tables. Indexes that + * are specified but missing will be computed on demand. * @throws SchemaMappingException Error creating a parquet table schema for the given table (likely due to * unsupported types) * @throws IOException For file writing related errors @@ -102,25 +116,49 @@ static void write( @NotNull final ParquetInstructions writeInstructions, @NotNull final String destPathName, @NotNull final Map incomingMeta, - final Map groupingColumnsWritingInfoMap) + @Nullable final List indexInfoList) throws SchemaMappingException, IOException { + if (t.isRefreshing()) { + /* + * We mustn't write inconsistent tables or data indexes. This check is "basic". Snapshotting logic here + * would probably be inappropriate, as we might be writing very large tables. Hopefully users aren't naively + * writing Parquet tables from within listeners or transforms without ensuring proper dependency + * satisfaction for the table and any indexes it has. + */ + t.getUpdateGraph().checkInitiateSerialTableOperation(); + } + final TableInfo.Builder tableInfoBuilder = TableInfo.builder(); List cleanupFiles = null; try { - if (groupingColumnsWritingInfoMap != null) { - cleanupFiles = new ArrayList<>(groupingColumnsWritingInfoMap.size()); + if (indexInfoList != null) { + cleanupFiles = new ArrayList<>(indexInfoList.size()); final Path destDirPath = Paths.get(destPathName).getParent(); - for (Map.Entry entry : groupingColumnsWritingInfoMap.entrySet()) { - final String groupingColumnName = entry.getKey(); - final Table auxiliaryTable = groupingAsTable(t, groupingColumnName); - final String parquetColumnName = entry.getValue().parquetColumnName; - final File metadataFilePath = entry.getValue().metadataFilePath; - final File groupingDestFile = entry.getValue().destFile; - cleanupFiles.add(groupingDestFile); - tableInfoBuilder.addGroupingColumns(GroupingColumnInfo.of(parquetColumnName, - destDirPath.relativize(metadataFilePath.toPath()).toString())); - write(auxiliaryTable, auxiliaryTable.getDefinition(), writeInstructions, - groupingDestFile.getAbsolutePath(), Collections.emptyMap(), TableInfo.builder()); + for (final ParquetTableWriter.IndexWritingInfo info : indexInfoList) { + try (final SafeCloseable ignored = t.isRefreshing() ? LivenessScopeStack.open() : null) { + // This will retrieve an existing index if one exists, or create a new one if not + final BasicDataIndex dataIndex = Optional + .ofNullable(DataIndexer.getDataIndex(t, info.indexColumnNames)) + .or(() -> Optional.of(DataIndexer.getOrCreateDataIndex(t, info.indexColumnNames))) + .get() + .transform(DataIndexTransformer.builder().invertRowSet(t.getRowSet()).build()); + final Table indexTable = dataIndex.table(); + + cleanupFiles.add(info.destFile); + tableInfoBuilder.addDataIndexes(DataIndexInfo.of( + destDirPath.relativize(info.metadataFilePath.toPath()).toString(), + info.parquetColumnNames)); + final ParquetInstructions writeInstructionsToUse; + if (INDEX_ROW_SET_COLUMN_NAME.equals(dataIndex.rowSetColumnName())) { + writeInstructionsToUse = writeInstructions; + } else { + writeInstructionsToUse = new ParquetInstructions.Builder(writeInstructions) + .addColumnNameMapping(INDEX_ROW_SET_COLUMN_NAME, dataIndex.rowSetColumnName()) + .build(); + } + write(indexTable, indexTable.getDefinition(), writeInstructionsToUse, + info.destFile.getAbsolutePath(), Collections.emptyMap(), TableInfo.builder()); + } } } write(t, definition, writeInstructions, destPathName, incomingMeta, tableInfoBuilder); @@ -455,22 +493,4 @@ private static void encodePlain( } while (transferObject.hasMoreDataToBuffer()); } } - - private static Table groupingAsTable(Table tableToSave, String columnName) { - final QueryTable coalesced = (QueryTable) tableToSave.coalesce(); - final Table tableToGroup = (coalesced.isRefreshing() ? (QueryTable) coalesced.silent() : coalesced) - .withAttributes(Map.of(Table.BLINK_TABLE_ATTRIBUTE, true)); // We want persistent first/last-by - final Table grouped = tableToGroup - .view(List.of(Selectable.of(ColumnName.of(GROUPING_KEY), ColumnName.of(columnName)), - Selectable.of(ColumnName.of(BEGIN_POS), RawString.of("ii")), // Range start, inclusive - Selectable.of(ColumnName.of(END_POS), RawString.of("ii+1")))) // Range end, exclusive - .aggBy(List.of(Aggregation.AggFirst(BEGIN_POS), Aggregation.AggLast(END_POS)), - List.of(ColumnName.of(GROUPING_KEY))); - final Table invalid = grouped.where(BEGIN_POS + " != 0 && " + BEGIN_POS + " != " + END_POS + "_[ii-1]"); - if (!invalid.isEmpty()) { - throw new UncheckedDeephavenException( - "Range grouping is not possible for column because some indices are not contiguous"); - } - return grouped; - } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java index dbbdfd88ad7..be4bcad53c2 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/ParquetTools.java @@ -4,26 +4,28 @@ package io.deephaven.parquet.table; import io.deephaven.UncheckedDeephavenException; +import io.deephaven.api.util.NameValidator; import io.deephaven.base.ClassUtil; import io.deephaven.base.FileUtils; import io.deephaven.base.Pair; import io.deephaven.base.verify.Require; import io.deephaven.engine.context.ExecutionContext; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; -import io.deephaven.engine.updategraph.UpdateSourceRegistrar; -import io.deephaven.util.channel.SeekableChannelsProvider; -import io.deephaven.util.channel.SeekableChannelsProviderLoader; -import io.deephaven.util.channel.SeekableChannelsProviderPlugin; -import io.deephaven.vector.*; -import io.deephaven.stringset.StringSet; +import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.PartitionAwareSourceTable; import io.deephaven.engine.table.impl.SimpleSourceTable; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableLocationProvider; -import io.deephaven.engine.table.impl.locations.impl.*; +import io.deephaven.engine.table.impl.locations.impl.KnownLocationKeyFinder; +import io.deephaven.engine.table.impl.locations.impl.PollingTableLocationProvider; +import io.deephaven.engine.table.impl.locations.impl.StandaloneTableKey; +import io.deephaven.engine.table.impl.locations.impl.TableLocationKeyFinder; +import io.deephaven.engine.table.impl.locations.util.TableDataRefreshService; +import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; +import io.deephaven.engine.updategraph.UpdateSourceRegistrar; +import io.deephaven.internal.log.LoggerFactory; +import io.deephaven.io.logger.Logger; +import io.deephaven.parquet.base.ParquetFileReader; import io.deephaven.parquet.table.layout.ParquetFlatPartitionedLayout; import io.deephaven.parquet.table.layout.ParquetKeyValuePartitionedLayout; import io.deephaven.parquet.table.layout.ParquetMetadataFileLayout; @@ -31,19 +33,20 @@ import io.deephaven.parquet.table.location.ParquetTableLocationFactory; import io.deephaven.parquet.table.location.ParquetTableLocationKey; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; -import io.deephaven.api.util.NameValidator; +import io.deephaven.stringset.StringSet; import io.deephaven.util.SimpleTypeMap; -import io.deephaven.engine.table.impl.sources.regioned.RegionedTableComponentFactoryImpl; -import io.deephaven.internal.log.LoggerFactory; -import io.deephaven.io.logger.Logger; -import io.deephaven.parquet.base.ParquetFileReader; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import io.deephaven.util.channel.CachedChannelProvider; import io.deephaven.util.annotations.VisibleForTesting; +import io.deephaven.util.channel.CachedChannelProvider; +import io.deephaven.util.channel.SeekableChannelsProvider; +import io.deephaven.util.channel.SeekableChannelsProviderLoader; +import io.deephaven.util.channel.SeekableChannelsProviderPlugin; +import io.deephaven.vector.*; import org.apache.commons.lang3.mutable.MutableObject; +import org.apache.parquet.format.converter.ParquetMetadataConverter; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.File; import java.io.IOException; @@ -53,10 +56,11 @@ import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributes; import java.util.*; +import java.util.stream.Collectors; import static io.deephaven.base.FileUtils.convertToURI; import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; -import static io.deephaven.parquet.table.ParquetTableWriter.PARQUET_FILE_EXTENSION; +import static io.deephaven.parquet.table.ParquetTableWriter.*; import static io.deephaven.util.type.TypeUtils.getUnboxedTypeIfBoxed; /** @@ -200,7 +204,7 @@ private static URI convertParquetSourceToURI(@NotNull final String source) { } /** - * Write a table to a file. + * Write a table to a file. Data indexes to write are determined by those present on {@code sourceTable}. * * @param sourceTable source table * @param destPath destination file path; the file name should end in ".parquet" extension If the path includes @@ -214,7 +218,7 @@ public static void writeTable( } /** - * Write a table to a file. + * Write a table to a file. Data indexes to write are determined by those present on {@code sourceTable}. * * @param sourceTable source table * @param destFile destination file; the file name should end in ".parquet" extension If the path includes @@ -227,7 +231,7 @@ public static void writeTable( } /** - * Write a table to a file. + * Write a table to a file. Data indexes to write are determined by those present on {@code sourceTable}. * * @param sourceTable source table * @param destFile destination file; its path must end in ".parquet". Any non existing directories in the path are @@ -243,7 +247,7 @@ public static void writeTable( } /** - * Write a table to a file. + * Write a table to a file. Data indexes to write are determined by those present on {@code sourceTable}. * * @param sourceTable source table * @param destFile destination file; its path must end in ".parquet". Any non existing directories in the path are @@ -259,7 +263,7 @@ public static void writeTable( } /** - * Write a table to a file. + * Write a table to a file. Data indexes to write are determined by those present on {@code sourceTable}. * * @param sourceTable source table * @param destPath destination path; it must end in ".parquet". Any non existing directories in the path are created @@ -268,7 +272,8 @@ public static void writeTable( * @param definition table definition to use (instead of the one implied by the table itself) * @param writeInstructions instructions for customizations while writing */ - public static void writeTable(@NotNull final Table sourceTable, + public static void writeTable( + @NotNull final Table sourceTable, @NotNull final String destPath, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions) { @@ -276,7 +281,7 @@ public static void writeTable(@NotNull final Table sourceTable, } /** - * Write a table to a file. + * Write a table to a file. Data indexes to write are determined by those present on {@code sourceTable}. * * @param sourceTable source table * @param destFile destination file; its path must end in ".parquet". Any non-existing directories in the path are @@ -285,7 +290,8 @@ public static void writeTable(@NotNull final Table sourceTable, * @param definition table definition to use (instead of the one implied by the table itself) * @param writeInstructions instructions for customizations while writing */ - public static void writeTable(@NotNull final Table sourceTable, + public static void writeTable( + @NotNull final Table sourceTable, @NotNull final File destFile, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions) { @@ -312,14 +318,15 @@ private static String minusParquetSuffix(@NotNull final String s) { * Generates the index file path relative to the table destination file path. * * @param tableDest Destination path for the main table containing these indexing columns - * @param columnName Name of the indexing column + * @param columnNames Array of names of the indexing columns * * @return The relative index file path. For example, for table with destination {@code "table.parquet"} and * indexing column {@code "IndexingColName"}, the method will return * {@code ".dh_metadata/indexes/IndexingColName/index_IndexingColName_table.parquet"} */ - public static String getRelativeIndexFilePath(@NotNull final File tableDest, @NotNull final String columnName) { - return String.format(".dh_metadata/indexes/%s/index_%s_%s", columnName, columnName, tableDest.getName()); + public static String getRelativeIndexFilePath(@NotNull final File tableDest, @NotNull final String[] columnNames) { + final String columns = String.join(",", columnNames); + return String.format(".dh_metadata/indexes/%s/index_%s_%s", columns, columns, tableDest.getName()); } /** @@ -442,52 +449,60 @@ private static File prepareDestinationFileLocation(@NotNull File destination) { } /** - * Helper function for building grouping column info for writing and deleting any backup grouping column files + * Helper function for building index column info for writing and deleting any backup index column files * - * @param groupingColumnNames Names of grouping columns - * @param parquetColumnNames Names of grouping columns for the parquet file - * @param destFile The destination path for the main table containing these grouping columns + * @param indexColumnNameArr Names of index columns, stored as String[] for each index + * @param parquetColumnNameArr Names of index columns for the parquet file, stored as String[] for each index + * @param destFile The destination path for the main table containing these index columns */ - private static Map groupingColumnInfoBuilderHelper( - @NotNull final String[] groupingColumnNames, - @NotNull final String[] parquetColumnNames, + private static List indexInfoBuilderHelper( + @NotNull final String[][] indexColumnNameArr, + @NotNull final String[][] parquetColumnNameArr, @NotNull final File destFile) { - Require.eq(groupingColumnNames.length, "groupingColumnNames.length", parquetColumnNames.length, - "parquetColumnNames.length"); - final Map gcwim = new HashMap<>(); - for (int gci = 0; gci < groupingColumnNames.length; gci++) { - final String groupingColumnName = groupingColumnNames[gci]; - final String parquetColumnName = parquetColumnNames[gci]; - final String indexFileRelativePath = getRelativeIndexFilePath(destFile, parquetColumnName); + Require.eq(indexColumnNameArr.length, "indexColumnNameArr.length", parquetColumnNameArr.length, + "parquetColumnNameArr.length"); + final List indexInfoList = new ArrayList<>(); + for (int gci = 0; gci < indexColumnNameArr.length; gci++) { + final String[] indexColumnNames = indexColumnNameArr[gci]; + final String[] parquetColumnNames = parquetColumnNameArr[gci]; + final String indexFileRelativePath = getRelativeIndexFilePath(destFile, parquetColumnNames); final File indexFile = new File(destFile.getParent(), indexFileRelativePath); prepareDestinationFileLocation(indexFile); deleteBackupFile(indexFile); + final File shadowIndexFile = getShadowFile(indexFile); - gcwim.put(groupingColumnName, new ParquetTableWriter.GroupingColumnWritingInfo(parquetColumnName, - indexFile, shadowIndexFile)); + + final ParquetTableWriter.IndexWritingInfo info = new ParquetTableWriter.IndexWritingInfo( + indexColumnNames, + parquetColumnNames, + indexFile, + shadowIndexFile); + + indexInfoList.add(info); } - return gcwim; + return indexInfoList; } /** - * Writes tables to disk in parquet format to a supplied set of destinations. If you specify grouping columns, there - * must already be grouping information for those columns in the sources. This can be accomplished with - * {@code .groupBy().ungroup()} or {@code .sort()}. + * Writes tables to disk in parquet format to a supplied set of destinations. * * @param sources The tables to write * @param definition The common schema for all the tables to write * @param writeInstructions Write instructions for customizations while writing * @param destinations The destinations paths. Any non-existing directories in the paths provided are created. If * there is an error any intermediate directories previously created are removed; note this makes this method - * unsafe for concurrent use - * @param groupingColumns List of columns the tables are grouped by (the write operation will store the grouping - * info) + * unsafe for concurrent use. + * @param indexColumnArr Arrays containing the column names for indexes to persist. The write operation will store + * the index info as sidecar tables. This argument is used to narrow the set of indexes to write, or to be + * explicit about the expected set of indexes present on all sources. Indexes that are specified but missing + * will be computed on demand. */ - public static void writeParquetTables(@NotNull final Table[] sources, + public static void writeParquetTables( + @NotNull final Table[] sources, @NotNull final TableDefinition definition, @NotNull final ParquetInstructions writeInstructions, @NotNull final File[] destinations, - @NotNull final String[] groupingColumns) { + @Nullable final String[][] indexColumnArr) { Require.eq(sources.length, "sources.length", destinations.length, "destinations.length"); if (definition.numColumns() == 0) { throw new TableDataException("Cannot write a parquet table with zero columns"); @@ -503,41 +518,42 @@ public static void writeParquetTables(@NotNull final Table[] sources, // List of shadow files, to clean up in case of exceptions final List shadowFiles = new ArrayList<>(); - // List of all destination files (including grouping files), to roll back in case of exceptions + // List of all destination files (including index files), to roll back in case of exceptions final List destFiles = new ArrayList<>(); try { - final List> groupingColumnWritingInfoMaps; - if (groupingColumns.length == 0) { - // Write the tables without any grouping info - groupingColumnWritingInfoMaps = null; + final List> indexInfoLists; + if (indexColumnArr == null || indexColumnArr.length == 0) { + // Write the tables without any index info + indexInfoLists = null; for (int tableIdx = 0; tableIdx < sources.length; tableIdx++) { shadowFiles.add(shadowDestFiles[tableIdx]); final Table source = sources[tableIdx]; ParquetTableWriter.write(source, definition, writeInstructions, shadowDestFiles[tableIdx].getPath(), - Collections.emptyMap(), (Map) null); + Collections.emptyMap(), (List) null); } } else { - // Create grouping info for each table and write the table and grouping files to shadow path - groupingColumnWritingInfoMaps = new ArrayList<>(sources.length); + // Create index info for each table and write the table and index files to shadow path + indexInfoLists = new ArrayList<>(sources.length); - // Same parquet column names across all tables - final String[] parquetColumnNames = Arrays.stream(groupingColumns) - .map(writeInstructions::getParquetColumnNameFromColumnNameOrDefault) - .toArray(String[]::new); + // Shared parquet column names across all tables + final String[][] parquetColumnNameArr = Arrays.stream(indexColumnArr) + .map((String[] columns) -> Arrays.stream(columns) + .map(writeInstructions::getParquetColumnNameFromColumnNameOrDefault) + .toArray(String[]::new)) + .toArray(String[][]::new); for (int tableIdx = 0; tableIdx < sources.length; tableIdx++) { final File tableDestination = destinations[tableIdx]; - final Map groupingColumnWritingInfoMap = - groupingColumnInfoBuilderHelper(groupingColumns, parquetColumnNames, tableDestination); - groupingColumnWritingInfoMaps.add(groupingColumnWritingInfoMap); + final List indexInfoList = + indexInfoBuilderHelper(indexColumnArr, parquetColumnNameArr, tableDestination); + indexInfoLists.add(indexInfoList); shadowFiles.add(shadowDestFiles[tableIdx]); - groupingColumnWritingInfoMap.values().forEach(gcwi -> shadowFiles.add(gcwi.destFile)); + indexInfoList.forEach(item -> shadowFiles.add(item.destFile)); final Table sourceTable = sources[tableIdx]; ParquetTableWriter.write(sourceTable, definition, writeInstructions, - shadowDestFiles[tableIdx].getPath(), - Collections.emptyMap(), groupingColumnWritingInfoMap); + shadowDestFiles[tableIdx].getPath(), Collections.emptyMap(), indexInfoList); } } @@ -545,12 +561,11 @@ public static void writeParquetTables(@NotNull final Table[] sources, for (int tableIdx = 0; tableIdx < sources.length; tableIdx++) { destFiles.add(destinations[tableIdx]); installShadowFile(destinations[tableIdx], shadowDestFiles[tableIdx]); - if (groupingColumnWritingInfoMaps != null) { - final Map gcwim = - groupingColumnWritingInfoMaps.get(tableIdx); - for (final ParquetTableWriter.GroupingColumnWritingInfo gfwi : gcwim.values()) { - final File indexDestFile = gfwi.metadataFilePath; - final File shadowIndexFile = gfwi.destFile; + if (indexInfoLists != null) { + final List indexInfoList = indexInfoLists.get(tableIdx); + for (final ParquetTableWriter.IndexWritingInfo info : indexInfoList) { + final File indexDestFile = info.metadataFilePath; + final File shadowIndexFile = info.destFile; destFiles.add(indexDestFile); installShadowFile(indexDestFile, shadowIndexFile); } @@ -574,37 +589,85 @@ public static void writeParquetTables(@NotNull final Table[] sources, } throw new UncheckedDeephavenException("Error writing parquet tables", e); } - destFiles.stream().forEach(ParquetTools::deleteBackupFileNoExcept); + destFiles.forEach(ParquetTools::deleteBackupFileNoExcept); } /** - * Write out tables to disk. + * Examine the source tables to retrieve the list of indexes as String[] arrays. + * + * @param sources The tables from which to retrieve the indexes + * @return An array containing the indexes as String[] arrays + * @implNote This only examines the first source table. The writing code will compute missing indexes for the other + * source tables. + */ + private static String[][] indexedColumnNames(@NotNull Table @NotNull [] sources) { + if (sources.length == 0) { + return new String[0][]; + } + + // Use the first table as the source of indexed columns + final Table firstTable = sources[0]; + final DataIndexer dataIndexer = DataIndexer.existingOf(firstTable.getRowSet()); + if (dataIndexer == null) { + return new String[0][]; + } + final List dataIndexes = dataIndexer.dataIndexes(true); + if (dataIndexes.isEmpty()) { + return new String[0][]; + } + final Map> nameToColumn = firstTable.getColumnSourceMap(); + // We disregard collisions, here; any mapped name is an adequate choice. + final Map, String> columnToName = nameToColumn.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getValue, Map.Entry::getKey)); + + final List indexesToWrite = new ArrayList<>(); + + // Build the list of indexes to write + dataIndexes.forEach(di -> { + final Map, String> keyColumnNamesByIndexedColumn = di.keyColumnNamesByIndexedColumn(); + + // Re-map the index columns to their names in this table + final String[] keyColumnNames = keyColumnNamesByIndexedColumn.keySet().stream() + .map(columnToName::get) + .filter(Objects::nonNull) + .toArray(String[]::new); + + // Make sure all the columns actually exist in the table + if (keyColumnNames.length == keyColumnNamesByIndexedColumn.size()) { + indexesToWrite.add(keyColumnNames); + } + }); + return indexesToWrite.toArray(String[][]::new); + } + + /** + * Write out tables to disk. Data indexes to write are determined by those already present on the first source. * * @param sources source tables * @param definition table definition * @param destinations destinations */ - public static void writeTables(@NotNull final Table[] sources, + public static void writeTables( + @NotNull final Table[] sources, @NotNull final TableDefinition definition, @NotNull final File[] destinations) { - writeParquetTables(sources, definition, ParquetInstructions.EMPTY, destinations, - definition.getGroupingColumnNamesArray()); + writeParquetTables(sources, definition, ParquetInstructions.EMPTY, destinations, indexedColumnNames(sources)); } /** - * Write out tables to disk. + * Write out tables to disk. Data indexes to write are determined by those already present on the first source. * * @param sources source tables * @param definition table definition * @param destinations destinations * @param writeInstructions instructions for customizations while writing */ - public static void writeTables(@NotNull final Table[] sources, + public static void writeTables( + @NotNull final Table[] sources, @NotNull final TableDefinition definition, @NotNull final File[] destinations, @NotNull final ParquetInstructions writeInstructions) { - writeParquetTables(sources, definition, writeInstructions, destinations, - definition.getGroupingColumnNamesArray()); + writeParquetTables(sources, definition, writeInstructions, destinations, indexedColumnNames(sources)); } /** @@ -1107,9 +1170,6 @@ private static ParquetSchemaReader.ColumnDefinitionConsumer makeSchemaReaderCons colDef = ColumnDefinition.fromGenericType(parquetColDef.name, baseType, null); } } - if (parquetColDef.isGrouping) { - colDef = colDef.withGrouping(); - } colsOut.add(colDef); }; } @@ -1220,6 +1280,8 @@ public static Pair>, ParquetInstructions> convertSchema public static final ParquetInstructions UNCOMPRESSED = ParquetInstructions.builder().setCompressionCodecName("UNCOMPRESSED").build(); + // endregion + /** * @deprecated Use LZ4_RAW instead, as explained * here diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java index dfb18f8889a..f2e379552c0 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/TypeInfos.java @@ -7,6 +7,7 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.CodecLookup; +import io.deephaven.engine.table.impl.dataindex.RowSetCodec; import io.deephaven.stringset.StringSet; import io.deephaven.util.codec.ExternalizableCodec; import io.deephaven.util.codec.SerializableCodec; @@ -105,6 +106,10 @@ static Pair getCodecAndArgs( // Impute an appropriate codec for the data type final Class dataType = columnDefinition.getDataType(); + // TODO (https://github.com/deephaven/deephaven-core/issues/5262): Eliminate reliance on RowSetCodec + if (dataType.equals(RowSet.class)) { + return new ImmutablePair<>(RowSetCodec.class.getName(), null); + } if (Externalizable.class.isAssignableFrom(dataType)) { return new ImmutablePair<>(ExternalizableCodec.class.getName(), dataType.getName()); } @@ -144,7 +149,7 @@ static TypeInfo bigDecimalTypeInfo( // noinspection unchecked final PrecisionAndScale precisionAndScale = getPrecisionAndScale( computedCache, columnName, rowSet, () -> (ColumnSource) columnSourceMap.get(columnName)); - final Set> clazzes = Collections.singleton(BigDecimal.class); + final Set> clazzes = Set.of(BigDecimal.class); return new TypeInfo() { @Override public Set> getTypes() { @@ -152,7 +157,8 @@ public Set> getTypes() { } @Override - public PrimitiveBuilder getBuilderImpl(boolean required, boolean repeating, Class dataType) { + public PrimitiveBuilder getBuilderImpl(boolean required, boolean repeating, + Class dataType) { return type(PrimitiveTypeName.BINARY, required, repeating) .as(LogicalTypeAnnotation.decimalType(precisionAndScale.scale, precisionAndScale.precision)); } @@ -183,8 +189,7 @@ private static PrimitiveBuilder type(PrimitiveTypeName type, bool private enum IntType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList(int.class, Integer.class))); + private static final Set> clazzes = Set.of(int.class, Integer.class); @Override public Set> getTypes() { @@ -200,8 +205,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum LongType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList(long.class, Long.class))); + private static final Set> clazzes = Set.of(long.class, Long.class); @Override public Set> getTypes() { @@ -217,8 +221,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum ShortType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList(short.class, Short.class))); + private static final Set> clazzes = Set.of(short.class, Short.class); @Override public Set> getTypes() { @@ -234,8 +237,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum BooleanType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList(boolean.class, Boolean.class))); + private static final Set> clazzes = Set.of(boolean.class, Boolean.class); @Override public Set> getTypes() { @@ -251,8 +253,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum FloatType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList(float.class, Float.class))); + private static final Set> clazzes = Set.of(float.class, Float.class); @Override public Set> getTypes() { @@ -268,8 +269,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum DoubleType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList(double.class, Double.class))); + private static final Set> clazzes = Set.of(double.class, Double.class); @Override public Set> getTypes() { @@ -285,8 +285,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum CharType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList(char.class, Character.class))); + private static final Set> clazzes = Set.of(char.class, Character.class); @Override public Set> getTypes() { @@ -302,8 +301,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum ByteType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections - .unmodifiableSet(new HashSet<>(Arrays.asList(byte.class, Byte.class))); + private static final Set> clazzes = Set.of(byte.class, Byte.class); @Override public Set> getTypes() { @@ -319,7 +317,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum StringType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections.singleton(String.class); + private static final Set> clazzes = Set.of(String.class); @Override public Set> getTypes() { @@ -336,7 +334,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum InstantType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections.singleton(Instant.class); + private static final Set> clazzes = Set.of(Instant.class); @Override public Set> getTypes() { @@ -354,7 +352,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum LocalDateTimeType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections.singleton(LocalDateTime.class); + private static final Set> clazzes = Set.of(LocalDateTime.class); @Override public Set> getTypes() { @@ -372,7 +370,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum LocalDateType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections.singleton(LocalDate.class); + private static final Set> clazzes = Set.of(LocalDate.class); @Override public Set> getTypes() { @@ -389,7 +387,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum LocalTimeType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections.singleton(LocalTime.class); + private static final Set> clazzes = Set.of(LocalTime.class); @Override public Set> getTypes() { @@ -413,7 +411,7 @@ public PrimitiveBuilder getBuilderImpl(boolean required, boolean private enum BigIntegerType implements TypeInfo { INSTANCE; - private static final Set> clazzes = Collections.singleton(BigInteger.class); + private static final Set> clazzes = Set.of(BigInteger.class); @Override public Set> getTypes() { diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java index 801f8e214b0..2f246d1dabb 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetColumnLocation.java @@ -5,35 +5,22 @@ import io.deephaven.base.verify.Assert; import io.deephaven.base.verify.Require; -import io.deephaven.chunk.*; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.attributes.Values; import io.deephaven.configuration.Configuration; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSequenceFactory; -import io.deephaven.engine.rowset.chunkattributes.UnorderedRowKeys; -import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.Context; import io.deephaven.engine.table.impl.CodecLookup; import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; -import io.deephaven.engine.table.impl.chunkboxer.ChunkBoxer; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.impl.AbstractColumnLocation; import io.deephaven.engine.table.impl.sources.regioned.*; -import io.deephaven.internal.log.LoggerFactory; -import io.deephaven.io.logger.Logger; import io.deephaven.parquet.base.ColumnChunkReader; -import io.deephaven.parquet.base.ParquetFileReader; -import io.deephaven.parquet.base.RowGroupReader; -import io.deephaven.util.channel.SeekableChannelContext; -import io.deephaven.util.channel.SeekableChannelsProvider; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import io.deephaven.parquet.table.*; +import io.deephaven.parquet.table.BigDecimalParquetBytesCodec; +import io.deephaven.parquet.table.BigIntegerParquetBytesCodec; +import io.deephaven.parquet.table.ParquetInstructions; import io.deephaven.parquet.table.metadata.CodecInfo; import io.deephaven.parquet.table.metadata.ColumnTypeInfo; -import io.deephaven.parquet.table.metadata.GroupingColumnInfo; -import io.deephaven.parquet.table.metadata.TableInfo; import io.deephaven.parquet.table.pagestore.ColumnChunkPageStore; import io.deephaven.parquet.table.pagestore.PageCache; import io.deephaven.parquet.table.pagestore.topage.*; @@ -47,43 +34,32 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.io.File; import java.io.IOException; -import java.io.UncheckedIOException; import java.math.BigDecimal; import java.math.BigInteger; -import java.net.URI; -import java.util.*; +import java.util.Arrays; +import java.util.Optional; import java.util.function.Function; import java.util.function.LongFunction; import java.util.function.Supplier; import java.util.stream.IntStream; import java.util.stream.Stream; -import static io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource.ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK; -import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; -import static io.deephaven.parquet.table.ParquetTableWriter.*; - final class ParquetColumnLocation extends AbstractColumnLocation { private static final String IMPLEMENTATION_NAME = ParquetColumnLocation.class.getSimpleName(); - private static final int CHUNK_SIZE = Configuration.getInstance() - .getIntegerForClassWithDefault(ParquetColumnLocation.class, "chunkSize", 4096); private static final int INITIAL_PAGE_CACHE_SIZE = Configuration.getInstance() .getIntegerForClassWithDefault(ParquetColumnLocation.class, "initialPageCacheSize", 128); private static final int MAX_PAGE_CACHE_SIZE = Configuration.getInstance() .getIntegerForClassWithDefault(ParquetColumnLocation.class, "maxPageCacheSize", 8192); - private static final Logger log = LoggerFactory.getLogger(ParquetColumnLocation.class); - private final String parquetColumnName; /** * Factory object needed for deferred initialization of the remaining fields. Reference serves as a barrier to * ensure visibility of the derived fields. */ private volatile ColumnChunkReader[] columnChunkReaders; - private final boolean hasGroupingTable; // We should consider moving this to column level if needed. Column-location level likely allows more parallelism. private volatile PageCache pageCache; @@ -98,17 +74,15 @@ final class ParquetColumnLocation extends AbstractColumnLoc * @param tableLocation The table location enclosing this column location * @param parquetColumnName The Parquet file column name * @param columnChunkReaders The {@link ColumnChunkReader column chunk readers} for this location - * @param hasGroupingTable Whether this column has an associated grouping table file */ - ParquetColumnLocation(@NotNull final ParquetTableLocation tableLocation, + ParquetColumnLocation( + @NotNull final ParquetTableLocation tableLocation, @NotNull final String columnName, @NotNull final String parquetColumnName, - @Nullable final ColumnChunkReader[] columnChunkReaders, - final boolean hasGroupingTable) { + @Nullable final ColumnChunkReader[] columnChunkReaders) { super(tableLocation, columnName); this.parquetColumnName = parquetColumnName; this.columnChunkReaders = columnChunkReaders; - this.hasGroupingTable = hasGroupingTable; } private PageCache ensurePageCache() { @@ -146,113 +120,10 @@ private ParquetTableLocation tl() { return (ParquetTableLocation) getTableLocation(); } - private static final ColumnDefinition FIRST_KEY_COL_DEF = - ColumnDefinition.ofLong("__firstKey__"); - private static final ColumnDefinition LAST_KEY_COL_DEF = - ColumnDefinition.ofLong("__lastKey__"); - - /** - * Helper method for logging a warning on failure in reading an index file - */ - private void logWarnFailedToRead(final String indexFilePath) { - log.warn().append("Failed to read expected index file ").append(indexFilePath) - .append(" for table location ").append(tl()).append(", column ") - .append(getName()) - .endl(); - } - @Override @Nullable public METADATA_TYPE getMetadata(@NotNull final ColumnDefinition columnDefinition) { - if (!hasGroupingTable) { - return null; - } - final URI parquetFileURI = tl().getParquetKey().getURI(); - Assert.assertion(FILE_URI_SCHEME.equals(parquetFileURI.getScheme()), - "Expected a file uri, got " + parquetFileURI); - final File parquetFile = new File(parquetFileURI); - try { - ParquetFileReader parquetFileReader; - final String indexFilePath; - final GroupingColumnInfo groupingColumnInfo = tl().getGroupingColumns().get(parquetColumnName); - final SeekableChannelsProvider channelsProvider = tl().getChannelProvider(); - if (groupingColumnInfo != null) { - final String indexFileRelativePath = groupingColumnInfo.groupingTablePath(); - indexFilePath = parquetFile.toPath().getParent().resolve(indexFileRelativePath).toString(); - try { - parquetFileReader = new ParquetFileReader(indexFilePath, channelsProvider); - } catch (final RuntimeException e) { - logWarnFailedToRead(indexFilePath); - return null; - } - } else { - final String relativeIndexFilePath = - ParquetTools.getRelativeIndexFilePath(parquetFile, parquetColumnName); - indexFilePath = parquetFile.toPath().getParent().resolve(relativeIndexFilePath).toString(); - try { - parquetFileReader = new ParquetFileReader(indexFilePath, channelsProvider); - } catch (final RuntimeException e1) { - // Retry with legacy grouping file path - final String legacyGroupingFileName = - ParquetTools.legacyGroupingFileName(parquetFile, parquetColumnName); - final File legacyGroupingFile = new File(parquetFile.getParent(), legacyGroupingFileName); - try { - parquetFileReader = - new ParquetFileReader(legacyGroupingFile.getAbsolutePath(), channelsProvider); - } catch (final RuntimeException e2) { - logWarnFailedToRead(indexFilePath); - return null; - } - } - } - final Optional tableInfo = ParquetSchemaReader.parseMetadata( - new ParquetMetadataConverter().fromParquetMetadata(parquetFileReader.fileMetaData) - .getFileMetaData().getKeyValueMetaData()); - final Map columnTypes = - tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); - final String version = tableInfo.map(TableInfo::version).orElse(null); - - final RowGroupReader rowGroupReader = parquetFileReader.getRowGroup(0, version); - final ColumnChunkReader groupingKeyReader, beginPosReader, endPosReader; - try (final SeekableChannelContext channelContext = channelsProvider.makeSingleUseContext()) { - groupingKeyReader = - rowGroupReader.getColumnChunk(Collections.singletonList(GROUPING_KEY), channelContext); - beginPosReader = rowGroupReader.getColumnChunk(Collections.singletonList(BEGIN_POS), channelContext); - endPosReader = rowGroupReader.getColumnChunk(Collections.singletonList(END_POS), channelContext); - } - if (groupingKeyReader == null || beginPosReader == null || endPosReader == null) { - log.warn().append("Index file ").append(indexFilePath) - .append(" is missing one or more expected columns for table location ") - .append(tl()).append(", column ").append(getName()).endl(); - return null; - } - - final PageCache localPageCache = ensurePageCache(); - - // noinspection unchecked - return (METADATA_TYPE) new MetaDataTableFactory( - ColumnChunkPageStore.create( - localPageCache.castAttr(), groupingKeyReader, - ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, - makeToPage(columnTypes.get(GROUPING_KEY), ParquetInstructions.EMPTY, - GROUPING_KEY, groupingKeyReader, columnDefinition), - columnDefinition).pageStore, - ColumnChunkPageStore.create( - localPageCache.castAttr(), beginPosReader, - ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, - makeToPage(columnTypes.get(BEGIN_POS), ParquetInstructions.EMPTY, BEGIN_POS, - beginPosReader, FIRST_KEY_COL_DEF), - columnDefinition).pageStore, - ColumnChunkPageStore.create( - localPageCache.castAttr(), endPosReader, - ROW_KEY_TO_SUB_REGION_ROW_INDEX_MASK, - makeToPage(columnTypes.get(END_POS), ParquetInstructions.EMPTY, END_POS, - endPosReader, LAST_KEY_COL_DEF), - columnDefinition).pageStore) - .get(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } + return null; } private REGION_TYPE makeColumnRegion( @@ -459,148 +330,6 @@ private void fetchValues(@NotNull final ColumnDefinition columnDefinition) { } } - private static final class MetaDataTableFactory { - - private final ColumnChunkPageStore keyColumn; - private final ColumnChunkPageStore firstColumn; - private final ColumnChunkPageStore lastColumn; - - private volatile Object metaData; - - private MetaDataTableFactory(@NotNull final ColumnChunkPageStore keyColumn, - @NotNull final ColumnChunkPageStore firstColumn, - @NotNull final ColumnChunkPageStore lastColumn) { - this.keyColumn = Require.neqNull(keyColumn, "keyColumn"); - this.firstColumn = Require.neqNull(firstColumn, "firstColumn"); - this.lastColumn = Require.neqNull(lastColumn, "lastColumn"); - } - - public Object get() { - if (metaData != null) { - return metaData; - } - synchronized (this) { - if (metaData != null) { - return metaData; - } - final int numRows = (int) keyColumn.numRows(); - - try ( - final ChunkBoxer.BoxerKernel boxerKernel = - ChunkBoxer.getBoxer(keyColumn.getChunkType(), CHUNK_SIZE); - final BuildGrouping buildGrouping = - BuildGrouping.builder(firstColumn.getChunkType(), numRows); - final ChunkSource.GetContext keyContext = keyColumn.makeGetContext(CHUNK_SIZE); - final ChunkSource.GetContext firstContext = - firstColumn.makeGetContext(CHUNK_SIZE); - final ChunkSource.GetContext lastContext = - lastColumn.makeGetContext(CHUNK_SIZE); - final RowSequence rows = RowSequenceFactory.forRange(0, numRows - 1); - final RowSequence.Iterator rowsIterator = rows.getRowSequenceIterator()) { - - while (rowsIterator.hasMore()) { - final RowSequence chunkRows = - rowsIterator.getNextRowSequenceWithLength(CHUNK_SIZE); - - buildGrouping.build( - boxerKernel.box(keyColumn.getChunk(keyContext, chunkRows)), - firstColumn.getChunk(firstContext, chunkRows), - lastColumn.getChunk(lastContext, chunkRows)); - } - - metaData = buildGrouping.getGrouping(); - } - } - return metaData; - } - - private interface BuildGrouping extends Context { - void build(@NotNull ObjectChunk keyChunk, - @NotNull Chunk firstChunk, - @NotNull Chunk lastChunk); - - Object getGrouping(); - - static BuildGrouping builder(@NotNull final ChunkType chunkType, final int numRows) { - switch (chunkType) { - case Int: - return new IntBuildGrouping(numRows); - case Long: - return new LongBuildGrouping(numRows); - default: - throw new IllegalArgumentException( - "Unknown type for a rowSet: " + chunkType); - } - } - - final class IntBuildGrouping implements BuildGrouping { - - private final Map grouping; - - IntBuildGrouping(final int numRows) { - grouping = new LinkedHashMap<>(numRows); - } - - @Override - public void build(@NotNull final ObjectChunk keyChunk, - @NotNull final Chunk firstChunk, - @NotNull final Chunk lastChunk) { - final IntChunk firstIntChunk = - firstChunk.asIntChunk(); - final IntChunk lastIntChunk = - lastChunk.asIntChunk(); - - for (int ki = 0; ki < keyChunk.size(); ++ki) { - final int[] range = new int[2]; - - range[0] = firstIntChunk.get(ki); - range[1] = lastIntChunk.get(ki); - - grouping.put(keyChunk.get(ki), range); - } - } - - @Override - public Object getGrouping() { - return grouping; - } - } - - final class LongBuildGrouping implements BuildGrouping { - - private final Map grouping; - - LongBuildGrouping(final int numRows) { - grouping = new LinkedHashMap<>(numRows); - } - - @Override - public void build(@NotNull final ObjectChunk keyChunk, - @NotNull final Chunk firstChunk, - @NotNull final Chunk lastChunk) { - final LongChunk firstLongChunk = - firstChunk.asLongChunk(); - final LongChunk lastLongChunk = - lastChunk.asLongChunk(); - - for (int ki = 0; ki < keyChunk.size(); ++ki) { - final long[] range = new long[2]; - - range[0] = firstLongChunk.get(ki); - range[1] = lastLongChunk.get(ki); - - grouping.put(keyChunk.get(ki), range); - } - } - - @Override - public Object getGrouping() { - return grouping; - } - } - } - } - private static ToPage makeToPage( @Nullable final ColumnTypeInfo columnTypeInfo, @NotNull final ParquetInstructions readInstructions, diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java index 8c045150819..69d30847418 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java @@ -3,49 +3,73 @@ // package io.deephaven.parquet.table.location; +import io.deephaven.api.SortColumn; +import io.deephaven.base.verify.Assert; +import io.deephaven.base.verify.Require; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetBuilderSequential; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.BasicDataIndex; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.dataindex.StandaloneDataIndex; +import io.deephaven.engine.table.impl.locations.ColumnLocation; +import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.table.impl.locations.TableKey; import io.deephaven.engine.table.impl.locations.TableLocationState; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; -import io.deephaven.util.channel.SeekableChannelContext; -import io.deephaven.parquet.table.ParquetInstructions; -import io.deephaven.parquet.table.ParquetSchemaReader; -import io.deephaven.parquet.table.metadata.ColumnTypeInfo; -import io.deephaven.parquet.table.metadata.GroupingColumnInfo; -import io.deephaven.parquet.table.metadata.TableInfo; -import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.impl.select.MultiSourceFunctionalColumn; +import io.deephaven.engine.table.impl.select.SourceColumn; import io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource; import io.deephaven.engine.table.impl.sources.regioned.RegionedPageStore; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderSequential; -import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ParquetFileReader; import io.deephaven.parquet.base.RowGroupReader; +import io.deephaven.parquet.table.ParquetInstructions; +import io.deephaven.parquet.table.ParquetSchemaReader; +import io.deephaven.parquet.table.ParquetTools; +import io.deephaven.parquet.table.metadata.ColumnTypeInfo; +import io.deephaven.parquet.table.metadata.DataIndexInfo; +import io.deephaven.parquet.table.metadata.GroupingColumnInfo; +import io.deephaven.parquet.table.metadata.TableInfo; +import io.deephaven.util.channel.SeekableChannelContext; import io.deephaven.util.channel.SeekableChannelsProvider; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.format.RowGroup; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.File; +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.*; import java.util.stream.IntStream; import static io.deephaven.parquet.base.ParquetFileReader.FILE_URI_SCHEME; +import static io.deephaven.parquet.table.ParquetTableWriter.*; +import static io.deephaven.parquet.table.ParquetTableWriter.GROUPING_END_POS_COLUMN_NAME; public class ParquetTableLocation extends AbstractTableLocation { private static final String IMPLEMENTATION_NAME = ParquetColumnLocation.class.getSimpleName(); private final ParquetInstructions readInstructions; + private final List sortingColumns; private final ParquetFileReader parquetFileReader; private final int[] rowGroupIndices; private final RowGroup[] rowGroups; private final RegionedPageStore.Parameters regionParameters; private final Map parquetColumnNameToPath; + + private final TableInfo tableInfo; private final Map groupingColumns; + private final List dataIndexes; private final Map columnTypes; + private final String version; private volatile RowGroupReader[] rowGroupReaders; @@ -83,13 +107,16 @@ public ParquetTableLocation(@NotNull final TableKey tableKey, // TODO (https://github.com/deephaven/deephaven-core/issues/958): // When/if we support _metadata files for Deephaven-written Parquet tables, we may need to revise this // in order to read *this* file's metadata, rather than inheriting file metadata from the _metadata file. - // Obvious issues included grouping table paths, codecs, etc. + // Obvious issues included data index table paths, codecs, etc. // Presumably, we could store per-file instances of the metadata in the _metadata file's map. - final Optional tableInfo = - ParquetSchemaReader.parseMetadata(parquetMetadata.getFileMetaData().getKeyValueMetaData()); - groupingColumns = tableInfo.map(TableInfo::groupingColumnMap).orElse(Collections.emptyMap()); - columnTypes = tableInfo.map(TableInfo::columnTypeMap).orElse(Collections.emptyMap()); - version = tableInfo.map(TableInfo::version).orElse(null); + tableInfo = ParquetSchemaReader + .parseMetadata(parquetMetadata.getFileMetaData().getKeyValueMetaData()) + .orElse(TableInfo.builder().build()); + version = tableInfo.version(); + groupingColumns = tableInfo.groupingColumnMap(); + dataIndexes = tableInfo.dataIndexes(); + columnTypes = tableInfo.columnTypeMap(); + sortingColumns = tableInfo.sortingColumns(); if (!FILE_URI_SCHEME.equals(tableLocationKey.getURI().getScheme())) { // We do not have the last modified time for non-file URIs @@ -123,10 +150,6 @@ RegionedPageStore.Parameters getRegionParameters() { return regionParameters; } - public Map getGroupingColumns() { - return groupingColumns; - } - public Map getColumnTypes() { return columnTypes; } @@ -147,9 +170,15 @@ private RowGroupReader[] getRowGroupReaders() { } } + @Override @NotNull + public List getSortedColumns() { + return sortingColumns; + } + @Override - protected ParquetColumnLocation makeColumnLocation(@NotNull final String columnName) { + @NotNull + protected ColumnLocation makeColumnLocation(@NotNull final String columnName) { final String parquetColumnName = readInstructions.getParquetColumnNameFromColumnNameOrDefault(columnName); final String[] columnPath = parquetColumnNameToPath.get(parquetColumnName); final List nameList = @@ -161,8 +190,7 @@ protected ParquetColumnLocation makeColumnLocation(@NotNull final String } final boolean exists = Arrays.stream(columnChunkReaders).anyMatch(ccr -> ccr != null && ccr.numRows() > 0); return new ParquetColumnLocation<>(this, columnName, parquetColumnName, - exists ? columnChunkReaders : null, - exists && groupingColumns.containsKey(parquetColumnName)); + exists ? columnChunkReaders : null); } private RowSet computeIndex() { @@ -176,4 +204,176 @@ private RowSet computeIndex() { } return sequentialBuilder.build(); } + + @Override + @NotNull + public List getDataIndexColumns() { + if (dataIndexes.isEmpty() && groupingColumns.isEmpty()) { + return List.of(); + } + final List dataIndexColumns = new ArrayList<>(dataIndexes.size() + groupingColumns.size()); + // Add the data indexes to the list + dataIndexes.stream().map(di -> di.columns().toArray(String[]::new)).forEach(dataIndexColumns::add); + // Add grouping columns to the list + groupingColumns.keySet().stream().map(colName -> new String[] {colName}).forEach(dataIndexColumns::add); + return dataIndexColumns; + } + + @Override + public boolean hasDataIndex(@NotNull final String... columns) { + // Check if the column name matches any of the grouping columns + if (columns.length == 1 && groupingColumns.containsKey(columns[0])) { + // Validate the index file exists (without loading and parsing it) + final IndexFileMetadata metadata = getIndexFileMetadata(getParquetKey().getURI(), tableInfo, columns); + return metadata != null && parquetFileExists(metadata.fileURI); + } + // Check if the column names match any of the data indexes + for (final DataIndexInfo dataIndex : dataIndexes) { + if (dataIndex.matchesColumns(columns)) { + // Validate the index file exists (without loading and parsing it) + final IndexFileMetadata metadata = getIndexFileMetadata( + getParquetKey().getURI(), + tableInfo, + columns); + return metadata != null && parquetFileExists(metadata.fileURI); + } + } + return false; + } + + private static boolean parquetFileExists(@NotNull final URI fileURI) { + // We assume non-file URIs are always present + return !fileURI.getScheme().equals(FILE_URI_SCHEME) || Files.exists(Path.of(fileURI)); + } + + @Nullable + @Override + public BasicDataIndex loadDataIndex(@NotNull final String... columns) { + if (tableInfo == null) { + return null; + } + final IndexFileMetadata indexFileMetaData = getIndexFileMetadata(getParquetKey().getURI(), tableInfo, columns); + if (indexFileMetaData == null) { + throw new TableDataException( + String.format( + "No index metadata for table %s with index key columns %s was present in TableInfo", + getParquetKey().getURI(), Arrays.toString(columns))); + } + // Create a new index from the parquet table + final Table table = readDataIndexTable(getParquetKey().getURI(), indexFileMetaData, readInstructions); + if (table == null) { + return null; + } + return StandaloneDataIndex.from(table, columns, INDEX_ROW_SET_COLUMN_NAME); + } + + private static class IndexFileMetadata { + + private final URI fileURI; + private final GroupingColumnInfo groupingColumnInfo; + private final DataIndexInfo dataIndexInfo; + + private IndexFileMetadata( + @NotNull final URI fileURI, + @Nullable final GroupingColumnInfo groupingColumnInfo, + @Nullable final DataIndexInfo dataIndexInfo) { + this.fileURI = fileURI; + Require.requirement(groupingColumnInfo != null ^ dataIndexInfo != null, + "Exactly one of groupingColumnInfo and dataIndexInfo must be non-null"); + this.groupingColumnInfo = groupingColumnInfo; + this.dataIndexInfo = dataIndexInfo; + } + } + + private static URI makeRelativeURI(@NotNull final URI parentFileURI, @NotNull final String relativePath) { + final String parentPath = parentFileURI.getPath(); + final int lastSlashIndex = parentPath.lastIndexOf('/'); + try { + return new URI(parentFileURI.getScheme(), parentFileURI.getAuthority(), + (lastSlashIndex == -1 ? parentPath : parentPath.substring(0, lastSlashIndex + 1)) + relativePath, + null, null); + } catch (URISyntaxException e) { + throw new TableDataException(String.format("Failed to format relative URI for parent %s, child %s", + parentFileURI, relativePath), e); + } + } + + private static IndexFileMetadata getIndexFileMetadata( + @NotNull final URI parentFileURI, + @NotNull final TableInfo info, + @NotNull final String... keyColumnNames) { + if (keyColumnNames.length == 1) { + // If there's only one key column, there might be (legacy) grouping info + final GroupingColumnInfo groupingColumnInfo = info.groupingColumnMap().get(keyColumnNames[0]); + if (groupingColumnInfo != null) { + return new IndexFileMetadata( + makeRelativeURI(parentFileURI, groupingColumnInfo.groupingTablePath()), + groupingColumnInfo, + null); + } + } + + // Either there are more than 1 key columns, or there was no grouping info, so lets see if there was a + // DataIndex. + final DataIndexInfo dataIndexInfo = info.dataIndexes().stream() + .filter(item -> item.matchesColumns(keyColumnNames)) + .findFirst() + .orElse(null); + + if (dataIndexInfo != null) { + return new IndexFileMetadata( + makeRelativeURI(parentFileURI, dataIndexInfo.indexTablePath()), + null, + dataIndexInfo); + } + + // We have no index metadata. We intentionally do not fall back to the legacy path from pre-metadata versions + // of this codee, as it's not expected that such tables exist in the wild. + return null; + } + + // region Indexing + /** + * Read a Data Index table from disk. + * + * @param parentFileURI The path to the base table + * @param indexFileMetaData Index file metadata + * @param parquetInstructions The instructions for reading the table + * + * @return The data index table for the specified key columns or {@code null} if none was found + */ + @Nullable + private static Table readDataIndexTable( + @NotNull final URI parentFileURI, + @NotNull final ParquetTableLocation.IndexFileMetadata indexFileMetaData, + @NotNull final ParquetInstructions parquetInstructions) { + final Table indexTable = + ParquetTools.readSingleFileTable(indexFileMetaData.fileURI.toString(), parquetInstructions); + if (indexFileMetaData.dataIndexInfo != null) { + return indexTable; + } + Assert.neqNull(indexFileMetaData.groupingColumnInfo, "indexFileMetaData.groupingColumnInfo"); + if (indexTable.hasColumns( + GROUPING_KEY_COLUMN_NAME, GROUPING_BEGIN_POS_COLUMN_NAME, GROUPING_END_POS_COLUMN_NAME)) { + // Legacy grouping tables are written with a key, start position, and end position. We must convert the + // ranges to RowSets. + return indexTable.view(List.of( + new SourceColumn(GROUPING_KEY_COLUMN_NAME, indexFileMetaData.groupingColumnInfo.columnName()), + // Using this lets us avoid a compilation or boxing, but does force us to do single-cell access + // rather than using chunks. + new MultiSourceFunctionalColumn<>( + List.of(GROUPING_BEGIN_POS_COLUMN_NAME, GROUPING_END_POS_COLUMN_NAME), + INDEX_ROW_SET_COLUMN_NAME, + RowSet.class, + (final long rowKey, final ColumnSource[] sources) -> RowSetFactory + .fromRange(sources[0].getLong(rowKey), sources[1].getLong(rowKey) - 1)))); + } else { + throw new TableDataException(String.format( + "Index table %s for table %s was not in the expected format. Expected columns [%s] but encountered [%s]", + indexFileMetaData.fileURI, parentFileURI, + String.join(", ", + GROUPING_KEY_COLUMN_NAME, GROUPING_BEGIN_POS_COLUMN_NAME, GROUPING_END_POS_COLUMN_NAME), + indexTable.getDefinition().getColumnNamesAsString())); + } + } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/metadata/DataIndexInfo.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/metadata/DataIndexInfo.java new file mode 100644 index 00000000000..9cbd4d3349b --- /dev/null +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/metadata/DataIndexInfo.java @@ -0,0 +1,65 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.table.metadata; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import io.deephaven.annotations.SimpleStyle; +import io.deephaven.engine.util.string.StringUtils; +import org.immutables.value.Value; +import org.jetbrains.annotations.NotNull; + +import java.util.Arrays; +import java.util.Set; + +/** + * Representation class for data index information stored in key-value metadata for Deephaven-written Parquet files. + */ +@Value.Immutable +@SimpleStyle +@JsonSerialize(as = ImmutableDataIndexInfo.class) +@JsonDeserialize(as = ImmutableDataIndexInfo.class) +@JsonInclude(JsonInclude.Include.NON_EMPTY) +public abstract class DataIndexInfo { + + /** + * @return The column names + */ + @Value.Parameter + public abstract Set columns(); + + /** + * @return The relative path name for the columns' data index sidecar table + */ + @Value.Parameter + public abstract String indexTablePath(); + + @Value.Check + final void checkColumns() { + if (columns().isEmpty()) { + throw new IllegalArgumentException("No columns provided"); + } + if (columns().stream().anyMatch(StringUtils::isNullOrEmpty)) { + throw new IllegalArgumentException("Empty column name"); + } + } + + @Value.Check + final void checkIndexTablePath() { + if (indexTablePath().isEmpty()) { + throw new IllegalArgumentException("Empty index table path"); + } + } + + public boolean matchesColumns(final String... columnsToMatch) { + final Set localColumns = columns(); + return localColumns.size() == columnsToMatch.length + && Arrays.stream(columnsToMatch).allMatch(localColumns::contains); + } + + public static DataIndexInfo of(@NotNull final String indexTablePath, final String... columnNames) { + return ImmutableDataIndexInfo.of(Arrays.asList(columnNames), indexTablePath); + } +} diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/metadata/TableInfo.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/metadata/TableInfo.java index de7cbe2187e..a506cd8e91f 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/metadata/TableInfo.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/metadata/TableInfo.java @@ -10,13 +10,13 @@ import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; import io.deephaven.annotations.BuildableStyle; +import io.deephaven.api.SortColumn; import org.immutables.value.Value; import org.jetbrains.annotations.NotNull; import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -47,10 +47,6 @@ public static TableInfo deserializeFromJSON(@NotNull final String tableInfoRaw) return OBJECT_MAPPER.readValue(tableInfoRaw, ImmutableTableInfo.class); } - public final Set groupingColumnNames() { - return groupingColumns().stream().map(GroupingColumnInfo::columnName).collect(Collectors.toSet()); - } - public final Map groupingColumnMap() { return groupingColumns().stream() .collect(Collectors.toMap(GroupingColumnInfo::columnName, Function.identity())); @@ -66,6 +62,7 @@ public final Map columnTypeMap() { @Value.Default public String version() { final String version = TableInfo.class.getPackage().getImplementationVersion(); + // noinspection ReplaceNullCheck if (version == null) { // When the code is run from class files as opposed to jars, like in unit tests return "unknown"; @@ -78,12 +75,20 @@ public String version() { */ public abstract List groupingColumns(); + /** + * @return List of {@link DataIndexInfo data indexes} for this table + */ + public abstract List dataIndexes(); + /** * @return List of {@link ColumnTypeInfo column types} for columns requiring non-default deserialization or type * selection */ public abstract List columnTypes(); + public abstract List sortingColumns(); + + @Value.Check final void checkVersion() { if (version().isEmpty()) { @@ -105,12 +110,24 @@ public interface Builder { Builder addAllGroupingColumns(Iterable groupingColumns); + Builder addDataIndexes(DataIndexInfo info); + + Builder addDataIndexes(DataIndexInfo... infos); + + Builder addAllDataIndexes(Iterable infos); + Builder addColumnTypes(ColumnTypeInfo columnType); Builder addColumnTypes(ColumnTypeInfo... columnTypes); Builder addAllColumnTypes(Iterable columnTypes); + Builder addSortingColumns(SortColumn sortPair); + + Builder addSortingColumns(SortColumn... sortPairs); + + Builder addAllSortingColumns(Iterable sortPairs); + TableInfo build(); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToVectorPage.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToVectorPage.java index 32e436c288e..6d64fce2239 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToVectorPage.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/topage/ToVectorPage.java @@ -17,7 +17,8 @@ public class ToVectorPage nativeType; - public static ToPage create(@NotNull final Class nativeType, + public static ToPage create( + @NotNull final Class nativeType, @NotNull final Class componentType, @NotNull final ToPage toPage) { if (!Vector.class.isAssignableFrom(nativeType)) { diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java index 0dfeec49305..d9b35726664 100644 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java +++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableReadWriteTest.java @@ -6,6 +6,7 @@ import io.deephaven.UncheckedDeephavenException; import io.deephaven.api.Selectable; import io.deephaven.base.FileUtils; +import io.deephaven.base.verify.Assert; import io.deephaven.configuration.Configuration; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.primitive.function.ByteConsumer; @@ -13,56 +14,54 @@ import io.deephaven.engine.primitive.function.FloatConsumer; import io.deephaven.engine.primitive.function.ShortConsumer; import io.deephaven.engine.primitive.iterator.CloseableIterator; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.SourceTable; +import io.deephaven.engine.table.impl.dataindex.DataIndexUtils; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.locations.TableDataException; +import io.deephaven.engine.table.impl.select.FormulaEvaluationException; import io.deephaven.engine.table.impl.select.FunctionalColumn; import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.util.ColumnHolder; -import io.deephaven.engine.table.impl.select.FormulaEvaluationException; import io.deephaven.engine.table.iterators.*; import io.deephaven.engine.testutil.ControlledUpdateGraph; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.engine.util.BigDecimalUtils; +import io.deephaven.engine.util.TableTools; import io.deephaven.engine.util.file.TrackedFileHandleFactory; import io.deephaven.extensions.s3.Credentials; -import io.deephaven.parquet.base.NullStatistics; +import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.parquet.base.InvalidParquetFileException; +import io.deephaven.parquet.base.NullStatistics; import io.deephaven.parquet.table.location.ParquetTableLocationKey; import io.deephaven.parquet.table.pagestore.ColumnChunkPageStore; import io.deephaven.parquet.table.transfer.StringDictionary; -import io.deephaven.extensions.s3.S3Instructions; import io.deephaven.stringset.ArrayStringSet; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableDefinition; import io.deephaven.stringset.StringSet; -import io.deephaven.engine.util.TableTools; -import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.test.types.OutOfBandTest; +import io.deephaven.time.DateTimeUtils; import io.deephaven.util.QueryConstants; import io.deephaven.util.codec.SimpleByteArrayCodec; -import junit.framework.TestCase; -import org.apache.parquet.column.Encoding; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import io.deephaven.time.DateTimeUtils; import io.deephaven.util.compare.DoubleComparisons; import io.deephaven.util.compare.FloatComparisons; +import io.deephaven.vector.Vector; import io.deephaven.vector.*; +import junit.framework.TestCase; import org.apache.commons.lang3.mutable.*; +import org.apache.parquet.column.Encoding; import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.PrimitiveType; -import org.junit.After; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; +import org.junit.*; +import org.junit.experimental.categories.Category; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.io.Serializable; @@ -71,43 +70,16 @@ import java.net.URI; import java.time.Duration; import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Collection; -import java.util.Map; -import java.util.Set; -import java.util.Objects; +import java.util.*; import java.util.function.DoubleConsumer; import java.util.function.Function; import java.util.function.IntConsumer; import java.util.function.LongConsumer; -import org.junit.experimental.categories.Category; - -import javax.annotation.Nullable; - import static io.deephaven.base.FileUtils.convertToURI; import static io.deephaven.engine.testutil.TstUtils.assertTableEquals; -import static io.deephaven.engine.util.TableTools.booleanCol; -import static io.deephaven.engine.util.TableTools.byteCol; -import static io.deephaven.engine.util.TableTools.charCol; -import static io.deephaven.engine.util.TableTools.doubleCol; -import static io.deephaven.engine.util.TableTools.emptyTable; -import static io.deephaven.engine.util.TableTools.floatCol; -import static io.deephaven.engine.util.TableTools.instantCol; -import static io.deephaven.engine.util.TableTools.intCol; -import static io.deephaven.engine.util.TableTools.longCol; -import static io.deephaven.engine.util.TableTools.merge; -import static io.deephaven.engine.util.TableTools.newTable; -import static io.deephaven.engine.util.TableTools.shortCol; -import static io.deephaven.engine.util.TableTools.stringCol; -import static io.deephaven.parquet.table.ParquetTools.readFlatPartitionedTable; -import static io.deephaven.parquet.table.ParquetTools.readKeyValuePartitionedTable; -import static io.deephaven.parquet.table.ParquetTools.readSingleFileTable; -import static io.deephaven.parquet.table.ParquetTools.readTable; -import static io.deephaven.parquet.table.ParquetTools.writeTable; +import static io.deephaven.engine.util.TableTools.*; +import static io.deephaven.parquet.table.ParquetTools.*; import static io.deephaven.util.QueryConstants.*; import static org.junit.Assert.*; @@ -323,47 +295,144 @@ public void vectorParquetFormat() { } @Test - public void groupingByLongKey() { + public void indexByLongKey() { final TableDefinition definition = TableDefinition.of( ColumnDefinition.ofInt("someInt"), - ColumnDefinition.ofLong("someLong").withGrouping()); + ColumnDefinition.ofLong("someLong")); + final Table testTable = ((QueryTable) TableTools.emptyTable(10).select("someInt = i", "someLong = ii % 3") .groupBy("someLong").ungroup("someInt")).withDefinitionUnsafe(definition); + DataIndexer.getOrCreateDataIndex(testTable, "someLong"); + DataIndexer.getOrCreateDataIndex(testTable, "someInt", "someLong"); + final File dest = new File(rootFile, "ParquetTest_groupByLong_test.parquet"); writeTable(testTable, dest); final Table fromDisk = checkSingleTable(testTable, dest); - TestCase.assertNotNull(fromDisk.getColumnSource("someLong").getGroupToRange()); + + // Validate the indexes and lookup functions. + ColumnSource[] columns = Arrays.stream(new String[] {"someLong"}).map(fromDisk::getColumnSource) + .toArray(ColumnSource[]::new); + DataIndex fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + Assert.neqNull(fullIndex, "fullIndex"); + assertLookupFromTable(fromDisk, fullIndex, columns); + + columns = Arrays.stream(new String[] {"someInt", "someLong"}).map(fromDisk::getColumnSource) + .toArray(ColumnSource[]::new); + fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + Assert.neqNull(fullIndex, "fullIndex"); + assertLookupFromTable(fromDisk, fullIndex, columns); + + columns = Arrays.stream(new String[] {"someLong", "someInt"}).map(fromDisk::getColumnSource) + .toArray(ColumnSource[]::new); + fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + Assert.neqNull(fullIndex, "fullIndex"); + assertLookupFromTable(fromDisk, fullIndex, columns); } @Test - public void groupingByStringKey() { + public void indexByStringKey() { final TableDefinition definition = TableDefinition.of( ColumnDefinition.ofInt("someInt"), - ColumnDefinition.ofString("someString").withGrouping()); + ColumnDefinition.ofString("someString")); final Table testTable = ((QueryTable) TableTools.emptyTable(10).select("someInt = i", "someString = `foo`") .where("i % 2 == 0").groupBy("someString").ungroup("someInt")) .withDefinitionUnsafe(definition); + + DataIndexer.getOrCreateDataIndex(testTable, "someString"); + DataIndexer.getOrCreateDataIndex(testTable, "someInt", "someString"); + final File dest = new File(rootFile, "ParquetTest_groupByString_test.parquet"); writeTable(testTable, dest); final Table fromDisk = checkSingleTable(testTable, dest); - TestCase.assertNotNull(fromDisk.getColumnSource("someString").getGroupToRange()); + + // Validate the indexes and lookup functions. + ColumnSource[] columns = Arrays.stream(new String[] {"someString"}).map(fromDisk::getColumnSource) + .toArray(ColumnSource[]::new); + DataIndex fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + Assert.neqNull(fullIndex, "fullIndex"); + assertLookupFromTable(fromDisk, fullIndex, columns); + + columns = Arrays.stream(new String[] {"someInt", "someString"}).map(fromDisk::getColumnSource) + .toArray(ColumnSource[]::new); + fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + Assert.neqNull(fullIndex, "fullIndex"); + assertLookupFromTable(fromDisk, fullIndex, columns); + + columns = Arrays.stream(new String[] {"someString", "someInt"}).map(fromDisk::getColumnSource) + .toArray(ColumnSource[]::new); + fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + Assert.neqNull(fullIndex, "fullIndex"); + assertLookupFromTable(fromDisk, fullIndex, columns); } @Test - public void groupingByBigInt() { + public void indexByBigInt() { ExecutionContext.getContext().getQueryLibrary().importClass(BigInteger.class); final TableDefinition definition = TableDefinition.of( ColumnDefinition.ofInt("someInt"), - ColumnDefinition.fromGenericType("someBigInt", BigInteger.class).withGrouping()); + ColumnDefinition.fromGenericType("someBigInt", BigInteger.class)); final Table testTable = ((QueryTable) TableTools.emptyTable(10) .select("someInt = i", "someBigInt = BigInteger.valueOf(i % 3)").where("i % 2 == 0") .groupBy("someBigInt").ungroup("someInt")).withDefinitionUnsafe(definition); + + DataIndexer.getOrCreateDataIndex(testTable, "someBigInt"); + DataIndexer.getOrCreateDataIndex(testTable, "someInt", "someBigInt"); + + final File dest = new File(rootFile, "ParquetTest_groupByBigInt_test.parquet"); writeTable(testTable, dest); final Table fromDisk = checkSingleTable(testTable, dest); - TestCase.assertNotNull(fromDisk.getColumnSource("someBigInt").getGroupToRange()); + + // Validate the indexes and lookup functions. + ColumnSource[] columns = Arrays.stream(new String[] {"someBigInt"}).map(fromDisk::getColumnSource) + .toArray(ColumnSource[]::new); + DataIndex fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + Assert.neqNull(fullIndex, "fullIndex"); + assertLookupFromTable(fromDisk, fullIndex, columns); + + columns = Arrays.stream(new String[] {"someInt", "someBigInt"}).map(fromDisk::getColumnSource) + .toArray(ColumnSource[]::new); + fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + Assert.neqNull(fullIndex, "fullIndex"); + assertLookupFromTable(fromDisk, fullIndex, columns); + + columns = Arrays.stream(new String[] {"someBigInt", "someInt"}).map(fromDisk::getColumnSource) + .toArray(ColumnSource[]::new); + fullIndex = DataIndexer.of(fromDisk.getRowSet()).getDataIndex(columns); + Assert.neqNull(fullIndex, "fullIndex"); + assertLookupFromTable(fromDisk, fullIndex, columns); + } + + private void assertLookupFromTable( + final Table sourceTable, + final DataIndex fullIndex, + final ColumnSource[] columns) { + final DataIndex.RowKeyLookup fullIndexRowKeyLookup = fullIndex.rowKeyLookup(columns); + final ColumnSource fullIndexRowSetColumn = fullIndex.rowSetColumn(); + + ChunkSource.WithPrev tableKeys = DataIndexUtils.makeBoxedKeySource(columns); + + // Iterate through the entire source table and verify the lookup row set is valid and contains this row. + try (final RowSet.Iterator rsIt = sourceTable.getRowSet().iterator(); + final CloseableIterator keyIt = + ChunkedColumnIterator.make(tableKeys, sourceTable.getRowSet())) { + + while (rsIt.hasNext() && keyIt.hasNext()) { + final long rowKey = rsIt.nextLong(); + final Object key = keyIt.next(); + + // Verify the row sets at the lookup keys match. + final long fullRowKey = fullIndexRowKeyLookup.apply(key, false); + Assert.geqZero(fullRowKey, "fullRowKey"); + + final RowSet fullRowSet = fullIndexRowSetColumn.get(fullRowKey); + assertNotNull(fullRowSet); + + assertTrue(fullRowSet.containsRange(rowKey, rowKey)); + } + } } private void compressionCodecTestHelper(final ParquetInstructions codec) { @@ -503,6 +572,7 @@ private static Table arrayToVectorTable(final Table table) { final Collection arrayToVectorFormulas = new ArrayList<>(); for (final ColumnDefinition columnDefinition : tableDefinition.getColumns()) { final String columnName = columnDefinition.getName(); + // noinspection unchecked final Class sourceDataType = (Class) columnDefinition.getDataType(); if (!sourceDataType.isArray()) { continue; @@ -686,7 +756,7 @@ public void stringDictionaryTest() { final Binary[] encodedKeys = dict.getEncodedKeys(); for (int i = 0; i < keys.length; i++) { final String decodedKey = encodedKeys[i].toStringUsingUTF8(); - final int expectedPos = keyToPos.get(decodedKey).intValue(); + final int expectedPos = keyToPos.get(decodedKey); assertEquals(i, expectedPos); } assertEquals(nullPos, dict.add(null)); @@ -738,22 +808,22 @@ private static Table readParquetFileFromGitLFS(final File dest) { * data is generated using following logic: * *
-     *  // Enforce a smaller page size to write multiple pages
-     *  final ParquetInstructions writeInstructions = new ParquetInstructions.Builder()
-     *        .setTargetPageSize(ParquetInstructions.MIN_TARGET_PAGE_SIZE)
-     *        .build();
+     * // Enforce a smaller page size to write multiple pages
+     * final ParquetInstructions writeInstructions = new ParquetInstructions.Builder()
+     *         .setTargetPageSize(ParquetInstructions.MIN_TARGET_PAGE_SIZE)
+     *         .build();
      *
-     *  final Table table = getTableFlat(5000, true, false);
-     *  ParquetTools.writeTable(table, new File("ReferenceParquetData.parquet"), writeInstructions);
+     * final Table table = getTableFlat(5000, true, false);
+     * ParquetTools.writeTable(table, new File("ReferenceParquetData.parquet"), writeInstructions);
      *
-     *  Table vectorTable = table.groupBy().select();
-     *  vectorTable = vectorTable.join(TableTools.emptyTable(100)).select();
-     *  ParquetTools.writeTable(vectorTable, new File("ReferenceParquetVectorData.parquet"), writeInstructions);
+     * Table vectorTable = table.groupBy().select();
+     * vectorTable = vectorTable.join(TableTools.emptyTable(100)).select();
+     * ParquetTools.writeTable(vectorTable, new File("ReferenceParquetVectorData.parquet"), writeInstructions);
      *
-     *  final Table arrayTable = vectorTable.updateView(vectorTable.getColumnSourceMap().keySet().stream()
+     * final Table arrayTable = vectorTable.updateView(vectorTable.getColumnSourceMap().keySet().stream()
      *         .map(name -> name + " = " + name + ".toArray()")
      *         .toArray(String[]::new));
-     *  ParquetTools.writeTable(arrayTable, new File("ReferenceParquetArrayData.parquet"), writeInstructions);
+     * ParquetTools.writeTable(arrayTable, new File("ReferenceParquetArrayData.parquet"), writeInstructions);
      * 
*/ @Test @@ -846,15 +916,15 @@ private interface TestParquetTableWriter { */ private static void verifyFilesInDir(final File parentDir, final String[] expectedDataFiles, @Nullable final Map indexingColumnToFileMap) { - final List filesInParentDir = Arrays.asList(parentDir.list()); + final List filesInParentDir = Arrays.asList(parentDir.list()); for (String expectedFile : expectedDataFiles) { assertTrue(filesInParentDir.contains(expectedFile)); } if (indexingColumnToFileMap == null) { - assertTrue(filesInParentDir.size() == expectedDataFiles.length); + assertEquals(filesInParentDir.size(), expectedDataFiles.length); return; } - assertTrue(filesInParentDir.size() == expectedDataFiles.length + 1); + assertEquals(filesInParentDir.size(), expectedDataFiles.length + 1); final File metadataDir = new File(parentDir, ".dh_metadata"); assertTrue(metadataDir.exists() && metadataDir.isDirectory() && metadataDir.list().length == 1); final File indexesDir = new File(metadataDir, "indexes"); @@ -866,7 +936,7 @@ private static void verifyFilesInDir(final File parentDir, final String[] expect final File indexColDir = new File(indexesDir, indexColName); assertTrue(indexColDir.exists() && indexColDir.isDirectory() && indexColDir.list().length == indexFilePaths.length); - final List filesInIndexColDir = Arrays.asList(indexColDir.list()); + final List filesInIndexColDir = Arrays.asList(indexColDir.list()); for (final String indexFilePath : indexFilePaths) { final File indexFile = new File(parentDir, indexFilePath); assertTrue(indexFile.exists() && indexFile.isFile() && indexFile.length() > 0 && @@ -894,7 +964,7 @@ public void readFromDirTest() { assertTableEquals(expected, fromDisk); // Read with a trailing slash - assertTrue(!filePath.endsWith("/")); + assertFalse(filePath.endsWith("/")); filePath = filePath + "/"; fromDisk = ParquetTools.readTable(filePath); assertTableEquals(expected, fromDisk); @@ -1042,7 +1112,7 @@ public void writeMultiTableExceptionTest() { } // All files should be deleted even though first table would be written successfully - assertTrue(parentDir.list().length == 0); + assertEquals(0, parentDir.list().length); } @Test @@ -1053,15 +1123,14 @@ public void writingParquetFilesWithSpacesInName() { .updateView("InputString = Long.toString(ii)", "A=InputString.charAt(0)"); writingParquetFilesWithSpacesInNameHelper(table, parentDirName, tableNameWithSpaces); - // Same test but for tables with grouping data - Integer data[] = new Integer[500 * 4]; + // Same test but for tables with index data + final int[] data = new int[500 * 4]; for (int i = 0; i < data.length; i++) { data[i] = i / 4; } - final TableDefinition groupingTableDefinition = - TableDefinition.of(ColumnDefinition.ofInt("vvv").withGrouping()); - final Table tableWithGroupingData = newTable(groupingTableDefinition, TableTools.col("vvv", data)); - writingParquetFilesWithSpacesInNameHelper(tableWithGroupingData, parentDirName, tableNameWithSpaces); + final Table indexedTable = newTable(TableTools.intCol("vvv", data)); + DataIndexer.getOrCreateDataIndex(indexedTable, "vvv"); + writingParquetFilesWithSpacesInNameHelper(indexedTable, parentDirName, tableNameWithSpaces); } private void writingParquetFilesWithSpacesInNameHelper(final Table table, final String parentDirName, @@ -1090,32 +1159,34 @@ private void writingParquetFilesWithSpacesInNameHelper(final Table table, final /** - * These are tests for writing to a table with grouping columns to a parquet file and making sure there are no - * unnecessary files left in the directory after we finish writing. + * These are tests for writing to a table with indexes to a parquet file and making sure there are no unnecessary + * files left in the directory after we finish writing. */ @Test - public void groupingColumnsBasicWriteTests() { - groupingColumnsBasicWriteTestsImpl(SINGLE_WRITER); - groupingColumnsBasicWriteTestsImpl(MULTI_WRITER); + public void indexedColumnsBasicWriteTests() { + indexedColumnsBasicWriteTestsImpl(SINGLE_WRITER); + indexedColumnsBasicWriteTestsImpl(MULTI_WRITER); } - public void groupingColumnsBasicWriteTestsImpl(TestParquetTableWriter writer) { + private void indexedColumnsBasicWriteTestsImpl(TestParquetTableWriter writer) { // Create an empty parent directory final File parentDir = new File(rootFile, "tempDir"); parentDir.mkdir(); assertTrue(parentDir.exists() && parentDir.isDirectory() && parentDir.list().length == 0); - Integer data[] = new Integer[500 * 4]; + final int[] data = new int[500 * 4]; for (int i = 0; i < data.length; i++) { data[i] = i / 4; } - final TableDefinition tableDefinition = TableDefinition.of(ColumnDefinition.ofInt("vvv").withGrouping()); - final Table tableToSave = newTable(tableDefinition, TableTools.col("vvv", data)); + final Table tableToSave = TableTools.newTable(TableTools.intCol("vvv", data)); + DataIndexer.getOrCreateDataIndex(tableToSave, "vvv"); - final String destFilename = "groupingColumnsWriteTests.parquet"; + // For a completed write, there should be two parquet files in the directory, the table data and the index + // data + final String destFilename = "indexedWriteTests.parquet"; final File destFile = new File(parentDir, destFilename); writer.writeTable(tableToSave, destFile); - String vvvIndexFilePath = ".dh_metadata/indexes/vvv/index_vvv_groupingColumnsWriteTests.parquet"; + String vvvIndexFilePath = ".dh_metadata/indexes/vvv/index_vvv_indexedWriteTests.parquet"; verifyFilesInDir(parentDir, new String[] {destFilename}, Map.of("vvv", new String[] {vvvIndexFilePath})); checkSingleTable(tableToSave, destFile); @@ -1127,8 +1198,7 @@ public void groupingColumnsBasicWriteTestsImpl(TestParquetTableWriter writer) { assertTrue(metadataString.contains(vvvIndexFilePath)); // Write another table but this write should fail - final TableDefinition badTableDefinition = TableDefinition.of(ColumnDefinition.ofInt("www").withGrouping()); - final Table badTable = newTable(badTableDefinition, TableTools.col("www", data)) + final Table badTable = TableTools.newTable(TableTools.intCol("www", data)) .updateView("InputString = ii % 2 == 0 ? Long.toString(ii) : null", "A=InputString.charAt(0)"); try { writer.writeTable(badTable, destFile); @@ -1152,7 +1222,7 @@ public void legacyGroupingFileReadTest() { // Read the legacy file and verify that grouping column is read correctly final Table fromDisk = readParquetFileFromGitLFS(destFile); final String groupingColName = "gcol"; - assertTrue(fromDisk.getDefinition().getColumn(groupingColName).isGrouping()); + assertTrue(DataIndexer.hasDataIndex(fromDisk, groupingColName)); // Verify that the key-value metadata in the file has the correct legacy grouping file name final ParquetTableLocationKey tableLocationKey = @@ -1162,14 +1232,23 @@ public void legacyGroupingFileReadTest() { assertTrue(metadataString.contains(groupingFileName)); // Following is how this file was generated, so verify the table read from disk against this - Integer data[] = new Integer[500 * 4]; + final int[] data = new int[500 * 4]; for (int i = 0; i < data.length; i++) { data[i] = i / 4; } - final TableDefinition tableDefinition = - TableDefinition.of(ColumnDefinition.ofInt(groupingColName).withGrouping()); - final Table table = newTable(tableDefinition, TableTools.col(groupingColName, data)); + final Table table = newTable(TableTools.intCol(groupingColName, data)); assertTableEquals(fromDisk, table); + + // Read the legacy grouping table. + final DataIndex fromDiskIndex = DataIndexer.getDataIndex(fromDisk, groupingColName); + final Table fromDiskIndexTable = Objects.requireNonNull(fromDiskIndex).table(); + + // Create a dynamic index from the table. + final DataIndex tableIndex = DataIndexer.getOrCreateDataIndex(table, groupingColName); + final Table tableIndexTable = Objects.requireNonNull(tableIndex).table(); + + // Validate the loaded and created index match. + assertTableEquals(fromDiskIndexTable, tableIndexTable); } @Test @@ -1179,12 +1258,12 @@ public void parquetDirectoryWithDotFilesTest() throws IOException { parentDir.mkdir(); assertTrue(parentDir.exists() && parentDir.isDirectory() && parentDir.list().length == 0); - Integer data[] = new Integer[500 * 4]; + int[] data = new int[500 * 4]; for (int i = 0; i < data.length; i++) { data[i] = i / 4; } - final TableDefinition tableDefinition = TableDefinition.of(ColumnDefinition.ofInt("vvv").withGrouping()); - final Table tableToSave = newTable(tableDefinition, TableTools.col("vvv", data)); + final Table tableToSave = newTable(TableTools.intCol("vvv", data)); + DataIndexer.getOrCreateDataIndex(tableToSave, "vvv"); final String destFilename = "data.parquet"; final File destFile = new File(parentDir, destFilename); @@ -1253,26 +1332,27 @@ public void partitionedParquetWithDotFilesTest() throws IOException { } /** - * These are tests for writing multiple parquet tables with grouping columns. + * These are tests for writing multiple parquet tables with indexes. */ @Test - public void writeMultiTableGroupingColumnTest() { + public void writeMultiTableIndexTest() { // Create an empty parent directory final File parentDir = new File(rootFile, "tempDir"); parentDir.mkdir(); - Integer data[] = new Integer[500 * 4]; + int[] data = new int[500 * 4]; for (int i = 0; i < data.length; i++) { data[i] = i / 4; } - final TableDefinition tableDefinition = TableDefinition.of(ColumnDefinition.ofInt("vvv").withGrouping()); - final Table firstTable = newTable(tableDefinition, TableTools.col("vvv", data)); + final Table firstTable = TableTools.newTable(TableTools.intCol("vvv", data)); final String firstFilename = "firstTable.parquet"; final File firstDestFile = new File(parentDir, firstFilename); + DataIndexer.getOrCreateDataIndex(firstTable, "vvv"); - final Table secondTable = newTable(tableDefinition, TableTools.col("vvv", data)); + final Table secondTable = newTable(TableTools.intCol("vvv", data)); final String secondFilename = "secondTable.parquet"; final File secondDestFile = new File(parentDir, secondFilename); + DataIndexer.getOrCreateDataIndex(secondTable, "vvv"); Table[] tablesToSave = new Table[] {firstTable, secondTable}; File[] destFiles = new File[] {firstDestFile, secondDestFile}; @@ -1299,36 +1379,38 @@ public void writeMultiTableGroupingColumnTest() { } @Test - public void groupingColumnsOverwritingTests() { - groupingColumnsOverwritingTestsImpl(SINGLE_WRITER); - groupingColumnsOverwritingTestsImpl(MULTI_WRITER); + public void indexOverwritingTests() { + indexOverwritingTestsImpl(SINGLE_WRITER); + indexOverwritingTestsImpl(MULTI_WRITER); } - public void groupingColumnsOverwritingTestsImpl(TestParquetTableWriter writer) { + private void indexOverwritingTestsImpl(TestParquetTableWriter writer) { // Create an empty parent directory final File parentDir = new File(rootFile, "tempDir"); parentDir.mkdir(); assertTrue(parentDir.exists() && parentDir.isDirectory() && parentDir.list().length == 0); - Integer data[] = new Integer[500 * 4]; + int[] data = new int[500 * 4]; for (int i = 0; i < data.length; i++) { data[i] = i / 4; } - final TableDefinition tableDefinition = TableDefinition.of(ColumnDefinition.ofInt("vvv").withGrouping()); - final Table tableToSave = newTable(tableDefinition, TableTools.col("vvv", data)); + final Table tableToSave = newTable(TableTools.intCol("vvv", data)); + DataIndexer.getOrCreateDataIndex(tableToSave, "vvv"); - final String destFilename = "groupingColumnsWriteTests.parquet"; + final String destFilename = "indexWriteTests.parquet"; final File destFile = new File(parentDir, destFilename); writer.writeTable(tableToSave, destFile); - String vvvIndexFilePath = ".dh_metadata/indexes/vvv/index_vvv_groupingColumnsWriteTests.parquet"; + String vvvIndexFilePath = ".dh_metadata/indexes/vvv/index_vvv_indexWriteTests.parquet"; + + // Write a new table successfully at the same position with different indexes + Table anotherTableToSave = newTable(TableTools.intCol("xxx", data)); + + DataIndexer.getOrCreateDataIndex(anotherTableToSave, "xxx"); - // Write a new table successfully at the same position with different grouping columns - final TableDefinition anotherTableDefinition = TableDefinition.of(ColumnDefinition.ofInt("xxx").withGrouping()); - Table anotherTableToSave = newTable(anotherTableDefinition, TableTools.col("xxx", data)); writer.writeTable(anotherTableToSave, destFile); - final String xxxIndexFilePath = ".dh_metadata/indexes/xxx/index_xxx_groupingColumnsWriteTests.parquet"; + final String xxxIndexFilePath = ".dh_metadata/indexes/xxx/index_xxx_indexWriteTests.parquet"; - // The directory now should contain the updated table, its grouping file for column xxx, and old grouping file + // The directory now should contain the updated table, its index file for column xxx, and old index file // for column vvv verifyFilesInDir(parentDir, new String[] {destFilename}, Map.of("vvv", new String[] {vvvIndexFilePath}, @@ -1344,7 +1426,7 @@ public void groupingColumnsOverwritingTestsImpl(TestParquetTableWriter writer) { // Overwrite the table writer.writeTable(anotherTableToSave, destFile); - // The directory should still contain the updated table, its grouping file for column xxx, and old grouping file + // The directory should still contain the updated table, its index file for column xxx, and old index file // for column vvv final File xxxIndexFile = new File(parentDir, xxxIndexFilePath); final File backupXXXIndexFile = ParquetTools.getBackupFile(xxxIndexFile); @@ -1399,7 +1481,7 @@ public void readModifyWriteTests() { readModifyWriteTestsImpl(MULTI_WRITER); } - public void readModifyWriteTestsImpl(TestParquetTableWriter writer) { + private void readModifyWriteTestsImpl(TestParquetTableWriter writer) { // Write a table to parquet file and read it back final Table tableToSave = TableTools.emptyTable(5).update("A=(int)i", "B=(long)i", "C=(double)i"); final String filename = "readModifyWriteTests.parquet"; @@ -1519,8 +1601,7 @@ private static ColumnChunkMetaData overflowingStringsTestHelper(final Collection checkSingleTable(stringTable, dest); ParquetMetadata metadata = new ParquetTableLocationKey(dest, 0, null, ParquetInstructions.EMPTY).getMetadata(); - ColumnChunkMetaData columnMetadata = metadata.getBlocks().get(0).getColumns().get(0); - return columnMetadata; + return metadata.getBlocks().get(0).getColumns().get(0); } @Test @@ -2082,7 +2163,7 @@ private void assertTableStatistics(Table inputTable, File dest) { new SerialByteColumnIterator( ReinterpretUtils.booleanToByteSource((ColumnSource) columnSource), inputTable.getRowSet()), - (Statistics) statistics); + (Statistics) statistics); } else if (csType == Boolean[].class) { assertBooleanArrayColumnStatistics( new SerialObjectColumnIterator<>( @@ -2260,7 +2341,7 @@ private void assertTableStatistics(Table inputTable, File dest) { } // region Column Statistics Assertions - private void assertBooleanColumnStatistics(SerialByteColumnIterator iterator, Statistics statistics) { + private void assertBooleanColumnStatistics(SerialByteColumnIterator iterator, Statistics statistics) { MutableLong itemCount = new MutableLong(0); MutableLong nullCount = new MutableLong(0); MutableInt min = new MutableInt(NULL_BYTE); diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetGrouping.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetGrouping.java deleted file mode 100644 index bfe4b05ebfa..00000000000 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetGrouping.java +++ /dev/null @@ -1,58 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.parquet.table; - -import io.deephaven.base.FileUtils; -import io.deephaven.engine.table.ColumnDefinition; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; -import io.deephaven.engine.util.TableTools; -import io.deephaven.engine.rowset.RowSetFactory; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.util.Map; - -public class TestParquetGrouping extends RefreshingTableTestCase { - - public void testOverflow() throws IOException { - // TODO: Figure out why this is called testOverflow - final File directory = Files.createTempDirectory("testOverflow").toFile(); - - try { - directory.mkdirs(); - - Integer data[] = new Integer[80_000 * 4]; - for (int i = 0; i < data.length; i++) { - data[i] = i / 4; - } - - final TableDefinition tableDefinition = TableDefinition.of(ColumnDefinition.ofInt("v").withGrouping()); - final Table table = TableTools.newTable(tableDefinition, TableTools.col("v", data)); - final ParquetInstructions instructions = ParquetInstructions.builder() - .addColumnNameMapping("V", "v") - .build(); - final File dest = new File(directory, "testOverflow.parquet"); - ParquetTools.writeTable(table, dest, tableDefinition, instructions); - - final Table tableR = ParquetTools.readTable(dest); - assertEquals(data.length, tableR.size()); - assertNotNull(tableR.getColumnSource("V").getGroupToRange()); - assertEquals(80_000 * 4, tableR.getRowSet().size()); - assertEquals(80_000, tableR.getColumnSource("V").getGroupToRange().size()); - assertEquals(80_000, tableR.getColumnSource("V").getValuesMapping(tableR.getRowSet()).size()); - assertEquals(80_000, tableR.getColumnSource("V") - .getValuesMapping(tableR.getRowSet().subSetByPositionRange(0, tableR.size())).size()); - final Map mapper = tableR.getColumnSource("V").getGroupToRange(); - for (int i = 0; i < data.length / 4; i++) { - assertEquals(mapper.get(i), RowSetFactory.fromRange(i * 4, i * 4 + 3)); - } - } finally { - FileUtils.deleteRecursively(directory); - } - - } -} diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetIndexing.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetIndexing.java new file mode 100644 index 00000000000..523f8e2fd43 --- /dev/null +++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetIndexing.java @@ -0,0 +1,75 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.parquet.table; + +import io.deephaven.base.FileUtils; +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.indexer.DataIndexer; +import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; +import io.deephaven.engine.util.TableTools; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; + +public class TestParquetIndexing extends RefreshingTableTestCase { + + public void testOverflow() throws IOException { + // TODO: Figure out why this is called testOverflow + final File directory = Files.createTempDirectory("testOverflow").toFile(); + + try { + directory.mkdirs(); + + final int[] data = new int[80_000 * 4]; + for (int i = 0; i < data.length; i++) { + data[i] = i / 4; + } + + final Table table = TableTools.newTable(TableTools.intCol("v", data)); + DataIndexer.getOrCreateDataIndex(table, "v"); + + final ParquetInstructions instructions = ParquetInstructions.builder() + .addColumnNameMapping("V", "v") + .build(); + final File dest = new File(directory, "testOverflow.parquet"); + ParquetTools.writeTable(table, dest, instructions); + + final Table tableR = ParquetTools.readTable(dest); + assertEquals(data.length, tableR.size()); + final DataIndex dataIndex = DataIndexer.getDataIndex(tableR, "V"); + Assert.neqNull(dataIndex, "dataIndex"); + final Table allGroupingTable = dataIndex.table(); + + assertNotNull(allGroupingTable); + assertEquals(80_000 * 4, tableR.getRowSet().size()); + assertEquals(80_000, allGroupingTable.size()); + + final DataIndex.RowKeyLookup rowKeyLookup = dataIndex.rowKeyLookup(); + + final ColumnSource rowSetColumnSource = dataIndex.rowSetColumn(); + + for (int i = 0; i < data.length / 4; i++) { + final long key = rowKeyLookup.apply(i, false); + assertEquals(rowSetColumnSource.get(key), RowSetFactory.fromRange(i * 4, i * 4 + 3)); + } + + // Clamp the index rowset and assert it is still correct. + Table clampedTable = dataIndex.transform(DataIndexTransformer.builder() + .intersectRowSet(tableR.getRowSet()) + .build()).table(); + assertEquals(80_000, clampedTable.size()); + + clampedTable = dataIndex.transform(DataIndexTransformer.builder() + .intersectRowSet(tableR.getRowSet().subSetByPositionRange(0, tableR.size())) + .build()).table(); + assertEquals(80_000, clampedTable.size()); + } finally { + FileUtils.deleteRecursively(directory); + } + } +} diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetTools.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetTools.java index a8fd6d885d3..db959e8483e 100644 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetTools.java +++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/TestParquetTools.java @@ -12,6 +12,7 @@ import io.deephaven.engine.table.impl.DataAccessHelpers; import io.deephaven.engine.table.impl.InMemoryTable; import io.deephaven.engine.table.impl.UncoalescedTable; +import io.deephaven.engine.table.impl.indexer.DataIndexer; import io.deephaven.engine.table.impl.locations.TableDataException; import io.deephaven.engine.testutil.junit4.EngineCleanup; import io.deephaven.engine.util.TableTools; @@ -38,6 +39,7 @@ import static io.deephaven.engine.testutil.TstUtils.assertTableEquals; import static io.deephaven.engine.testutil.TstUtils.tableRangesAreEqual; import static io.deephaven.engine.util.TableTools.*; +import static org.junit.Assert.assertTrue; /** * Tests for {@link ParquetTools}. @@ -165,13 +167,16 @@ public void testWriteTable() { test = TableTools.newTable(TableDefinition.of( ColumnDefinition.ofInt("anInt"), - ColumnDefinition.ofString("aString").withGrouping()), + ColumnDefinition.ofString("aString")), col("anInt", 1, 2, 3), col("aString", "ab", "ab", "bc")); + + DataIndexer.getOrCreateDataIndex(test, "aString"); path = testRoot + File.separator + "Table4.parquet"; ParquetTools.writeTable(test, path); + test2 = ParquetTools.readTable(new File(path)); - TestCase.assertNotNull(test2.getColumnSource("aString").getGroupToRange()); + assertTrue(DataIndexer.hasDataIndex(test2, "aString")); test2.close(); } diff --git a/py/server/deephaven/column.py b/py/server/deephaven/column.py index 7bdd38ce417..bbb5f5008b9 100644 --- a/py/server/deephaven/column.py +++ b/py/server/deephaven/column.py @@ -25,8 +25,6 @@ class ColumnType(Enum): NORMAL = _JColumnDefinitionType.Normal """ A regular column. """ - GROUPING = _JColumnDefinitionType.Grouping - """ A grouping column. """ PARTITIONING = _JColumnDefinitionType.Partitioning """ A partitioning column. """ diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateRegionsAndRegionedSources.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateRegionsAndRegionedSources.java index a149934f24d..a708a4857ed 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateRegionsAndRegionedSources.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateRegionsAndRegionedSources.java @@ -21,8 +21,8 @@ import static io.deephaven.replication.ReplicationUtils.*; /** - * Code generation for basic RegionedColumnSource implementations as well as well as the primary region interfaces for - * some primitive types. + * Code generation for basic RegionedColumnSource implementations as well as the primary region interfaces for some + * primitive types. */ public class ReplicateRegionsAndRegionedSources { private static final String TASK = "replicateRegionsAndRegionedSources"; diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourceAndChunkTests.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourceAndChunkTests.java index a76c87c05e2..86d1ed0ecda 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourceAndChunkTests.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourceAndChunkTests.java @@ -248,8 +248,6 @@ private static void fixupObjectTestSource(String objectPath) throws IOException } lines = replaceRegion(lines, "chunk add", Arrays.asList( " public synchronized void add(final RowSet rowSet, Chunk vs) {\n" + - " setGroupToRange(null);\n" + - "\n" + " if (rowSet.size() != vs.size()) {\n" + " throw new IllegalArgumentException(\"rowSet=\" + rowSet + \", data size=\" + vs.size());\n" +