From f15c8ff6fef10c0ad2e0455dc4485b8a72eb8138 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Fri, 4 Aug 2023 08:07:03 -0700 Subject: [PATCH] Add `Multijoin` feature to community (#4170) * Initial commit, added major chunk-hashing files, now compiles. * Added tests and fillChunk fixes. * MultiJoin static OA Hashing implemented and tests passing. * MultiJoin incremental OA Hashing implemented and tests passing. --- .../engine/table/MultiJoinFactory.java | 97 +++ .../engine/table/MultiJoinInput.java | 116 +++ .../engine/table/MultiJoinTable.java | 19 + engine/table/build.gradle | 2 + .../impl/MultiJoinModifiedSlotTracker.java | 236 ++++++ .../table/impl/MultiJoinStateManager.java | 51 ++ .../table/impl/MultiJoinTableCreatorImpl.java | 35 + .../engine/table/impl/MultiJoinTableImpl.java | 710 ++++++++++++++++ .../table/impl/by/typed/HasherConfig.java | 28 +- .../impl/by/typed/TypedHasherFactory.java | 110 ++- ...ementalMultiJoinStateManagerTypedBase.java | 542 ++++++++++++ .../StaticMultiJoinStateManagerTypedBase.java | 214 +++++ .../impl/multijoin/TypedMultiJoinFactory.java | 151 ++++ .../gen/IncrementalMultiJoinHasherByte.java | 346 ++++++++ .../gen/IncrementalMultiJoinHasherChar.java | 346 ++++++++ .../gen/IncrementalMultiJoinHasherDouble.java | 346 ++++++++ .../gen/IncrementalMultiJoinHasherFloat.java | 346 ++++++++ .../gen/IncrementalMultiJoinHasherInt.java | 346 ++++++++ .../gen/IncrementalMultiJoinHasherLong.java | 345 ++++++++ .../gen/IncrementalMultiJoinHasherObject.java | 348 ++++++++ .../gen/IncrementalMultiJoinHasherShort.java | 346 ++++++++ .../incopen/gen/TypedHashDispatcher.java | 44 + .../gen/StaticMultiJoinHasherByte.java | 106 +++ .../gen/StaticMultiJoinHasherChar.java | 106 +++ .../gen/StaticMultiJoinHasherDouble.java | 106 +++ .../gen/StaticMultiJoinHasherFloat.java | 106 +++ .../gen/StaticMultiJoinHasherInt.java | 106 +++ .../gen/StaticMultiJoinHasherLong.java | 105 +++ .../gen/StaticMultiJoinHasherObject.java | 107 +++ .../gen/StaticMultiJoinHasherShort.java | 106 +++ .../staticopen/gen/TypedHashDispatcher.java | 44 + .../impl/sources/BooleanArraySource.java | 18 +- .../table/impl/sources/ByteArraySource.java | 81 +- .../impl/sources/CharacterArraySource.java | 81 +- .../table/impl/sources/DoubleArraySource.java | 81 +- .../table/impl/sources/FloatArraySource.java | 81 +- .../impl/sources/IntegerArraySource.java | 81 +- .../table/impl/sources/LongArraySource.java | 158 +++- .../table/impl/sources/ObjectArraySource.java | 161 +++- .../table/impl/sources/ShortArraySource.java | 81 +- .../table/impl/QueryTableMultiJoinTest.java | 781 ++++++++++++++++++ .../impl/sources/TestBooleanArraySource.java | 12 +- .../impl/sources/TestByteArraySource.java | 12 +- .../sources/TestCharacterArraySource.java | 12 +- .../impl/sources/TestDoubleArraySource.java | 12 +- .../impl/sources/TestFloatArraySource.java | 12 +- .../impl/sources/TestIntegerArraySource.java | 12 +- .../impl/sources/TestLongArraySource.java | 12 +- .../impl/sources/TestObjectArraySource.java | 10 +- .../impl/sources/TestShortArraySource.java | 12 +- .../replicators/ReplicateTypedHashers.java | 4 + .../ReplicateSourceAndChunkTests.java | 2 + .../java/io/deephaven/api/JoinAddition.java | 5 + 53 files changed, 7618 insertions(+), 119 deletions(-) create mode 100644 engine/api/src/main/java/io/deephaven/engine/table/MultiJoinFactory.java create mode 100644 engine/api/src/main/java/io/deephaven/engine/table/MultiJoinInput.java create mode 100644 engine/api/src/main/java/io/deephaven/engine/table/MultiJoinTable.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinModifiedSlotTracker.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinStateManager.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinTableCreatorImpl.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinTableImpl.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/IncrementalMultiJoinStateManagerTypedBase.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/StaticMultiJoinStateManagerTypedBase.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/TypedMultiJoinFactory.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherByte.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherChar.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherDouble.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherFloat.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherInt.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherLong.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherObject.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherShort.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/TypedHashDispatcher.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherByte.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherChar.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherDouble.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherFloat.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherInt.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherLong.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherObject.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherShort.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/TypedHashDispatcher.java create mode 100644 engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableMultiJoinTest.java diff --git a/engine/api/src/main/java/io/deephaven/engine/table/MultiJoinFactory.java b/engine/api/src/main/java/io/deephaven/engine/table/MultiJoinFactory.java new file mode 100644 index 00000000000..c706d607da1 --- /dev/null +++ b/engine/api/src/main/java/io/deephaven/engine/table/MultiJoinFactory.java @@ -0,0 +1,97 @@ +package io.deephaven.engine.table; + +import io.deephaven.api.JoinMatch; +import org.jetbrains.annotations.NotNull; + +import java.util.ServiceLoader; + +/** + *

+ * Join unique rows from a set of tables onto a set of common keys. + *

+ * + *

+ * The multiJoin operation collects the set of distinct keys from the input tables, then joins at most one row per key + * from each of the input tables onto the result. Input tables need not have a matching row for each key, but they may + * not have multiple matching rows for a given key. + *

+ * + *

+ * Input tables with non-matching key column names must use the {@link JoinMatch} format to map keys to the common + * output table key column names (e.g. "OutputKey=SourceKey"). Also, individual columns to include from input tables may + * be specified and optionally renamed using {@link io.deephaven.api.JoinAddition} format (e.g. "NewCol=OldColName"). If + * no output columns are specified then every non-key column from the input table will be included in the multi-join + * output table. + *

+ * + *

+ * The multiJoin operation can be thought of as a merge of the key columns, followed by a selectDistinct and then a + * series of iterative naturalJoin operations as follows (this example has common key column names and includes all + * columns from the input tables): + *

+ * + *
{@code
+ *     private Table doIterativeMultiJoin(String [] keyColumns, List inputTables) {
+ *         final List keyTables = inputTables.stream().map(t -> t.view(keyColumns)).collect(Collectors.toList());
+ *         final Table base = TableTools.merge(keyTables).selectDistinct(keyColumns);
+ *
+ *         Table result = base;
+ *         for (int ii = 0; ii < inputTables.size(); ++ii) {
+ *             result = result.naturalJoin(inputTables.get(ii), Arrays.asList(keyColumns));
+ *         }
+ *
+ *         return result;
+ *     }
+ *     }
+ * 
+ */
+
+public class MultiJoinFactory {
+
+    /**
+     * Creator interface for runtime-supplied implementation.
+     */
+    public interface Creator {
+        MultiJoinTable of(@NotNull final MultiJoinInput... multiJoinInputs);
+    }
+
+    /**
+     * Creator provider to supply the implementation at runtime.
+     */
+    @FunctionalInterface
+    public interface CreatorProvider {
+        Creator get();
+    }
+
+    private static final class MultiJoinTableCreatorHolder {
+        private static final MultiJoinFactory.Creator creator =
+                ServiceLoader.load(MultiJoinFactory.CreatorProvider.class).iterator().next().get();
+    }
+
+    private static MultiJoinFactory.Creator multiJoinTableCreator() {
+        return MultiJoinTableCreatorHolder.creator;
+    }
+
+    /**
+     * Join tables that have common key column names; include all columns from the input tables.
+     * 

+ * + * @param keys the key column pairs in the format "Result=Source" or "ColumnInBoth" + * @param inputTables the tables to join together + * @return a MultiJoinTable with one row for each key and the corresponding row in each input table + */ + public static MultiJoinTable of(@NotNull final String[] keys, @NotNull final Table... inputTables) { + return multiJoinTableCreator().of(MultiJoinInput.from(keys, inputTables)); + } + + /** + * Perform a multiJoin for one or more tables; allows renaming of key column names and specifying individual input + * table columns to include in the final output table. + * + * @param multiJoinInputs the description of each table that contributes to the result + * @return a MultiJoinTable with one row for each key and the corresponding row in each input table + */ + public static MultiJoinTable of(@NotNull final MultiJoinInput... multiJoinInputs) { + return multiJoinTableCreator().of(multiJoinInputs); + } +} diff --git a/engine/api/src/main/java/io/deephaven/engine/table/MultiJoinInput.java b/engine/api/src/main/java/io/deephaven/engine/table/MultiJoinInput.java new file mode 100644 index 00000000000..f4c8facceb3 --- /dev/null +++ b/engine/api/src/main/java/io/deephaven/engine/table/MultiJoinInput.java @@ -0,0 +1,116 @@ +package io.deephaven.engine.table; + +import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.JoinAddition; +import io.deephaven.api.JoinMatch; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; +import org.jetbrains.annotations.NotNull; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + +/** + * An input to a multiJoin. + *

+ * The table, key columns, and columns to add are encapsulated in the join descriptor. + */ +@Immutable +@SimpleStyle +public abstract class MultiJoinInput { + /** + * Create a multiJoin table input. + * + * @param inputTable The table to include in a multiJoin + * @param columnsToMatch An array of {@link JoinMatch} specifying match conditions + * @param columnsToAdd An array of {@link JoinAddition} specifying the columns to add + */ + public static MultiJoinInput of( + @NotNull final Table inputTable, + @NotNull JoinMatch[] columnsToMatch, + @NotNull JoinAddition[] columnsToAdd) { + return ImmutableMultiJoinInput.of(inputTable, columnsToMatch, columnsToAdd); + } + + /** + * Create a multiJoin table input. + * + * @param inputTable The table to include in a multiJoin + * @param columnsToMatch A collection of {@link JoinMatch} specifying the key columns + * @param columnsToAdd A collection of {@link JoinAddition} specifying the columns to add + */ + public static MultiJoinInput of( + @NotNull final Table inputTable, + @NotNull final Collection columnsToMatch, + @NotNull final Collection columnsToAdd) { + return of(inputTable, columnsToMatch.toArray(JoinMatch[]::new), columnsToAdd.toArray(JoinAddition[]::new)); + } + + /** + * Create a multiJoin table input. + * + * @param inputTable The table to include in a multiJoin + * @param columnsToMatch The key columns, in string format (e.g. "ResultKey=SourceKey" or "KeyInBoth"). + * @param columnsToAdd The columns to add, in string format (e.g. "ResultColumn=SourceColumn" or + * "SourceColumnToAddWithSameName"); empty for all columns + */ + public static MultiJoinInput of( + @NotNull final Table inputTable, + @NotNull final String[] columnsToMatch, + @NotNull final String[] columnsToAdd) { + return of(inputTable, JoinMatch.from(columnsToMatch), JoinAddition.from(columnsToAdd)); + } + + /** + * Create a multiJoin table input. + *

+ * + * @param inputTable The table to include in a multiJoin + * @param columnsToMatch The key columns, in string format (e.g. "ResultKey=SourceKey" or "KeyInBoth"). + */ + public static MultiJoinInput of(@NotNull final Table inputTable, @NotNull final String... columnsToMatch) { + return of(inputTable, JoinMatch.from(columnsToMatch), Collections.emptyList()); + } + + /** + * Create a multiJoin table input. + * + * @param inputTable The table to include in a multiJoin + * @param columnsToMatch A comma separated list of key columns, in string format (e.g. "ResultKey=SourceKey" or + * "KeyInBoth"). + * @param columnsToAdd A comma separated list of columns to add, in string format (e.g. "ResultColumn=SourceColumn" + * or "SourceColumnToAddWithSameName"); empty for all columns + */ + public static MultiJoinInput of(@NotNull final Table inputTable, String columnsToMatch, String columnsToAdd) { + return of(inputTable, + columnsToMatch == null || columnsToMatch.isEmpty() + ? Collections.emptyList() + : JoinMatch.from(columnsToMatch), + columnsToAdd == null || columnsToAdd.isEmpty() + ? Collections.emptyList() + : JoinAddition.from(columnsToAdd)); + } + + /** + * Create an array of {@link MultiJoinInput} with common keys; includes all non-key columns as output columns. + * + * @param keys The key columns, common to all tables + * @param inputTables An array of tables to include in the output + */ + @NotNull + public static MultiJoinInput[] from(@NotNull final String[] keys, @NotNull final Table[] inputTables) { + return Arrays.stream(inputTables) + .map(t -> MultiJoinInput.of(t, keys)) + .toArray(MultiJoinInput[]::new); + } + + @Parameter + public abstract Table inputTable(); + + @Parameter + public abstract JoinMatch[] columnsToMatch(); + + @Parameter + public abstract JoinAddition[] columnsToAdd(); +} diff --git a/engine/api/src/main/java/io/deephaven/engine/table/MultiJoinTable.java b/engine/api/src/main/java/io/deephaven/engine/table/MultiJoinTable.java new file mode 100644 index 00000000000..80aa89ceb3c --- /dev/null +++ b/engine/api/src/main/java/io/deephaven/engine/table/MultiJoinTable.java @@ -0,0 +1,19 @@ +package io.deephaven.engine.table; + +import java.util.Collection; + +public interface MultiJoinTable { + /** + * Get the output {@link Table table} from this multi-join table. + * + * @return The output {@link Table table} + */ + Table table(); + + /** + * Get the key column names from this multi-join table. + * + * @return The key column names as a collection of strings + */ + Collection keyColumns(); +} diff --git a/engine/table/build.gradle b/engine/table/build.gradle index 2af2095865d..1ebc7960d11 100644 --- a/engine/table/build.gradle +++ b/engine/table/build.gradle @@ -120,6 +120,8 @@ spotless { '**/asofjoin/typed/rightincopen/gen/*.java', '**/asofjoin/typed/staticopen/gen/*.java', '**/updateby/hashing/typed/open/gen/*.java', + '**/multijoin/typed/staticopen/gen/*.java', + '**/multijoin/typed/incopen/gen/*.java', 'src/main/java/io/deephaven/engine/table/impl/SymbolTableCombiner.java', 'src/main/java/io/deephaven/libs/GroovyStaticImports.java', 'src/test/java/**/*Sample.java' diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinModifiedSlotTracker.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinModifiedSlotTracker.java new file mode 100644 index 00000000000..d120a2726e6 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinModifiedSlotTracker.java @@ -0,0 +1,236 @@ +package io.deephaven.engine.table.impl; + + +import io.deephaven.engine.table.impl.sources.IntegerArraySource; +import io.deephaven.engine.table.impl.sources.LongArraySource; + +import java.util.ArrayList; +import java.util.List; + +public class MultiJoinModifiedSlotTracker { + /** The output row that was modified. */ + private final IntegerArraySource modifiedOutputRows = new IntegerArraySource(); + /** + * We store flags, one per nibble; 16 per location in flags, contiguous for multiple tables when we have more than + * 16. + */ + private final LongArraySource flagSource = new LongArraySource(); + /** The original right values, parallel to modifiedSlots. */ + private final List originalRedirection = new ArrayList<>(); + + int numTables = 0; + int flagLocationsPerSlot = 0; + + /** + * The location that we must write to in modified slots; also if we have a pointer that falls outside the range [0, + * pointer); then we know it is invalid + */ + private long pointer; + /** How many slots we have allocated */ + private int allocated; + /** + * Each time we clear, we add an offset to our cookies, this prevents us from reading old values. Initialize to one + * to ensure newly allocated arrays cannot be interpreted as valid cookies. + */ + private long cookieGeneration = 1; + + static final long SENTINEL_UNINITIALIZED_KEY = -2; + + private static final int FLAG_BITS = 4; + private static final int FLAGS_PER_LOCATION = 16; + + public static final byte FLAG_ADD = 0x1; + public static final byte FLAG_REMOVE = 0x2; + public static final byte FLAG_MODIFY = 0x4; + public static final byte FLAG_SHIFT = 0x8; + + /** + * Remove all entries from the tracker. + */ + void clear() { + cookieGeneration += pointer; + if (cookieGeneration > Long.MAX_VALUE / 2) { + cookieGeneration = 1; + } + pointer = 0; + } + + void ensureTableCapacity(int numTables) { + while (originalRedirection.size() < numTables) { + final LongArraySource las = new LongArraySource(); + las.ensureCapacity(allocated); + originalRedirection.add(las); + } + this.numTables = numTables; + this.flagLocationsPerSlot = (numTables + FLAGS_PER_LOCATION - 1) / FLAGS_PER_LOCATION; + this.flagSource.ensureCapacity((long) allocated * flagLocationsPerSlot); + } + + /** + * Is this cookie within our valid range (greater than or equal to our generation, but less than the pointer after + * adjustment? + * + * @param cookie The cookie to check for validity + * + * @return true if the cookie is from the current generation, and references a valid row in our table + */ + private boolean isValidCookie(long cookie) { + return cookie >= cookieGeneration && getPointerFromCookie(cookie) < pointer; + } + + /** + * Get a cookie to return to the user, given a pointer value. + * + * @param pointer The pointer to convert to a cookie + * @return The cookie to return to the user + */ + private long getCookieFromPointer(long pointer) { + return cookieGeneration + pointer; + } + + /** + * Given a valid user's cookie, return the corresponding pointer. + * + * @param cookie The valid cookie + * @return The pointer into modifiedSlots + */ + private long getPointerFromCookie(long cookie) { + return cookie - cookieGeneration; + } + + /** + * Add a slot in the tracker to mark an add/remove/shift of an output row. + * + * @param outputRow The row to mark for modifications + * @param originalRedirection The redirection value before our modification + * @param flags The flags to or into our state + * + * @return The cookie for future access + */ + public long addSlot(final long cookie, final int outputRow, final int tableNumber, final long originalRedirection, + final byte flags) { + if (!isValidCookie(cookie)) { + // Create a new slot in the tracker and reset the flags and redirection. + maybeAllocateChunk(); + initializeNextTrackerSlot(outputRow, tableNumber, flags); + + for (int ii = 0; ii < this.originalRedirection.size(); ++ii) { + final LongArraySource originalRedirectionForTable = this.originalRedirection.get(ii); + // Store the original redirection for this table, reset the others. + if (ii == tableNumber) { + originalRedirectionForTable.set(pointer, originalRedirection); + } else { + originalRedirectionForTable.set(pointer, SENTINEL_UNINITIALIZED_KEY); + } + } + return getCookieFromPointer(pointer++); + } else { + // This tracker slot exists, update the flags and set the redirection for this row. + final long pointer = getPointerFromCookie(cookie); + + doFlagUpdate(tableNumber, flags, pointer); + + final LongArraySource originalRedirectionForTable = this.originalRedirection.get(tableNumber); + if (originalRedirectionForTable.getUnsafe(pointer) == SENTINEL_UNINITIALIZED_KEY) { + originalRedirectionForTable.set(pointer, originalRedirection); + } + return cookie; + } + } + + /** + * Add a slot in the tracker to mark a modification to an output row. + * + * @param outputRow The slot to add to the tracker. + * @param flags The flags to or into our state + * + * @return The cookie for future access + */ + public long modifySlot(final long cookie, final int outputRow, final int tableNumber, final byte flags) { + if (!isValidCookie(cookie)) { + // Create a new slot in the tracker and reset the flags and redirection for this row in all tables. + maybeAllocateChunk(); + initializeNextTrackerSlot(outputRow, tableNumber, flags); + for (final LongArraySource originalRedirectionForTable : this.originalRedirection) { + originalRedirectionForTable.set(pointer, SENTINEL_UNINITIALIZED_KEY); + } + return getCookieFromPointer(pointer++); + } else { + // This tracker slot exists, update the flags only. + final long pointer = getPointerFromCookie(cookie); + doFlagUpdate(tableNumber, flags, pointer); + return cookie; + } + } + + private long flagLocationForSlotAndTable(final long pointer, final int tableNumber) { + return pointer * flagLocationsPerSlot + (tableNumber / FLAGS_PER_LOCATION); + } + + private int flagShiftForTable(final int tableNumber) { + return (tableNumber % FLAGS_PER_LOCATION) * FLAG_BITS; + } + + private long setFlagInLong(final int tableNumber, final byte flag) { + return ((long) flag) << flagShiftForTable(tableNumber); + } + + private long setFlagInLong(long existingLong, int tableNumber, byte flag) { + return existingLong | setFlagInLong(tableNumber, flag); + } + + private void maybeAllocateChunk() { + if (pointer == allocated) { + allocated += JoinControl.CHUNK_SIZE; + modifiedOutputRows.ensureCapacity(allocated); + flagSource.ensureCapacity((long) allocated * flagLocationsPerSlot); + this.originalRedirection.forEach(las -> las.ensureCapacity(allocated)); + } + } + + private void initializeNextTrackerSlot(final int outputRow, final int tableNumber, final byte flags) { + modifiedOutputRows.set(pointer, outputRow); + for (int ii = 0; ii < flagLocationsPerSlot; ++ii) { + flagSource.set(pointer * flagLocationsPerSlot + ii, 0L); + } + flagSource.set(flagLocationForSlotAndTable(pointer, tableNumber), setFlagInLong(tableNumber, flags)); + } + + private void doFlagUpdate(int tableNumber, byte flags, long pointer) { + final long flagLocation = flagLocationForSlotAndTable(pointer, tableNumber); + final long existingFlagLong = flagSource.getUnsafe(flagLocation); + final long updatedFlagLong = setFlagInLong(existingFlagLong, tableNumber, flags); + flagSource.set(flagLocation, updatedFlagLong); + } + + public interface ModifiedSlotConsumer { + void accept(int outputRow, long[] previousRedirections, byte[] flags); + } + + void forAllModifiedSlots(ModifiedSlotConsumer slotConsumer) { + final long[] previousRedirections = new long[numTables]; + final byte[] flagValues = new byte[numTables]; + for (long ii = 0; ii < pointer; ++ii) { + final int outputRow = modifiedOutputRows.getInt(ii); + + int tt = 0; + // Processes fully populated longs. + for (int ff = 0; ff < flagLocationsPerSlot - 1; ++ff) { + final long flagLong = flagSource.getUnsafe((ii * flagLocationsPerSlot) + ff); + for (int jj = 0; jj < FLAGS_PER_LOCATION; ++jj) { + flagValues[tt++] = (byte) ((flagLong >> (FLAG_BITS * jj)) & ((1L << FLAG_BITS) - 1)); + } + } + // Processes the final long containing <= 16 flags. + final long flagLong = flagSource.getUnsafe((ii * flagLocationsPerSlot) + flagLocationsPerSlot - 1); + for (int jj = 0; tt < numTables; ++jj) { + flagValues[tt++] = (byte) ((flagLong >> (FLAG_BITS * jj)) & ((1L << FLAG_BITS) - 1)); + } + + for (tt = 0; tt < previousRedirections.length; ++tt) { + previousRedirections[tt] = originalRedirection.get(tt).getUnsafe(ii); + } + slotConsumer.accept(outputRow, previousRedirections, flagValues); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinStateManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinStateManager.java new file mode 100644 index 00000000000..bc7eb4b6e12 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinStateManager.java @@ -0,0 +1,51 @@ +package io.deephaven.engine.table.impl; + +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.util.RowRedirection; + +/** + * This is a common interface for the static and incremental state manager so that our bucketed MultiJoinTable system is + * capable of using them interchangeably to build the table. + */ +public interface MultiJoinStateManager { + /** + * Add the given table to this multiJoin result. + * + * @param table the table to add + * @param sources the column sources that contain the keys + * @param tableNumber the table number for which we are adding rows + */ + void build(final Table table, ColumnSource[] sources, int tableNumber); + + /** + * Get the number of rows in the result table + * + * @return the number of rows in the result table + */ + long getResultSize(); + + /** + * Get the hash table column sources for the result table. These are used as the key columns of our result. + */ + ColumnSource[] getKeyHashTableSources(); + + /** + * Get the result {@link RowRedirection row redirection} for a given table + * + * @param tableNumber the table to fetch + * @return the row redirection for the table + */ + RowRedirection getRowRedirectionForTable(int tableNumber); + + /** + * Ensure that this state manager can handle {@code numTables} tables as constituents of the multiJoin. + * + * @param numTables the number of tables that participate + */ + void ensureTableCapacity(int numTables); + + void setTargetLoadFactor(final double targetLoadFactor); + + void setMaximumLoadFactor(final double maximumLoadFactor); +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinTableCreatorImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinTableCreatorImpl.java new file mode 100644 index 00000000000..c329ecddd7b --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinTableCreatorImpl.java @@ -0,0 +1,35 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.engine.table.impl; + +import com.google.auto.service.AutoService; +import io.deephaven.engine.table.MultiJoinFactory; +import io.deephaven.engine.table.MultiJoinInput; +import io.deephaven.engine.table.MultiJoinTable; +import org.jetbrains.annotations.NotNull; + +/** + * Engine-specific implementation of {@link MultiJoinFactory.Creator}. + */ +@SuppressWarnings("unused") +public enum MultiJoinTableCreatorImpl implements MultiJoinFactory.Creator { + INSTANCE; + + @Override + public MultiJoinTable of(@NotNull MultiJoinInput... multiJoinInputs) { + if (multiJoinInputs.length == 0) { + throw new IllegalArgumentException("At least one table must be included in MultiJoinTable."); + } + return MultiJoinTableImpl.of(multiJoinInputs); + } + + @SuppressWarnings("unused") + @AutoService(MultiJoinFactory.CreatorProvider.class) + public static final class ProviderImpl implements MultiJoinFactory.CreatorProvider { + @Override + public MultiJoinFactory.Creator get() { + return INSTANCE; + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinTableImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinTableImpl.java new file mode 100644 index 00000000000..95f2f84cf4a --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/MultiJoinTableImpl.java @@ -0,0 +1,710 @@ +package io.deephaven.engine.table.impl; + +import gnu.trove.map.hash.TObjectIntHashMap; +import io.deephaven.api.ColumnName; +import io.deephaven.api.JoinAddition; +import io.deephaven.api.JoinMatch; +import io.deephaven.engine.context.ExecutionContext; +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.*; +import io.deephaven.engine.table.impl.by.BitmapRandomBuilder; +import io.deephaven.engine.table.impl.by.typed.TypedHasherFactory; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; +import io.deephaven.engine.table.impl.sources.RedirectedColumnSource; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.engine.table.impl.util.RowRedirection; +import io.deephaven.engine.table.impl.util.SingleValueRowRedirection; +import io.deephaven.engine.table.impl.util.WritableSingleValueRowRedirection; +import io.deephaven.engine.updategraph.NotificationQueue; +import io.deephaven.engine.updategraph.UpdateGraph; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.annotations.VisibleForTesting; +import org.jetbrains.annotations.NotNull; + +import java.util.*; +import java.util.stream.Collectors; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker.*; + +public class MultiJoinTableImpl implements MultiJoinTable { + + private static final int KEY_COLUMN_SENTINEL = -2; + private final Table table; + + private final List keyColumns; + + private static class MultiJoinInputHelper { + Table table; + + /** The output column keys in the order provided by the MutiJoinInput */ + final String[] keyColumnNames; + /** The input column keys in the order provided by the MutiJoinInput */ + final String[] originalKeyColumnNames; + /** The output non-key columns in the order provided by the MutiJoinInput */ + final String[] addColumnNames; + /** The input non-key columns in the order provided by the MutiJoinInput */ + final String[] originalAddColumnNames; + + final Map> keySourceMap; + final Map> originalKeySourceMap; + + final JoinAddition[] columnsToAdd; + + MultiJoinInputHelper(@NotNull MultiJoinInput input) { + table = input.inputTable().coalesce(); + + final int matchCount = input.columnsToMatch().length; + + // Create the ordered list of input as well as the deterministic order from the hashmap. + keyColumnNames = new String[matchCount]; + originalKeyColumnNames = new String[matchCount]; + + keySourceMap = new HashMap<>(matchCount); + originalKeySourceMap = new HashMap<>(matchCount); + + for (int ii = 0; ii < matchCount; ii++) { + final JoinMatch jm = input.columnsToMatch()[ii]; + final String left = jm.left().name(); + final String right = jm.right().name(); + + keyColumnNames[ii] = left; + originalKeyColumnNames[ii] = right; + + keySourceMap.put(left, ReinterpretUtils.maybeConvertToPrimitive(table.getColumnSource(right))); + originalKeySourceMap.put(left, table.getColumnSource(right)); + } + + // Create the lists of addition columns (or use every non-key column if unspecified). + if (input.columnsToAdd().length == 0) { + // Compare against the input table key column names (not output). + final Set keys = new HashSet<>(Arrays.asList(originalKeyColumnNames)); + // create them on the fly from the table + columnsToAdd = input.inputTable().getDefinition().getColumnNames().stream() + .filter(cn -> !keys.contains(cn)).map(ColumnName::of) + .toArray(JoinAddition[]::new); + } else { + columnsToAdd = input.columnsToAdd(); + } + + addColumnNames = new String[columnsToAdd.length]; + originalAddColumnNames = new String[columnsToAdd.length]; + for (int ii = 0; ii < columnsToAdd.length; ii++) { + final JoinAddition ja = columnsToAdd[ii]; + addColumnNames[ii] = ja.newColumn().name(); + originalAddColumnNames[ii] = ja.existingColumn().name(); + } + } + + void assertCompatible(@NotNull MultiJoinTableImpl.MultiJoinInputHelper inputHelper, int tableNumber) { + // Verify the key column names. + if (!keySourceMap.keySet().equals(inputHelper.keySourceMap.keySet())) { + throw new IllegalArgumentException( + String.format( + "Key column mismatch for table %d, first table has key columns=%s, this table has %s", + tableNumber, inputHelper.keySourceMap.keySet(), keySourceMap.keySet())); + } + // Verify matching column types. + final String[] keys = inputHelper.keyColumnNames; + final Collection> currentColumnTypes = + Arrays.stream(keySources(keys)).map(ColumnSource::getType).collect(Collectors.toSet()); + final Collection> expectedColumnTypes = + Arrays.stream(inputHelper.keySources(keys)).map(ColumnSource::getType).collect(Collectors.toSet()); + if (!currentColumnTypes.equals(expectedColumnTypes)) { + throw new IllegalArgumentException( + String.format( + "Key column type mismatch for table %d, first table has key column types=%s, this table has %s", + tableNumber, expectedColumnTypes, currentColumnTypes)); + } + // Verify matching column component types. + final Collection> currentComponentTypes = + Arrays.stream(keySources(keys)).map(ColumnSource::getComponentType).collect(Collectors.toSet()); + final Collection> expectedComponentTypes = Arrays.stream(inputHelper.keySources(keys)) + .map(ColumnSource::getComponentType).collect(Collectors.toSet()); + if (!currentComponentTypes.equals(expectedComponentTypes)) { + throw new IllegalArgumentException(String.format( + "Key column component type mismatch for table %d, first table has key column component types=%s, this table has %s", + tableNumber, expectedComponentTypes, currentComponentTypes)); + } + } + + ColumnSource[] keySources(final String[] columns) { + return Arrays.stream(columns).map(keySourceMap::get).toArray(ColumnSource[]::new); + } + + ColumnSource[] keySources() { + return keySources(keyColumnNames); + } + + ColumnSource[] originalKeySources(final String[] columns) { + return Arrays.stream(columns).map(originalKeySourceMap::get).toArray(ColumnSource[]::new); + } + + ColumnSource[] originalKeySources() { + return originalKeySources(keyColumnNames); + } + } + + @VisibleForTesting + static MultiJoinTableImpl of(@NotNull final JoinControl joinControl, + @NotNull final MultiJoinInput... multiJoinInputs) { + + final Table[] tables = Arrays.stream(multiJoinInputs).map(MultiJoinInput::inputTable).toArray(Table[]::new); + final UpdateGraph updateGraph = tables[0].getUpdateGraph(tables); + if (updateGraph != null) { + updateGraph.checkInitiateSerialTableOperation(); + } + try (final SafeCloseable ignored = ExecutionContext.getContext().withUpdateGraph(updateGraph).open()) { + return QueryPerformanceRecorder.withNugget("multiJoin", + () -> new MultiJoinTableImpl(joinControl, multiJoinInputs)); + } + } + + static MultiJoinTableImpl of(@NotNull final MultiJoinInput... multiJoinInputs) { + return of(new JoinControl(), multiJoinInputs); + } + + /** + * Get the output {@link Table table} from this multi-join table. + * + * @return The output {@link Table table} + */ + public Table table() { + return table; + } + + @Override + public Collection keyColumns() { + return keyColumns; + } + + private MultiJoinTableImpl(@NotNull final JoinControl joinControl, + @NotNull final MultiJoinInput... multiJoinInputs) { + keyColumns = new ArrayList<>(); + + // Create the join input helpers we'll use during the join creation phase. + final MultiJoinInputHelper[] joinInputHelpers = + Arrays.stream(multiJoinInputs).map(MultiJoinInputHelper::new).toArray(MultiJoinInputHelper[]::new); + final TObjectIntHashMap usedColumns = + new TObjectIntHashMap<>(joinInputHelpers[0].columnsToAdd.length, 0.5f, -1); + + for (String keyColName : joinInputHelpers[0].keyColumnNames) { + keyColumns.add(keyColName); + usedColumns.put(keyColName, KEY_COLUMN_SENTINEL); + } + + for (int ii = 1; ii < joinInputHelpers.length; ++ii) { + // Verify this input table is compatible with the first table. + joinInputHelpers[ii].assertCompatible(joinInputHelpers[0], ii); + } + + // Verify the non-key output columns do not conflict. + for (int ii = 0; ii < joinInputHelpers.length; ++ii) { + MultiJoinInputHelper inputHelper = joinInputHelpers[ii]; + for (String columnName : inputHelper.addColumnNames) { + final int previouslyUsed = usedColumns.put(columnName, ii); + if (previouslyUsed != usedColumns.getNoEntryValue()) { + throw new IllegalArgumentException(String.format("Column %s defined in table %s and table %d", + columnName, + previouslyUsed == KEY_COLUMN_SENTINEL ? "key columns" : Integer.toString(previouslyUsed), + ii)); + } + } + } + + if (multiJoinInputs[0].columnsToMatch().length == 0) { + table = doMultiJoinZeroKey(joinInputHelpers); + return; + } + table = bucketedMultiJoin(joinControl, joinInputHelpers); + } + + private Table bucketedMultiJoin(@NotNull final JoinControl joinControl, + @NotNull final MultiJoinInputHelper[] joinInputHelpers) { + + final MultiJoinStateManager stateManager; + final String[] firstKeyColumnNames = joinInputHelpers[0].keyColumnNames; + + // If any tables are refreshing, we must use a refreshing JoinManager. + final boolean refreshing = Arrays.stream(joinInputHelpers).anyMatch(ih -> ih.table.isRefreshing()); + if (refreshing) { + stateManager = TypedHasherFactory.make(IncrementalMultiJoinStateManagerTypedBase.class, + joinInputHelpers[0].keySources(), + joinInputHelpers[0].originalKeySources(), + joinControl.initialBuildSize(), joinControl.getMaximumLoadFactor(), + joinControl.getTargetLoadFactor()); + } else { + stateManager = TypedHasherFactory.make(StaticMultiJoinStateManagerTypedBase.class, + joinInputHelpers[0].keySources(), + joinInputHelpers[0].originalKeySources(), + joinControl.initialBuildSize(), joinControl.getMaximumLoadFactor(), + joinControl.getTargetLoadFactor()); + } + stateManager.setMaximumLoadFactor(joinControl.getMaximumLoadFactor()); + stateManager.setTargetLoadFactor(joinControl.getTargetLoadFactor()); + stateManager.ensureTableCapacity(joinInputHelpers.length); + + for (int tableNumber = 0; tableNumber < joinInputHelpers.length; ++tableNumber) { + stateManager.build( + joinInputHelpers[tableNumber].table, + joinInputHelpers[tableNumber].keySources(firstKeyColumnNames), + tableNumber); + } + + final Map> resultSources = new LinkedHashMap<>(); + + final ColumnSource[] keyHashTableSources = stateManager.getKeyHashTableSources(); + final ColumnSource[] originalColumns = joinInputHelpers[0].originalKeySources(); + + // We are careful to add the output key columns in the order of the first table input. + for (int cc = 0; cc < keyColumns.size(); ++cc) { + if (originalColumns[cc].getType() != keyHashTableSources[cc].getType()) { + resultSources.put(keyColumns.get(cc), + ReinterpretUtils.convertToOriginalType(originalColumns[cc], keyHashTableSources[cc])); + } else { + resultSources.put(keyColumns.get(cc), keyHashTableSources[cc]); + } + } + + for (int tableNumber = 0; tableNumber < joinInputHelpers.length; ++tableNumber) { + final RowRedirection rowRedirection = stateManager.getRowRedirectionForTable(tableNumber); + if (refreshing) { + ((IncrementalMultiJoinStateManagerTypedBase) stateManager) + .startTrackingPrevRedirectionValues(tableNumber); + } + final MultiJoinInputHelper inputHelper = joinInputHelpers[tableNumber]; + for (final JoinAddition ja : inputHelper.columnsToAdd) { + resultSources.put(ja.newColumn().name(), RedirectedColumnSource.alwaysRedirect(rowRedirection, + inputHelper.table.getColumnSource(ja.existingColumn().name()))); + } + } + + final QueryTable result = + new QueryTable(RowSetFactory.flat(stateManager.getResultSize()).toTracking(), resultSources); + + if (refreshing) { + final ModifiedColumnSet[] resultModifiedColumnSet = new ModifiedColumnSet[joinInputHelpers.length]; + final List listenerRecorders = new ArrayList<>(); + + final MergedListener mergedListener = new MultiJoinMergedListener( + (IncrementalMultiJoinStateManagerTypedBase) stateManager, + listenerRecorders, + Collections.emptyList(), + "multiJoin(" + keyColumns + ")", + result, + resultModifiedColumnSet); + + for (int ii = 0; ii < joinInputHelpers.length; ++ii) { + final MultiJoinInputHelper inputHelper = joinInputHelpers[ii]; + if (inputHelper.table.isRefreshing()) { + final QueryTable input = (QueryTable) inputHelper.table; + final ColumnSource[] keySources = inputHelper.keySources(firstKeyColumnNames); + + final ModifiedColumnSet sourceKeyModifiedColumnSet = + input.newModifiedColumnSet(inputHelper.originalKeyColumnNames); + final ModifiedColumnSet sourceAdditionModifiedColumnSet = + input.newModifiedColumnSet(inputHelper.originalAddColumnNames); + + resultModifiedColumnSet[ii] = + result.newModifiedColumnSet(inputHelper.addColumnNames); + final MatchPair[] pairs = MatchPair.fromAddition(Arrays.asList(inputHelper.columnsToAdd)); + final ModifiedColumnSet.Transformer transformer = + input.newModifiedColumnSetTransformer(result, pairs); + + final MultiJoinListenerRecorder listenerRecorder = + new MultiJoinListenerRecorder("multiJoin(" + ii + ")", input, mergedListener, keySources, + sourceKeyModifiedColumnSet, sourceAdditionModifiedColumnSet, transformer, ii); + input.addUpdateListener(listenerRecorder); + listenerRecorders.add(listenerRecorder); + } + } + result.addParentReference(mergedListener); + } + + return result; + } + + private static Table doMultiJoinZeroKey(@NotNull final MultiJoinInputHelper[] joinInputHelpers) { + final SingleValueRowRedirection[] redirections = new SingleValueRowRedirection[joinInputHelpers.length]; + final boolean refreshing = Arrays.stream(joinInputHelpers).anyMatch(ih -> ih.table.isRefreshing()); + + final Map> resultSources = new LinkedHashMap<>(); + boolean hasResults = false; + for (int tableNumber = 0; tableNumber < joinInputHelpers.length; ++tableNumber) { + final MultiJoinInputHelper inputHelper = joinInputHelpers[tableNumber]; + final Table inputTable = inputHelper.table; + + final long key; + if (inputTable.size() == 0) { + key = NULL_ROW_KEY; + } else if (inputTable.size() == 1) { + key = inputTable.getRowSet().firstRowKey(); + hasResults = true; + } else { + throw new IllegalStateException("Duplicate rows for table " + tableNumber + " on zero-key multiJoin."); + } + + final SingleValueRowRedirection rowRedirection; + if (refreshing) { + rowRedirection = new WritableSingleValueRowRedirection(key); + rowRedirection.writableSingleValueCast().startTrackingPrevValues(); + } else { + rowRedirection = new SingleValueRowRedirection(key); + } + redirections[tableNumber] = rowRedirection; + + for (final JoinAddition ja : inputHelper.columnsToAdd) { + resultSources.put(ja.newColumn().name(), RedirectedColumnSource.alwaysRedirect(rowRedirection, + inputTable.getColumnSource(ja.existingColumn().name()))); + } + } + + final QueryTable result = new QueryTable(RowSetFactory.flat(hasResults ? 1 : 0).toTracking(), resultSources); + + if (refreshing) { + final ModifiedColumnSet[] resultModifiedColumnSet = new ModifiedColumnSet[joinInputHelpers.length]; + final List listenerRecorders = new ArrayList<>(); + + final MergedListener mergedListener = new MultiJoinZeroKeyMergedListener( + listenerRecorders, + Collections.emptyList(), + "multiJoin()", + result, + resultModifiedColumnSet, + redirections); + + for (int ii = 0; ii < joinInputHelpers.length; ++ii) { + final MultiJoinInputHelper inputHelper = joinInputHelpers[ii]; + if (inputHelper.table.isRefreshing()) { + final QueryTable input = (QueryTable) inputHelper.table; + + final ModifiedColumnSet sourceAdditionModifiedColumnSet = + input.newModifiedColumnSet(inputHelper.originalAddColumnNames); + + resultModifiedColumnSet[ii] = + result.newModifiedColumnSet(inputHelper.addColumnNames); + final MatchPair[] pairs = MatchPair.fromAddition(Arrays.asList(inputHelper.columnsToAdd)); + final ModifiedColumnSet.Transformer transformer = + input.newModifiedColumnSetTransformer(result, pairs); + + final MultiJoinListenerRecorder listenerRecorder = new MultiJoinListenerRecorder( + "multiJoin(" + ii + ")", input, mergedListener, null, null, sourceAdditionModifiedColumnSet, + transformer, ii); + input.addUpdateListener(listenerRecorder); + listenerRecorders.add(listenerRecorder); + } + } + result.addParentReference(mergedListener); + } + + return result; + } + + private static class MultiJoinListenerRecorder extends ListenerRecorder { + private final ColumnSource[] keyColumns; + private final ModifiedColumnSet sourceKeyModifiedColumnSet; + private final ModifiedColumnSet sourceAdditionModifiedColumnSet; + private final ModifiedColumnSet.Transformer transformer; + private final int tableNumber; + + public MultiJoinListenerRecorder(@NotNull final String description, + @NotNull final QueryTable parent, + @NotNull final MergedListener dependent, + final ColumnSource[] keyColumns, + final ModifiedColumnSet sourceKeyModifiedColumnSet, + final ModifiedColumnSet sourceAdditionModifiedColumnSet, + @NotNull final ModifiedColumnSet.Transformer transformer, + final int tableNumber) { + super(description, parent, dependent); + this.keyColumns = keyColumns; + this.sourceKeyModifiedColumnSet = sourceKeyModifiedColumnSet; + this.sourceAdditionModifiedColumnSet = sourceAdditionModifiedColumnSet; + this.transformer = transformer; + this.tableNumber = tableNumber; + + setMergedListener(dependent); + } + + @Override + public Table getParent() { + return super.getParent(); + } + } + + private static class MultiJoinMergedListener extends MergedListener { + private final IncrementalMultiJoinStateManagerTypedBase stateManager; + private final List recorders; + private final ModifiedColumnSet[] modifiedColumnSets; + private final MultiJoinModifiedSlotTracker slotTracker = new MultiJoinModifiedSlotTracker(); + + protected MultiJoinMergedListener(@NotNull final IncrementalMultiJoinStateManagerTypedBase stateManager, + @NotNull final List recorders, + @NotNull final Collection dependencies, + @NotNull final String listenerDescription, + @NotNull final QueryTable result, + @NotNull final ModifiedColumnSet[] modifiedColumnSets) { + super(recorders, dependencies, listenerDescription, result); + this.stateManager = stateManager; + this.recorders = recorders; + this.modifiedColumnSets = modifiedColumnSets; + } + + @Override + protected void process() { + final int tableCount = stateManager.getTableCount(); + + slotTracker.clear(); + slotTracker.ensureTableCapacity(tableCount); + + final long originalSize = stateManager.getResultSize(); + + for (MultiJoinListenerRecorder recorder : recorders) { + if (recorder.recordedVariablesAreValid()) { + final boolean keysModified = recorder.getModified().isNonempty() + && recorder.getModifiedColumnSet().containsAny(recorder.sourceKeyModifiedColumnSet); + + if (recorder.getRemoved().isNonempty()) { + stateManager.processRemoved(recorder.getRemoved(), recorder.keyColumns, recorder.tableNumber, + slotTracker, FLAG_REMOVE); + } + if (keysModified) { + stateManager.processRemoved(recorder.getModifiedPreShift(), recorder.keyColumns, + recorder.tableNumber, slotTracker, FLAG_MODIFY); + } + + if (recorder.getShifted().nonempty()) { + try (final WritableRowSet previousToShift = recorder.getParent().getRowSet().copyPrev()) { + previousToShift.remove(recorder.getRemoved()); + if (keysModified) { + previousToShift.remove(recorder.getModifiedPreShift()); + } + stateManager.processShifts(previousToShift, recorder.getShifted(), recorder.keyColumns, + recorder.tableNumber, + slotTracker); + } + } + + if (!keysModified && recorder.getModified().isNonempty()) { + // If none of our input columns changed, we have no modifications to pass downstream. + if (recorder.getModifiedColumnSet().containsAny(recorder.sourceAdditionModifiedColumnSet)) { + stateManager.processModified(recorder.getModified(), recorder.keyColumns, + recorder.tableNumber, + slotTracker, FLAG_MODIFY); + } + } + + if (keysModified) { + stateManager.processAdded(recorder.getModified(), recorder.keyColumns, recorder.tableNumber, + slotTracker, FLAG_MODIFY); + } + + if (recorder.getAdded().isNonempty()) { + stateManager.processAdded(recorder.getAdded(), recorder.keyColumns, recorder.tableNumber, + slotTracker, FLAG_ADD); + } + } + } + + final long newSize = stateManager.getResultSize(); + + final TableUpdateImpl downstream = new TableUpdateImpl(); + + + if (newSize > originalSize) { + downstream.added = RowSetFactory.fromRange(originalSize, newSize - 1); + } else { + downstream.added = RowSetFactory.empty(); + } + + final long[] currentRedirections = new long[tableCount]; + final boolean[] modifiedTables = new boolean[tableCount]; + final boolean[] modifiedTablesOnThisRow = new boolean[tableCount]; + + final RowSetBuilderRandom modifiedBuilder = new BitmapRandomBuilder((int) newSize); + final RowSetBuilderRandom emptiedBuilder = RowSetFactory.builderRandom(); + final RowSetBuilderRandom reincarnatedBuilder = RowSetFactory.builderRandom(); + + downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); + downstream.modifiedColumnSet.clear(); + + final byte notShift = (FLAG_ADD | FLAG_REMOVE | FLAG_MODIFY); + final byte addOrRemove = (FLAG_ADD | FLAG_REMOVE); + + slotTracker.forAllModifiedSlots((row, previousRedirections, flagValues) -> { + if (row >= originalSize) { + return; + } + stateManager.getCurrentRedirections(row, currentRedirections); + boolean allNull = true; + boolean rowModified = false; + int numberOfOriginalNulls = 0; + Arrays.fill(modifiedTablesOnThisRow, false); + for (int tableNumber = 0; tableNumber < tableCount; ++tableNumber) { + if (currentRedirections[tableNumber] != NULL_ROW_KEY) { + allNull = false; + } + if (previousRedirections[tableNumber] == NULL_ROW_KEY) { + numberOfOriginalNulls++; + } + if (previousRedirections[tableNumber] == MultiJoinModifiedSlotTracker.SENTINEL_UNINITIALIZED_KEY) { + if (currentRedirections[tableNumber] == NULL_ROW_KEY) { + // This slot was previously deleted and may need to be reincarnated. This redirection + // previously and currently points to no row and should be considered previously null + // for this purpose. + numberOfOriginalNulls++; + } else { + rowModified |= (flagValues[tableNumber] & notShift) != 0; + } + } else { + // If the redirection has changed and we have done anything other than a shift, we must light + // up all the columns for the table as a modification. Similarly, if the row was added and + // deleted from the original table, then we must also light up all the columns as modified. + if ((flagValues[tableNumber] & addOrRemove) != 0) { + rowModified |= (modifiedTablesOnThisRow[tableNumber] = true); + } else if (currentRedirections[tableNumber] != previousRedirections[tableNumber]) { + rowModified |= + (modifiedTablesOnThisRow[tableNumber] = (flagValues[tableNumber] & notShift) != 0); + } else { + rowModified |= (flagValues[tableNumber] & notShift) != 0; + } + } + } + if (allNull) { + emptiedBuilder.addKey(row); + } else if (numberOfOriginalNulls == currentRedirections.length) { + reincarnatedBuilder.addKey(row); + } else if (rowModified) { + modifiedBuilder.addKey(row); + + for (int ii = 0; ii < currentRedirections.length; ++ii) { + if (modifiedTablesOnThisRow[ii]) { + modifiedTables[ii] = true; + } + } + } + }); + + downstream.modified = modifiedBuilder.build(); + if (downstream.modified.isEmpty()) { + downstream.modifiedColumnSet = ModifiedColumnSet.EMPTY; + } else { + int listenerIndex = 0; + for (int ii = 0; ii < modifiedTables.length; ++ii) { + MultiJoinListenerRecorder recorder = listenerIndex >= recorders.size() + ? null + : recorders.get(listenerIndex); + if (recorder == null || recorder.tableNumber != ii) { + // We have a static table, ignore it because it cannot modify anything. + continue; + } + if (modifiedTables[ii]) { + // If this table had any rows that moved slots, or any removed/added slots, _all_ its columns + // are modified. + downstream.modifiedColumnSet.setAll(modifiedColumnSets[ii]); + } else if (recorder.getModified().isNonempty()) { + // If we have "in-place" modifications (same row, in the same slot), we need only mark the + // _modified_ columns from this table modified in the downstream. + recorder.transformer.transform(recorder.getModifiedColumnSet(), downstream.modifiedColumnSet); + } + listenerIndex++; + } + } + downstream.removed = emptiedBuilder.build(); + downstream.added.writableCast().insert(reincarnatedBuilder.build()); + + downstream.shifted = RowSetShiftData.EMPTY; + + if (!downstream.empty()) { + result.getRowSet().writableCast().update(downstream.added, downstream.removed); + result.notifyListeners(downstream); + } + } + } + + private static class MultiJoinZeroKeyMergedListener extends MergedListener { + private final List recorders; + private final ModifiedColumnSet[] modifiedColumnSets; + private final SingleValueRowRedirection[] redirections; + + protected MultiJoinZeroKeyMergedListener(@NotNull final List recorders, + @NotNull final Collection dependencies, + @NotNull final String listenerDescription, + @NotNull final QueryTable result, + @NotNull final ModifiedColumnSet[] modifiedColumnSets, + @NotNull final SingleValueRowRedirection[] redirections) { + super(recorders, dependencies, listenerDescription, result); + this.recorders = recorders; + this.modifiedColumnSets = modifiedColumnSets; + this.redirections = redirections; + } + + @Override + protected void process() { + final TableUpdateImpl downstream = new TableUpdateImpl(); + downstream.modifiedColumnSet = result.getModifiedColumnSetForUpdates(); + downstream.shifted = RowSetShiftData.EMPTY; + + boolean resultModified = false; + + for (MultiJoinListenerRecorder recorder : recorders) { + if (recorder.recordedVariablesAreValid()) { + if (recorder.getRemoved().isNonempty() || recorder.getAdded().isNonempty()) { + if (recorder.getParent().size() > 1) { + throw new IllegalStateException( + "Multiple rows in " + recorder.tableNumber + " for zero-key multiJoin."); + } + resultModified = true; + redirections[recorder.tableNumber].writableSingleValueCast() + .setValue(recorder.getParent().getRowSet().firstRowKey()); + downstream.modifiedColumnSet.setAll(modifiedColumnSets[recorder.tableNumber]); + } else if (recorder.getModified().isNonempty()) { + // If none of our input columns changed, we have no modifications to pass downstream. + if (recorder.getModifiedColumnSet().containsAny(recorder.sourceAdditionModifiedColumnSet)) { + resultModified = true; + recorder.transformer.transform(recorder.getModifiedColumnSet(), + downstream.modifiedColumnSet); + } + redirections[recorder.tableNumber].writableSingleValueCast() + .setValue(recorder.getParent().getRowSet().firstRowKey()); + } else if (recorder.getShifted().nonempty()) { + redirections[recorder.tableNumber].writableSingleValueCast() + .setValue(recorder.getParent().getRowSet().firstRowKey()); + } + } + } + + final boolean hasResults = Arrays.stream(redirections).anyMatch(rd -> rd.getValue() != NULL_ROW_KEY); + if (hasResults && result.size() == 0) { + result.getRowSet().writableCast().insert(0); + downstream.added = RowSetFactory.flat(1); + downstream.removed = RowSetFactory.empty(); + downstream.modified = RowSetFactory.empty(); + downstream.modifiedColumnSet = ModifiedColumnSet.EMPTY; + } else if (!hasResults && result.size() == 1) { + result.getRowSet().writableCast().remove(0); + downstream.added = RowSetFactory.empty(); + downstream.removed = RowSetFactory.flat(1); + downstream.modified = RowSetFactory.empty(); + downstream.modifiedColumnSet = ModifiedColumnSet.EMPTY; + } else { + downstream.added = RowSetFactory.empty(); + downstream.removed = RowSetFactory.empty(); + if (resultModified) { + downstream.modified = RowSetFactory.flat(1); + } else { + // this would be a useless update + return; + } + } + + if (!downstream.empty()) { + result.notifyListeners(downstream); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/typed/HasherConfig.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/typed/HasherConfig.java index a20fd5ab07a..d1024c861a4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/typed/HasherConfig.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/typed/HasherConfig.java @@ -12,7 +12,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; @@ -101,17 +100,40 @@ public ProbeSpec(String name, String stateValueName, boolean requiresRowKeyChunk } static class BuildSpec { + @FunctionalInterface + public interface MethodBuilder { + void accept(HasherConfig config, CodeBlock.Builder builder); + } + + @FunctionalInterface + public interface MethodBuilderWithChunkTypes { + void accept(HasherConfig config, ChunkType[] chunkTypes, CodeBlock.Builder builder); + } + final String name; final String stateValueName; final boolean requiresRowKeyChunk; final boolean allowAlternates; final FoundMethodBuilder found; - final BiConsumer, CodeBlock.Builder> insert; + final MethodBuilderWithChunkTypes insert; final ParameterSpec[] params; public BuildSpec(String name, String stateValueName, boolean requiresRowKeyChunk, boolean allowAlternates, FoundMethodBuilder found, - BiConsumer, CodeBlock.Builder> insert, ParameterSpec... params) { + MethodBuilder insert, ParameterSpec... params) { + // Convert the MethodBuilder to MethodBuilderWithChunkTypes. + this(name, + stateValueName, + requiresRowKeyChunk, + allowAlternates, + found, + (config, chunkTypes, builder) -> insert.accept(config, builder), + params); + } + + public BuildSpec(String name, String stateValueName, boolean requiresRowKeyChunk, + boolean allowAlternates, FoundMethodBuilder found, + MethodBuilderWithChunkTypes insert, ParameterSpec... params) { this.name = name; this.stateValueName = stateValueName; this.requiresRowKeyChunk = requiresRowKeyChunk; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/typed/TypedHasherFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/typed/TypedHasherFactory.java index ae4663a6af4..601ae2fb3bc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/typed/TypedHasherFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/typed/TypedHasherFactory.java @@ -4,31 +4,36 @@ package io.deephaven.engine.table.impl.by.typed; import com.squareup.javapoet.*; - import io.deephaven.UncheckedDeephavenException; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.util.hashing.CharChunkHasher; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.configuration.Configuration; -import io.deephaven.engine.rowset.*; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetBuilderRandom; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; import io.deephaven.engine.table.impl.NaturalJoinModifiedSlotTracker; import io.deephaven.engine.table.impl.asofjoin.RightIncrementalAsOfJoinStateManagerTypedBase; import io.deephaven.engine.table.impl.asofjoin.StaticAsOfJoinStateManagerTypedBase; import io.deephaven.engine.table.impl.asofjoin.TypedAsOfJoinFactory; -import io.deephaven.engine.table.impl.naturaljoin.RightIncrementalNaturalJoinStateManagerTypedBase; import io.deephaven.engine.table.impl.by.*; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.multijoin.TypedMultiJoinFactory; import io.deephaven.engine.table.impl.naturaljoin.IncrementalNaturalJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.naturaljoin.RightIncrementalNaturalJoinStateManagerTypedBase; import io.deephaven.engine.table.impl.naturaljoin.StaticNaturalJoinStateManagerTypedBase; import io.deephaven.engine.table.impl.naturaljoin.TypedNaturalJoinFactory; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.sources.immutable.*; -import io.deephaven.engine.table.impl.updateby.hashing.UpdateByStateManagerTypedBase; import io.deephaven.engine.table.impl.updateby.hashing.TypedUpdateByFactory; +import io.deephaven.engine.table.impl.updateby.hashing.UpdateByStateManagerTypedBase; import io.deephaven.util.QueryConstants; import io.deephaven.util.compare.CharComparisons; import org.apache.commons.lang3.mutable.MutableInt; @@ -333,6 +338,79 @@ public static HasherConfig hasherConfigForBase(Class baseClass) { builder.addProbe(new HasherConfig.ProbeSpec("probeHashTable", "rowState", true, TypedUpdateByFactory::incrementalProbeFound, TypedUpdateByFactory::incrementalProbeMissing, outputPositions)); + } else if (baseClass.equals(StaticMultiJoinStateManagerTypedBase.class)) { + builder.classPrefix("StaticMultiJoinHasher").packageGroup("multijoin").packageMiddle("staticopen") + .openAddressedAlternate(false) + .stateType(int.class).mainStateName("slotToOutputRow") + .emptyStateName("EMPTY_OUTPUT_ROW") + .includeOriginalSources(true) + .supportRehash(true) + .moveMainFull(TypedMultiJoinFactory::staticMoveMainFull) + .alwaysMoveMain(true) + .rehashFullSetup(TypedMultiJoinFactory::staticRehashSetup); + + + builder.addBuild(new HasherConfig.BuildSpec("buildFromTable", "slotValue", + true, true, TypedMultiJoinFactory::staticBuildLeftFound, + TypedMultiJoinFactory::staticBuildLeftInsert, + ParameterSpec.builder(TypeName.get(LongArraySource.class), "tableRedirSource").build(), + ParameterSpec.builder(long.class, "tableNumber").build())); + } else if (baseClass.equals(IncrementalMultiJoinStateManagerTypedBase.class)) { + final ParameterSpec modifiedSlotTrackerParam = + ParameterSpec.builder(MultiJoinModifiedSlotTracker.class, "modifiedSlotTracker").build(); + final ParameterSpec tableRedirSourceParam = + ParameterSpec.builder(TypeName.get(LongArraySource.class), "tableRedirSource").build(); + final ParameterSpec tableNumberParam = + ParameterSpec.builder(int.class, "tableNumber").build(); + final ParameterSpec flagParam = + ParameterSpec.builder(byte.class, "trackerFlag").build(); + + builder.classPrefix("IncrementalMultiJoinHasher").packageGroup("multijoin") + .packageMiddle("incopen") + .openAddressedAlternate(true) + .stateType(int.class).mainStateName("slotToOutputRow") + .overflowOrAlternateStateName("alternateSlotToOutputRow") + .emptyStateName("EMPTY_OUTPUT_ROW") + .includeOriginalSources(true) + .supportRehash(true) + .moveMainFull(TypedMultiJoinFactory::incrementalMoveMainFull) + .moveMainAlternate(TypedMultiJoinFactory::incrementalMoveMainAlternate) + .alwaysMoveMain(true) + .rehashFullSetup(TypedMultiJoinFactory::incrementalRehashSetup); + + builder.addBuild(new HasherConfig.BuildSpec("buildFromTable", "slotValue", + true, true, + TypedMultiJoinFactory::incrementalBuildLeftFound, + TypedMultiJoinFactory::incrementalBuildLeftInsert, + tableRedirSourceParam, + tableNumberParam, + modifiedSlotTrackerParam, + flagParam)); + + builder.addProbe(new HasherConfig.ProbeSpec("remove", "slotValue", true, + TypedMultiJoinFactory::incrementalRemoveLeftFound, + TypedMultiJoinFactory::incrementalRemoveLeftMissing, + tableRedirSourceParam, + tableNumberParam, + modifiedSlotTrackerParam, + flagParam)); + + builder.addProbe(new HasherConfig.ProbeSpec("shift", "slotValue", true, + TypedMultiJoinFactory::incrementalShiftLeftFound, + TypedMultiJoinFactory::incrementalShiftLeftMissing, + tableRedirSourceParam, + tableNumberParam, + modifiedSlotTrackerParam, + flagParam, + ParameterSpec.builder(long.class, "shiftDelta").build())); + + builder.addProbe(new HasherConfig.ProbeSpec("modify", "slotValue", false, + TypedMultiJoinFactory::incrementalModifyLeftFound, + TypedMultiJoinFactory::incrementalModifyLeftMissing, + tableRedirSourceParam, + tableNumberParam, + modifiedSlotTrackerParam, + flagParam)); } else { throw new UnsupportedOperationException("Unknown class to make: " + baseClass); } @@ -455,6 +533,26 @@ public static T make(HasherConfig hasherConfig, ColumnSource[] tableKe if (pregeneratedHasher != null) { return pregeneratedHasher; } + } else if (hasherConfig.baseClass + .equals(StaticMultiJoinStateManagerTypedBase.class)) { + // noinspection unchecked + T pregeneratedHasher = + (T) io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen.TypedHashDispatcher + .dispatch(tableKeySources, originalKeySources, tableSize, maximumLoadFactor, + targetLoadFactor); + if (pregeneratedHasher != null) { + return pregeneratedHasher; + } + } else if (hasherConfig.baseClass + .equals(IncrementalMultiJoinStateManagerTypedBase.class)) { + // noinspection unchecked + T pregeneratedHasher = + (T) io.deephaven.engine.table.impl.multijoin.typed.incopen.gen.TypedHashDispatcher + .dispatch(tableKeySources, originalKeySources, tableSize, maximumLoadFactor, + targetLoadFactor); + if (pregeneratedHasher != null) { + return pregeneratedHasher; + } } } @@ -1120,7 +1218,7 @@ private static void doBuildSearch(HasherConfig hasherConfig, HasherConfig.Bui for (int ii = 0; ii < chunkTypes.length; ++ii) { builder.addStatement("mainKeySource$L.set($L, k$L)", ii, tableLocationName, ii); } - buildSpec.insert.accept(hasherConfig, builder); + buildSpec.insert.accept(hasherConfig, chunkTypes, builder); } builder.addStatement("break"); builder.nextControlFlow("else if (" diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/IncrementalMultiJoinStateManagerTypedBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/IncrementalMultiJoinStateManagerTypedBase.java new file mode 100644 index 00000000000..17fb4d2015d --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/IncrementalMultiJoinStateManagerTypedBase.java @@ -0,0 +1,542 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.engine.table.impl.multijoin; + +import io.deephaven.base.verify.Assert; +import io.deephaven.base.verify.Require; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; +import io.deephaven.engine.table.impl.MultiJoinStateManager; +import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableIntArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableLongArraySource; +import io.deephaven.engine.table.impl.util.*; +import io.deephaven.util.QueryConstants; +import org.apache.commons.lang3.mutable.MutableInt; +import org.jetbrains.annotations.NotNull; + +import java.util.ArrayList; +import java.util.List; + +import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; +import static io.deephaven.engine.table.impl.JoinControl.CHUNK_SIZE; +import static io.deephaven.engine.table.impl.JoinControl.MAX_TABLE_SIZE; +import static io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker.FLAG_SHIFT; +import static io.deephaven.engine.table.impl.util.TypedHasherUtil.getKeyChunks; +import static io.deephaven.engine.table.impl.util.TypedHasherUtil.getPrevKeyChunks; +import static io.deephaven.util.QueryConstants.NULL_BYTE; + +public abstract class IncrementalMultiJoinStateManagerTypedBase implements MultiJoinStateManager { + protected final ColumnSource[] keySourcesForErrorMessages; + private final List redirectionSources = new ArrayList<>(); + + public static final long NO_REDIRECTION = QueryConstants.NULL_LONG; + public static final int EMPTY_OUTPUT_ROW = QueryConstants.NULL_INT; + public static final long EMPTY_COOKIE_SLOT = -1; + + /** The number of slots in our hash table. */ + protected int tableSize; + /** + * The number of slots in our alternate table, to start with "1" is a lie, but rehashPointer is zero; so our + * location value is positive and can be compared against rehashPointer safely. + */ + protected int alternateTableSize = 1; + + /** The number of entries in our hash table in use. */ + protected int numEntries = 0; + + /** + * The table will be rehashed to a load factor of targetLoadFactor if our loadFactor exceeds maximumLoadFactor. + */ + private final double maximumLoadFactor; + + /** The keys for our hash entries. */ + protected final ChunkType[] chunkTypes; + protected final WritableColumnSource[] mainKeySources; + protected final WritableColumnSource[] alternateKeySources; + + /** The output sources representing the keys of our joined table. */ + protected final WritableColumnSource[] outputKeySources; + + /** Store sentinel information and maps hash slots to output row keys. */ + protected ImmutableIntArraySource slotToOutputRow = new ImmutableIntArraySource(); + protected ImmutableIntArraySource alternateSlotToOutputRow; + + protected ImmutableLongArraySource mainModifiedTrackerCookieSource = new ImmutableLongArraySource(); + protected ImmutableLongArraySource alternateModifiedTrackerCookieSource; + + /** how much of the alternate sources are necessary to rehash? */ + protected int rehashPointer = 0; + + protected IncrementalMultiJoinStateManagerTypedBase(ColumnSource[] tableKeySources, + ColumnSource[] keySourcesForErrorMessages, int tableSize, double maximumLoadFactor) { + this.keySourcesForErrorMessages = keySourcesForErrorMessages; + + // we start out with a chunk sized table, and will grow by rehashing as states are added + this.tableSize = tableSize; + Require.leq(tableSize, "tableSize", MAX_TABLE_SIZE); + Require.gtZero(tableSize, "tableSize"); + Require.eq(Integer.bitCount(tableSize), "Integer.bitCount(tableSize)", 1); + Require.inRange(maximumLoadFactor, 0.0, 0.95, "maximumLoadFactor"); + + mainKeySources = new WritableColumnSource[tableKeySources.length]; + alternateKeySources = new WritableColumnSource[tableKeySources.length]; + chunkTypes = new ChunkType[tableKeySources.length]; + + outputKeySources = new WritableColumnSource[tableKeySources.length]; + + for (int ii = 0; ii < tableKeySources.length; ++ii) { + chunkTypes[ii] = tableKeySources[ii].getChunkType(); + mainKeySources[ii] = InMemoryColumnSource.getImmutableMemoryColumnSource(tableSize, + tableKeySources[ii].getType(), tableKeySources[ii].getComponentType()); + outputKeySources[ii] = ArrayBackedColumnSource.getMemoryColumnSource(tableSize, + tableKeySources[ii].getType(), tableKeySources[ii].getComponentType()); + } + + this.maximumLoadFactor = maximumLoadFactor; + + // This is called only once. + ensureCapacity(tableSize); + } + + public int getTableCount() { + return redirectionSources.size(); + } + + private void ensureCapacity(int tableSize) { + slotToOutputRow.ensureCapacity(tableSize); + mainModifiedTrackerCookieSource.ensureCapacity(tableSize); + for (WritableColumnSource mainKeySource : mainKeySources) { + mainKeySource.ensureCapacity(tableSize); + } + } + + public static class BuildContext extends TypedHasherUtil.BuildOrProbeContext { + private BuildContext(ColumnSource[] buildSources, int chunkSize) { + super(buildSources, chunkSize); + } + + final MutableInt rehashCredits = new MutableInt(0); + } + + public static class ProbeContext extends TypedHasherUtil.BuildOrProbeContext { + private ProbeContext(ColumnSource[] probeSources, int chunkSize) { + super(probeSources, chunkSize); + } + } + + public BuildContext makeBuildContext(ColumnSource[] buildSources, long maxSize) { + return new BuildContext(buildSources, (int) Math.min(CHUNK_SIZE, maxSize)); + } + + public ProbeContext makeProbeContext(ColumnSource[] probeSources, long maxSize) { + return new ProbeContext(probeSources, (int) Math.min(CHUNK_SIZE, maxSize)); + } + + private class BuildHandler implements TypedHasherUtil.BuildHandler { + final LongArraySource tableRedirSource; + final int tableNumber; + final MultiJoinModifiedSlotTracker modifiedSlotTracker; + final byte trackerFlag; + + private BuildHandler(LongArraySource tableRedirSource, int tableNumber) { + this.tableRedirSource = tableRedirSource; + this.tableNumber = tableNumber; + this.modifiedSlotTracker = null; + this.trackerFlag = NULL_BYTE; + } + + private BuildHandler(LongArraySource tableRedirSource, int tableNumber, + @NotNull MultiJoinModifiedSlotTracker slotTracker, byte trackerFlag) { + this.tableRedirSource = tableRedirSource; + this.tableNumber = tableNumber; + this.modifiedSlotTracker = slotTracker; + this.trackerFlag = trackerFlag; + } + + @Override + public void doBuild(RowSequence rows, Chunk[] sourceKeyChunks) { + final long maxSize = numEntries + rows.intSize(); + tableRedirSource.ensureCapacity(maxSize); + for (WritableColumnSource src : outputKeySources) { + src.ensureCapacity(maxSize); + } + buildFromTable(rows, sourceKeyChunks, tableRedirSource, tableNumber, modifiedSlotTracker, + trackerFlag); + } + } + + private static abstract class ProbeHandler implements TypedHasherUtil.ProbeHandler { + final LongArraySource tableRedirSource; + final int tableNumber; + final MultiJoinModifiedSlotTracker modifiedSlotTracker; + final byte trackerFlag; + + private ProbeHandler(LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + this.tableRedirSource = tableRedirSource; + this.tableNumber = tableNumber; + this.modifiedSlotTracker = modifiedSlotTracker; + this.trackerFlag = trackerFlag; + } + } + + private class RemoveHandler extends ProbeHandler { + private RemoveHandler(LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + super(tableRedirSource, tableNumber, modifiedSlotTracker, trackerFlag); + } + + @Override + public void doProbe(RowSequence rows, Chunk[] sourceKeyChunks) { + remove(rows, sourceKeyChunks, tableRedirSource, tableNumber, modifiedSlotTracker, trackerFlag); + } + } + + private class ShiftHandler extends ProbeHandler { + final long shiftDelta; + + private ShiftHandler(LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta) { + super(tableRedirSource, tableNumber, modifiedSlotTracker, trackerFlag); + this.shiftDelta = shiftDelta; + } + + @Override + public void doProbe(RowSequence rows, Chunk[] sourceKeyChunks) { + shift(rows, sourceKeyChunks, tableRedirSource, tableNumber, modifiedSlotTracker, trackerFlag, + shiftDelta); + } + } + + private class ModifyHandler extends ProbeHandler { + private ModifyHandler(LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + super(tableRedirSource, tableNumber, modifiedSlotTracker, trackerFlag); + } + + @Override + public void doProbe(RowSequence rows, Chunk[] sourceKeyChunks) { + modify(rows, sourceKeyChunks, tableRedirSource, tableNumber, modifiedSlotTracker, trackerFlag); + } + } + + @Override + public void build(final Table table, ColumnSource[] keySources, int tableNumber) { + if (table.isEmpty()) { + return; + } + final LongArraySource tableRedirSource = redirectionSources.get(tableNumber); + try (final BuildContext bc = makeBuildContext(keySources, table.size())) { + buildTable(true, bc, table.getRowSet(), keySources, new BuildHandler(tableRedirSource, tableNumber)); + } + } + + public void processRemoved(final RowSet rowSet, ColumnSource[] sources, int tableNumber, + @NotNull MultiJoinModifiedSlotTracker slotTracker, byte trackerFlag) { + if (rowSet.isEmpty()) { + return; + } + + Assert.geq(redirectionSources.size(), "redirectionSources.size()", tableNumber, "tableNumber"); + final LongArraySource tableRedirSource = redirectionSources.get(tableNumber); + + try (final ProbeContext pc = makeProbeContext(sources, rowSet.size())) { + probeTable(pc, rowSet, true, sources, + new RemoveHandler(tableRedirSource, tableNumber, slotTracker, trackerFlag)); + } + } + + public void processShifts(final RowSet rowSet, final RowSetShiftData rowSetShiftData, ColumnSource[] sources, + int tableNumber, @NotNull MultiJoinModifiedSlotTracker slotTracker) { + if (rowSet.isEmpty() || rowSetShiftData.empty()) { + return; + } + + Assert.geq(redirectionSources.size(), "redirectionSources.size()", tableNumber, "tableNumber"); + final LongArraySource tableRedirSource = redirectionSources.get(tableNumber); + + // Re-use the probe context for each shift range. + try (final ProbeContext pc = makeProbeContext(sources, rowSet.size())) { + final RowSetShiftData.Iterator sit = rowSetShiftData.applyIterator(); + while (sit.hasNext()) { + sit.next(); + try (final WritableRowSet rowSetToShift = + rowSet.subSetByKeyRange(sit.beginRange(), sit.endRange())) { + probeTable(pc, rowSetToShift, true, sources, new ShiftHandler(tableRedirSource, tableNumber, + slotTracker, FLAG_SHIFT, sit.shiftDelta())); + } + } + } + } + + public void processModified(final RowSet rowSet, ColumnSource[] sources, int tableNumber, + @NotNull MultiJoinModifiedSlotTracker slotTracker, byte trackerFlag) { + if (rowSet.isEmpty()) { + return; + } + + Assert.geq(redirectionSources.size(), "redirectionSources.size()", tableNumber, "tableNumber"); + final LongArraySource tableRedirSource = redirectionSources.get(tableNumber); + + try (final ProbeContext pc = makeProbeContext(sources, rowSet.size())) { + probeTable(pc, rowSet, false, sources, + new ModifyHandler(tableRedirSource, tableNumber, slotTracker, trackerFlag)); + } + } + + public void processAdded(final RowSet rowSet, ColumnSource[] sources, int tableNumber, + @NotNull MultiJoinModifiedSlotTracker slotTracker, byte trackerFlag) { + if (rowSet.isEmpty()) { + return; + } + + Assert.geq(redirectionSources.size(), "redirectionSources.size()", tableNumber, "tableNumber"); + final LongArraySource tableRedirSource = redirectionSources.get(tableNumber); + + try (final BuildContext bc = makeBuildContext(sources, rowSet.size())) { + buildTable(false, bc, rowSet, sources, + new BuildHandler(tableRedirSource, tableNumber, slotTracker, trackerFlag)); + } + } + + protected abstract void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag); + + protected abstract void remove(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag); + + protected abstract void shift(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta); + + protected abstract void modify(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag); + + abstract protected void migrateFront(); + + private void buildTable( + final boolean initialBuild, + final BuildContext bc, + final RowSequence buildRows, + final ColumnSource[] buildSources, + final BuildHandler buildHandler) { + try (final RowSequence.Iterator rsIt = buildRows.getRowSequenceIterator()) { + // noinspection unchecked + final Chunk[] sourceKeyChunks = new Chunk[buildSources.length]; + + while (rsIt.hasMore()) { + final RowSequence rows = rsIt.getNextRowSequenceWithLength(bc.chunkSize); + final int nextChunkSize = rows.intSize(); + while (doRehash(initialBuild, bc.rehashCredits, nextChunkSize)) { + migrateFront(); + } + + getKeyChunks(buildSources, bc.getContexts, sourceKeyChunks, rows); + + final long oldEntries = numEntries; + buildHandler.doBuild(rows, sourceKeyChunks); + final long entriesAdded = numEntries - oldEntries; + // if we actually added anything, then take away from the "equity" we've built up rehashing, otherwise + // don't penalize this build call with additional rehashing + bc.rehashCredits.subtract(entriesAdded); + + bc.resetSharedContexts(); + } + } + } + + private void probeTable( + final ProbeContext pc, + final RowSequence probeRows, + final boolean usePrev, + final ColumnSource[] probeSources, + final TypedHasherUtil.ProbeHandler handler) { + try (final RowSequence.Iterator rsIt = probeRows.getRowSequenceIterator()) { + // noinspection unchecked + final Chunk[] sourceKeyChunks = new Chunk[probeSources.length]; + + while (rsIt.hasMore()) { + final RowSequence rows = rsIt.getNextRowSequenceWithLength(pc.chunkSize); + + if (usePrev) { + getPrevKeyChunks(probeSources, pc.getContexts, sourceKeyChunks, rows); + } else { + getKeyChunks(probeSources, pc.getContexts, sourceKeyChunks, rows); + } + + handler.doProbe(rows, sourceKeyChunks); + + pc.resetSharedContexts(); + } + } + } + + /** + * @param fullRehash should we rehash the entire table (if false, we rehash incrementally) + * @param rehashCredits the number of entries this operation has rehashed (input/output) + * @param nextChunkSize the size of the chunk we are processing + * @return true if a front migration is required + */ + public boolean doRehash(boolean fullRehash, MutableInt rehashCredits, int nextChunkSize) { + if (rehashPointer > 0) { + final int requiredRehash = nextChunkSize - rehashCredits.intValue(); + if (requiredRehash <= 0) { + return false; + } + + // before building, we need to do at least as much rehash work as we would do build work + rehashCredits.add(rehashInternalPartial(requiredRehash)); + if (rehashPointer == 0) { + clearAlternate(); + } + } + + int oldTableSize = tableSize; + while (rehashRequired(nextChunkSize)) { + tableSize *= 2; + + if (tableSize < 0 || tableSize > MAX_TABLE_SIZE) { + throw new UnsupportedOperationException("Hash table exceeds maximum size!"); + } + } + + if (oldTableSize == tableSize) { + return false; + } + + // we can't give the caller credit for rehashes with the old table, we need to begin migrating things again + if (rehashCredits.intValue() > 0) { + rehashCredits.setValue(0); + } + + if (fullRehash) { + // if we are doing a full rehash, we need to ditch the alternate + if (rehashPointer > 0) { + rehashInternalPartial(numEntries); + clearAlternate(); + } + + rehashInternalFull(oldTableSize); + + return false; + } + + Assert.eqZero(rehashPointer, "rehashPointer"); + + if (numEntries == 0) { + return false; + } + newAlternate(); + alternateTableSize = oldTableSize; + rehashPointer = alternateTableSize; + return true; + } + + protected void newAlternate() { + alternateSlotToOutputRow = slotToOutputRow; + slotToOutputRow = new ImmutableIntArraySource(); + slotToOutputRow.ensureCapacity(tableSize); + + alternateModifiedTrackerCookieSource = mainModifiedTrackerCookieSource; + mainModifiedTrackerCookieSource = new ImmutableLongArraySource(); + mainModifiedTrackerCookieSource.ensureCapacity(tableSize); + + for (int ii = 0; ii < mainKeySources.length; ++ii) { + alternateKeySources[ii] = mainKeySources[ii]; + mainKeySources[ii] = InMemoryColumnSource.getImmutableMemoryColumnSource(tableSize, + alternateKeySources[ii].getType(), alternateKeySources[ii].getComponentType()); + mainKeySources[ii].ensureCapacity(tableSize); + } + } + + protected void clearAlternate() { + alternateSlotToOutputRow = null; + alternateModifiedTrackerCookieSource = null; + for (int ii = 0; ii < mainKeySources.length; ++ii) { + alternateKeySources[ii] = null; + } + alternateTableSize = 1; + } + + public boolean rehashRequired(int nextChunkSize) { + return (numEntries + nextChunkSize) > (tableSize * maximumLoadFactor); + } + + abstract protected void rehashInternalFull(final int oldSize); + + /** + * @param numEntriesToRehash number of entries to rehash into main table + * @return actual number of entries rehashed + */ + protected abstract int rehashInternalPartial(int numEntriesToRehash); + + protected int hashToTableLocation(int hash) { + return hash & (tableSize - 1); + } + + protected int hashToTableLocationAlternate(int hash) { + return hash & (alternateTableSize - 1); + } + + /** produce a pretty key for error messages. */ + protected String keyString(Chunk[] sourceKeyChunks, int chunkPosition) { + return ChunkUtils.extractKeyStringFromChunks(chunkTypes, sourceKeyChunks, chunkPosition); + } + + public void getCurrentRedirections(long slot, long[] redirections) { + for (int tt = 0; tt < redirectionSources.size(); ++tt) { + final long redirection = redirectionSources.get(tt).getLong(slot); + redirections[tt] = redirection == QueryConstants.NULL_LONG ? NULL_ROW_KEY : redirection; + } + } + + public void startTrackingPrevRedirectionValues(int tableNumber) { + redirectionSources.get(tableNumber).startTrackingPrevValues(); + } + + @Override + public long getResultSize() { + return numEntries; + } + + @Override + public ColumnSource[] getKeyHashTableSources() { + return outputKeySources; + } + + @Override + public RowRedirection getRowRedirectionForTable(int tableNumber) { + return new LongColumnSourceRowRedirection<>(redirectionSources.get(tableNumber)); + } + + @Override + public void ensureTableCapacity(int tables) { + while (redirectionSources.size() < tables) { + final LongArraySource newRedirection = new LongArraySource(); + newRedirection.ensureCapacity(numEntries); + redirectionSources.add(newRedirection); + } + } + + @Override + public void setTargetLoadFactor(double targetLoadFactor) {} + + @Override + public void setMaximumLoadFactor(double maximumLoadFactor) {} +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/StaticMultiJoinStateManagerTypedBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/StaticMultiJoinStateManagerTypedBase.java new file mode 100644 index 00000000000..c662260f42a --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/StaticMultiJoinStateManagerTypedBase.java @@ -0,0 +1,214 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.engine.table.impl.multijoin; + +import io.deephaven.base.verify.Require; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.WritableColumnSource; +import io.deephaven.engine.table.impl.MultiJoinStateManager; +import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableIntArraySource; +import io.deephaven.engine.table.impl.util.*; +import io.deephaven.engine.table.impl.util.TypedHasherUtil.BuildOrProbeContext.BuildContext; +import io.deephaven.util.QueryConstants; + +import java.util.ArrayList; +import java.util.List; + +import static io.deephaven.engine.table.impl.JoinControl.CHUNK_SIZE; +import static io.deephaven.engine.table.impl.JoinControl.MAX_TABLE_SIZE; +import static io.deephaven.engine.table.impl.util.TypedHasherUtil.getKeyChunks; + +public abstract class StaticMultiJoinStateManagerTypedBase implements MultiJoinStateManager { + protected final ColumnSource[] keySourcesForErrorMessages; + private final List redirectionSources = new ArrayList<>(); + + public static final long NO_REDIRECTION = QueryConstants.NULL_LONG; + public static final int EMPTY_OUTPUT_ROW = QueryConstants.NULL_INT; + + /** The number of slots in our hash table. */ + protected int tableSize; + + /** The number of entries in our hash table in use. */ + protected int numEntries = 0; + + /** + * The table will be rehashed to a load factor of targetLoadFactor if our loadFactor exceeds maximumLoadFactor. + */ + private final double maximumLoadFactor; + + /** The keys for our hash entries. */ + protected final ChunkType[] chunkTypes; + protected final WritableColumnSource[] mainKeySources; + + /** The output sources representing the keys of our joined table. */ + protected final WritableColumnSource[] outputKeySources; + + /** Store sentinel information and maps hash slots to output row keys. */ + protected ImmutableIntArraySource slotToOutputRow = new ImmutableIntArraySource(); + + protected StaticMultiJoinStateManagerTypedBase(ColumnSource[] tableKeySources, + ColumnSource[] keySourcesForErrorMessages, + int tableSize, + double maximumLoadFactor) { + this.keySourcesForErrorMessages = keySourcesForErrorMessages; + + this.tableSize = tableSize; + Require.leq(tableSize, "tableSize", MAX_TABLE_SIZE); + Require.gtZero(tableSize, "tableSize"); + Require.eq(Integer.bitCount(tableSize), "Integer.bitCount(tableSize)", 1); + Require.inRange(maximumLoadFactor, 0.0, 0.95, "maximumLoadFactor"); + + mainKeySources = new WritableColumnSource[tableKeySources.length]; + chunkTypes = new ChunkType[tableKeySources.length]; + + outputKeySources = new WritableColumnSource[tableKeySources.length]; + + for (int ii = 0; ii < tableKeySources.length; ++ii) { + chunkTypes[ii] = tableKeySources[ii].getChunkType(); + mainKeySources[ii] = InMemoryColumnSource.getImmutableMemoryColumnSource(tableSize, + tableKeySources[ii].getType(), tableKeySources[ii].getComponentType()); + outputKeySources[ii] = ArrayBackedColumnSource.getMemoryColumnSource(tableSize, + tableKeySources[ii].getType(), tableKeySources[ii].getComponentType()); + } + + this.maximumLoadFactor = maximumLoadFactor; + + // Called only once, here in the constructor. + ensureCapacity(tableSize); + } + + private void ensureCapacity(int tableSize) { + slotToOutputRow.ensureCapacity(tableSize); + for (WritableColumnSource mainKeySource : mainKeySources) { + mainKeySource.ensureCapacity(tableSize); + } + } + + BuildContext makeBuildContext(ColumnSource[] buildSources, long maxSize) { + return new BuildContext(buildSources, (int) Math.min(CHUNK_SIZE, maxSize)); + } + + @Override + public void build(final Table table, ColumnSource[] keySources, int tableNumber) { + if (table.isEmpty()) { + return; + } + final LongArraySource tableRedirSource = redirectionSources.get(tableNumber); + try (final BuildContext bc = makeBuildContext(keySources, table.size())) { + buildTable(bc, table.getRowSet(), keySources, new BuildHandler(tableRedirSource, tableNumber)); + } + } + + private class BuildHandler implements TypedHasherUtil.BuildHandler { + final LongArraySource tableRedirSource; + final long tableNumber; + + private BuildHandler(LongArraySource tableRedirSource, long tableNumber) { + this.tableRedirSource = tableRedirSource; + this.tableNumber = tableNumber; + } + + @Override + public void doBuild(RowSequence rows, Chunk[] sourceKeyChunks) { + final long maxSize = numEntries + rows.intSize(); + tableRedirSource.ensureCapacity(maxSize); + for (WritableColumnSource src : outputKeySources) { + src.ensureCapacity(maxSize); + } + buildFromTable(rows, sourceKeyChunks, tableRedirSource, tableNumber); + } + } + + protected abstract void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, long tableNumber); + + protected void buildTable( + final BuildContext bc, + final RowSequence buildRows, + final ColumnSource[] buildSources, + final TypedHasherUtil.BuildHandler buildHandler) { + try (final RowSequence.Iterator rsIt = buildRows.getRowSequenceIterator()) { + // noinspection unchecked + final Chunk[] sourceKeyChunks = new Chunk[buildSources.length]; + + while (rsIt.hasMore()) { + final RowSequence rows = rsIt.getNextRowSequenceWithLength(bc.chunkSize); + + doRehash(rows.intSize()); + + getKeyChunks(buildSources, bc.getContexts, sourceKeyChunks, rows); + + buildHandler.doBuild(rows, sourceKeyChunks); + + bc.resetSharedContexts(); + } + } + } + + public void doRehash(final int nextChunkSize) { + final int oldSize = tableSize; + while (rehashRequired(nextChunkSize)) { + tableSize *= 2; + if (tableSize < 0 || tableSize > MAX_TABLE_SIZE) { + throw new UnsupportedOperationException("Hash table exceeds maximum size!"); + } + } + if (tableSize > oldSize) { + rehashInternalFull(oldSize); + } + } + + public boolean rehashRequired(int nextChunkSize) { + return (numEntries + nextChunkSize) > (tableSize * maximumLoadFactor); + } + + abstract protected void rehashInternalFull(final int oldSize); + + protected int hashToTableLocation(int hash) { + return hash & (tableSize - 1); + } + + /** produce a pretty key for error messages. */ + protected String keyString(Chunk[] sourceKeyChunks, int chunkPosition) { + return ChunkUtils.extractKeyStringFromChunks(chunkTypes, sourceKeyChunks, chunkPosition); + } + + @Override + public long getResultSize() { + return numEntries; + } + + @Override + public ColumnSource[] getKeyHashTableSources() { + return outputKeySources; + } + + @Override + public RowRedirection getRowRedirectionForTable(int tableNumber) { + return new LongColumnSourceRowRedirection<>(redirectionSources.get(tableNumber)); + } + + @Override + public void ensureTableCapacity(int tables) { + while (redirectionSources.size() < tables) { + final LongArraySource newRedirection = new LongArraySource(); + newRedirection.ensureCapacity(numEntries); + redirectionSources.add(newRedirection); + } + } + + @Override + public void setTargetLoadFactor(double targetLoadFactor) {} + + @Override + public void setMaximumLoadFactor(double maximumLoadFactor) {} +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/TypedMultiJoinFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/TypedMultiJoinFactory.java new file mode 100644 index 00000000000..e4dda8648af --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/TypedMultiJoinFactory.java @@ -0,0 +1,151 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.engine.table.impl.multijoin; + +import com.squareup.javapoet.CodeBlock; +import io.deephaven.chunk.ChunkType; +import io.deephaven.engine.table.impl.by.typed.HasherConfig; + +public class TypedMultiJoinFactory { + public static void staticBuildLeftInsert(HasherConfig hasherConfig, ChunkType[] chunkTypes, + CodeBlock.Builder builder) { + builder.addStatement("final int outputKey = numEntries - 1"); + builder.addStatement("slotToOutputRow.set(tableLocation, outputKey)"); + builder.addStatement("tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition))"); + for (int ii = 0; ii < chunkTypes.length; ii++) { + builder.addStatement("outputKeySources[" + ii + "].set((long)outputKey, k" + ii + ")"); + } + } + + public static void staticBuildLeftFound(HasherConfig hasherConfig, boolean alternate, + CodeBlock.Builder builder) { + builder.beginControlFlow("if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION)"); + builder.addStatement( + "throw new IllegalStateException(\"Duplicate key found for \" + keyString(sourceKeyChunks, chunkPosition) + \" in table \" + tableNumber + \".\")"); + builder.endControlFlow(); + builder.addStatement("tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition))"); + } + + public static void staticRehashSetup(CodeBlock.Builder builder) {} + + public static void staticMoveMainFull(CodeBlock.Builder builder) {} + + public static void incrementalRehashSetup(CodeBlock.Builder builder) { + builder.addStatement("final long [] oldModifiedCookie = mainModifiedTrackerCookieSource.getArray()"); + builder.addStatement("final long [] destModifiedCookie = new long[tableSize]"); + builder.addStatement("mainModifiedTrackerCookieSource.setArray(destModifiedCookie)"); + } + + public static void incrementalMoveMainFull(CodeBlock.Builder builder) { + builder.addStatement("destModifiedCookie[destinationTableLocation] = oldModifiedCookie[sourceBucket]"); + } + + public static void incrementalMoveMainAlternate(CodeBlock.Builder builder) { + builder.addStatement("final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(locationToMigrate)"); + builder.addStatement("mainModifiedTrackerCookieSource.set(destinationTableLocation, cookie)"); + builder.addStatement("alternateModifiedTrackerCookieSource.set(locationToMigrate, EMPTY_COOKIE_SLOT)"); + } + + public static void incrementalBuildLeftFound(HasherConfig hasherConfig, boolean alternate, + CodeBlock.Builder builder) { + builder.beginControlFlow("if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION)"); + builder.addStatement( + "throw new IllegalStateException(\"Duplicate key found for \" + keyString(sourceKeyChunks, chunkPosition) + \" in table \" + tableNumber + \".\")"); + builder.endControlFlow(); + builder.addStatement("tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition))"); + + builder.beginControlFlow("if (modifiedSlotTracker != null)"); + if (!alternate) { + builder.addStatement("final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation)"); + builder.addStatement( + "mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag))"); + } else { + builder.addStatement( + "final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation)"); + builder.addStatement( + "alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag))"); + } + builder.endControlFlow(); + } + + public static void incrementalBuildLeftInsert(HasherConfig hasherConfig, ChunkType[] chunkTypes, + CodeBlock.Builder builder) { + builder.addStatement("final int outputKey = numEntries - 1"); + builder.addStatement("slotToOutputRow.set(tableLocation, outputKey)"); + builder.addStatement("tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition))"); + for (int ii = 0; ii < chunkTypes.length; ii++) { + builder.addStatement("outputKeySources[" + ii + "].set((long)outputKey, k" + ii + ")"); + } + builder.add("// NOTE: if there are other tables adding this row this cycle, we will add these into the slot\n"); + builder.add( + "// tracker. However, when the modified slots are processed we will identify the output row as new\n"); + builder.add("// for this cycle and ignore the incomplete tracker data.\n"); + builder.addStatement("mainModifiedTrackerCookieSource.set(tableLocation, EMPTY_COOKIE_SLOT)"); + } + + public static void incrementalModifyLeftFound(HasherConfig hasherConfig, boolean alternate, + CodeBlock.Builder builder) { + if (!alternate) { + builder.addStatement("final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation)"); + builder.addStatement( + "mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag))"); + } else { + builder.addStatement( + "final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation)"); + builder.addStatement( + "alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag))"); + } + } + + public static void incrementalModifyLeftMissing(CodeBlock.Builder builder) { + builder.addStatement( + "throw new IllegalStateException(\"Matching row not found for \" + keyString(sourceKeyChunks, chunkPosition) + \" in table \" + tableNumber + \".\")"); + } + + public static void incrementalRemoveLeftFound(HasherConfig hasherConfig, boolean alternate, + CodeBlock.Builder builder) { + builder.addStatement("final long mappedRowKey = tableRedirSource.getUnsafe(slotValue)"); + builder.addStatement("tableRedirSource.set(slotValue, NO_REDIRECTION)"); + builder.addStatement( + "Assert.eq(rowKeyChunk.get(chunkPosition), \"rowKey\", mappedRowKey, \"mappedRowKey\")"); + if (!alternate) { + builder.addStatement("final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation)"); + builder.addStatement( + "mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag))"); + } else { + builder.addStatement( + "final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation)"); + builder.addStatement( + "alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag))"); + } + } + + public static void incrementalRemoveLeftMissing(CodeBlock.Builder builder) { + builder.addStatement( + "throw new IllegalStateException(\"Matching row not found for \" + keyString(sourceKeyChunks, chunkPosition) + \" in table \" + tableNumber + \".\")"); + } + + public static void incrementalShiftLeftFound(HasherConfig hasherConfig, boolean alternate, + CodeBlock.Builder builder) { + builder.addStatement("final long mappedRowKey = tableRedirSource.getUnsafe(slotValue)"); + builder.addStatement( + "Assert.eq(rowKeyChunk.get(chunkPosition), \"rowKey\", mappedRowKey, \"mappedRowKey\")"); + builder.addStatement("tableRedirSource.set(slotValue, mappedRowKey + shiftDelta)"); + if (!alternate) { + builder.addStatement("final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation)"); + builder.addStatement( + "mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag))"); + } else { + builder.addStatement( + "final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation)"); + builder.addStatement( + "alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag))"); + } + } + + public static void incrementalShiftLeftMissing(CodeBlock.Builder builder) { + builder.addStatement( + "throw new IllegalStateException(\"Matching row not found for \" + keyString(sourceKeyChunks, chunkPosition) + \" in table \" + tableNumber + \".\")"); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherByte.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherByte.java new file mode 100644 index 00000000000..d90e8e6ece5 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherByte.java @@ -0,0 +1,346 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.incopen.gen; + +import static io.deephaven.util.compare.ByteComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.ByteChunk; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.ByteChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableByteArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class IncrementalMultiJoinHasherByte extends IncrementalMultiJoinStateManagerTypedBase { + private ImmutableByteArraySource mainKeySource0; + + private ImmutableByteArraySource alternateKeySource0; + + public IncrementalMultiJoinHasherByte(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableByteArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + private int alternateNextTableLocation(int tableLocation) { + return (tableLocation + 1) & (alternateTableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final ByteChunk keyChunk0 = sourceKeyChunks[0].asByteChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final byte k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + MAIN_SEARCH: while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + int alternateTableLocation = firstAlternateTableLocation; + while (alternateTableLocation < rehashPointer) { + slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + break; + } else if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break MAIN_SEARCH; + } else { + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + // NOTE: if there are other tables adding this row this cycle, we will add these into the slot + // tracker. However, when the modified slots are processed we will identify the output row as new + // for this cycle and ignore the incomplete tracker data. + mainModifiedTrackerCookieSource.set(tableLocation, EMPTY_COOKIE_SLOT); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + protected void remove(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final ByteChunk keyChunk0 = sourceKeyChunks[0].asByteChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final byte k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void shift(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta) { + final ByteChunk keyChunk0 = sourceKeyChunks[0].asByteChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final byte k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void modify(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final ByteChunk keyChunk0 = sourceKeyChunks[0].asByteChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final byte k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + private static int hash(byte k0) { + int hash = ByteChunkHasher.hashInitialSingle(k0); + return hash; + } + + private boolean migrateOneLocation(int locationToMigrate) { + final int currentStateValue = alternateSlotToOutputRow.getUnsafe(locationToMigrate); + if (currentStateValue == EMPTY_OUTPUT_ROW) { + return false; + } + final byte k0 = alternateKeySource0.getUnsafe(locationToMigrate); + final int hash = hash(k0); + int destinationTableLocation = hashToTableLocation(hash); + while (slotToOutputRow.getUnsafe(destinationTableLocation) != EMPTY_OUTPUT_ROW) { + destinationTableLocation = nextTableLocation(destinationTableLocation); + } + mainKeySource0.set(destinationTableLocation, k0); + slotToOutputRow.set(destinationTableLocation, currentStateValue); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(locationToMigrate); + mainModifiedTrackerCookieSource.set(destinationTableLocation, cookie); + alternateModifiedTrackerCookieSource.set(locationToMigrate, EMPTY_COOKIE_SLOT); + alternateSlotToOutputRow.set(locationToMigrate, EMPTY_OUTPUT_ROW); + return true; + } + + @Override + protected int rehashInternalPartial(int entriesToRehash) { + int rehashedEntries = 0; + while (rehashPointer > 0 && rehashedEntries < entriesToRehash) { + if (migrateOneLocation(--rehashPointer)) { + rehashedEntries++; + } + } + return rehashedEntries; + } + + @Override + protected void newAlternate() { + super.newAlternate(); + this.mainKeySource0 = (ImmutableByteArraySource)super.mainKeySources[0]; + this.alternateKeySource0 = (ImmutableByteArraySource)super.alternateKeySources[0]; + } + + @Override + protected void clearAlternate() { + super.clearAlternate(); + this.alternateKeySource0 = null; + } + + @Override + protected void migrateFront() { + int location = 0; + while (migrateOneLocation(location++)); + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final byte[] destKeyArray0 = new byte[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final byte [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + final long [] oldModifiedCookie = mainModifiedTrackerCookieSource.getArray(); + final long [] destModifiedCookie = new long[tableSize]; + mainModifiedTrackerCookieSource.setArray(destModifiedCookie); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final byte k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + destModifiedCookie[destinationTableLocation] = oldModifiedCookie[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherChar.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherChar.java new file mode 100644 index 00000000000..e4929940c0e --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherChar.java @@ -0,0 +1,346 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.incopen.gen; + +import static io.deephaven.util.compare.CharComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.CharChunk; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.CharChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableCharArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class IncrementalMultiJoinHasherChar extends IncrementalMultiJoinStateManagerTypedBase { + private ImmutableCharArraySource mainKeySource0; + + private ImmutableCharArraySource alternateKeySource0; + + public IncrementalMultiJoinHasherChar(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableCharArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + private int alternateNextTableLocation(int tableLocation) { + return (tableLocation + 1) & (alternateTableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final CharChunk keyChunk0 = sourceKeyChunks[0].asCharChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final char k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + MAIN_SEARCH: while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + int alternateTableLocation = firstAlternateTableLocation; + while (alternateTableLocation < rehashPointer) { + slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + break; + } else if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break MAIN_SEARCH; + } else { + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + // NOTE: if there are other tables adding this row this cycle, we will add these into the slot + // tracker. However, when the modified slots are processed we will identify the output row as new + // for this cycle and ignore the incomplete tracker data. + mainModifiedTrackerCookieSource.set(tableLocation, EMPTY_COOKIE_SLOT); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + protected void remove(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final CharChunk keyChunk0 = sourceKeyChunks[0].asCharChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final char k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void shift(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta) { + final CharChunk keyChunk0 = sourceKeyChunks[0].asCharChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final char k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void modify(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final CharChunk keyChunk0 = sourceKeyChunks[0].asCharChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final char k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + private static int hash(char k0) { + int hash = CharChunkHasher.hashInitialSingle(k0); + return hash; + } + + private boolean migrateOneLocation(int locationToMigrate) { + final int currentStateValue = alternateSlotToOutputRow.getUnsafe(locationToMigrate); + if (currentStateValue == EMPTY_OUTPUT_ROW) { + return false; + } + final char k0 = alternateKeySource0.getUnsafe(locationToMigrate); + final int hash = hash(k0); + int destinationTableLocation = hashToTableLocation(hash); + while (slotToOutputRow.getUnsafe(destinationTableLocation) != EMPTY_OUTPUT_ROW) { + destinationTableLocation = nextTableLocation(destinationTableLocation); + } + mainKeySource0.set(destinationTableLocation, k0); + slotToOutputRow.set(destinationTableLocation, currentStateValue); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(locationToMigrate); + mainModifiedTrackerCookieSource.set(destinationTableLocation, cookie); + alternateModifiedTrackerCookieSource.set(locationToMigrate, EMPTY_COOKIE_SLOT); + alternateSlotToOutputRow.set(locationToMigrate, EMPTY_OUTPUT_ROW); + return true; + } + + @Override + protected int rehashInternalPartial(int entriesToRehash) { + int rehashedEntries = 0; + while (rehashPointer > 0 && rehashedEntries < entriesToRehash) { + if (migrateOneLocation(--rehashPointer)) { + rehashedEntries++; + } + } + return rehashedEntries; + } + + @Override + protected void newAlternate() { + super.newAlternate(); + this.mainKeySource0 = (ImmutableCharArraySource)super.mainKeySources[0]; + this.alternateKeySource0 = (ImmutableCharArraySource)super.alternateKeySources[0]; + } + + @Override + protected void clearAlternate() { + super.clearAlternate(); + this.alternateKeySource0 = null; + } + + @Override + protected void migrateFront() { + int location = 0; + while (migrateOneLocation(location++)); + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final char[] destKeyArray0 = new char[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final char [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + final long [] oldModifiedCookie = mainModifiedTrackerCookieSource.getArray(); + final long [] destModifiedCookie = new long[tableSize]; + mainModifiedTrackerCookieSource.setArray(destModifiedCookie); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final char k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + destModifiedCookie[destinationTableLocation] = oldModifiedCookie[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherDouble.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherDouble.java new file mode 100644 index 00000000000..ccc988612a6 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherDouble.java @@ -0,0 +1,346 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.incopen.gen; + +import static io.deephaven.util.compare.DoubleComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.DoubleChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.DoubleChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableDoubleArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class IncrementalMultiJoinHasherDouble extends IncrementalMultiJoinStateManagerTypedBase { + private ImmutableDoubleArraySource mainKeySource0; + + private ImmutableDoubleArraySource alternateKeySource0; + + public IncrementalMultiJoinHasherDouble(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableDoubleArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + private int alternateNextTableLocation(int tableLocation) { + return (tableLocation + 1) & (alternateTableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final DoubleChunk keyChunk0 = sourceKeyChunks[0].asDoubleChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final double k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + MAIN_SEARCH: while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + int alternateTableLocation = firstAlternateTableLocation; + while (alternateTableLocation < rehashPointer) { + slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + break; + } else if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break MAIN_SEARCH; + } else { + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + // NOTE: if there are other tables adding this row this cycle, we will add these into the slot + // tracker. However, when the modified slots are processed we will identify the output row as new + // for this cycle and ignore the incomplete tracker data. + mainModifiedTrackerCookieSource.set(tableLocation, EMPTY_COOKIE_SLOT); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + protected void remove(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final DoubleChunk keyChunk0 = sourceKeyChunks[0].asDoubleChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final double k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void shift(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta) { + final DoubleChunk keyChunk0 = sourceKeyChunks[0].asDoubleChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final double k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void modify(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final DoubleChunk keyChunk0 = sourceKeyChunks[0].asDoubleChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final double k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + private static int hash(double k0) { + int hash = DoubleChunkHasher.hashInitialSingle(k0); + return hash; + } + + private boolean migrateOneLocation(int locationToMigrate) { + final int currentStateValue = alternateSlotToOutputRow.getUnsafe(locationToMigrate); + if (currentStateValue == EMPTY_OUTPUT_ROW) { + return false; + } + final double k0 = alternateKeySource0.getUnsafe(locationToMigrate); + final int hash = hash(k0); + int destinationTableLocation = hashToTableLocation(hash); + while (slotToOutputRow.getUnsafe(destinationTableLocation) != EMPTY_OUTPUT_ROW) { + destinationTableLocation = nextTableLocation(destinationTableLocation); + } + mainKeySource0.set(destinationTableLocation, k0); + slotToOutputRow.set(destinationTableLocation, currentStateValue); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(locationToMigrate); + mainModifiedTrackerCookieSource.set(destinationTableLocation, cookie); + alternateModifiedTrackerCookieSource.set(locationToMigrate, EMPTY_COOKIE_SLOT); + alternateSlotToOutputRow.set(locationToMigrate, EMPTY_OUTPUT_ROW); + return true; + } + + @Override + protected int rehashInternalPartial(int entriesToRehash) { + int rehashedEntries = 0; + while (rehashPointer > 0 && rehashedEntries < entriesToRehash) { + if (migrateOneLocation(--rehashPointer)) { + rehashedEntries++; + } + } + return rehashedEntries; + } + + @Override + protected void newAlternate() { + super.newAlternate(); + this.mainKeySource0 = (ImmutableDoubleArraySource)super.mainKeySources[0]; + this.alternateKeySource0 = (ImmutableDoubleArraySource)super.alternateKeySources[0]; + } + + @Override + protected void clearAlternate() { + super.clearAlternate(); + this.alternateKeySource0 = null; + } + + @Override + protected void migrateFront() { + int location = 0; + while (migrateOneLocation(location++)); + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final double[] destKeyArray0 = new double[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final double [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + final long [] oldModifiedCookie = mainModifiedTrackerCookieSource.getArray(); + final long [] destModifiedCookie = new long[tableSize]; + mainModifiedTrackerCookieSource.setArray(destModifiedCookie); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final double k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + destModifiedCookie[destinationTableLocation] = oldModifiedCookie[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherFloat.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherFloat.java new file mode 100644 index 00000000000..115a25b0ca8 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherFloat.java @@ -0,0 +1,346 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.incopen.gen; + +import static io.deephaven.util.compare.FloatComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.FloatChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.FloatChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableFloatArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class IncrementalMultiJoinHasherFloat extends IncrementalMultiJoinStateManagerTypedBase { + private ImmutableFloatArraySource mainKeySource0; + + private ImmutableFloatArraySource alternateKeySource0; + + public IncrementalMultiJoinHasherFloat(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableFloatArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + private int alternateNextTableLocation(int tableLocation) { + return (tableLocation + 1) & (alternateTableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final FloatChunk keyChunk0 = sourceKeyChunks[0].asFloatChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final float k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + MAIN_SEARCH: while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + int alternateTableLocation = firstAlternateTableLocation; + while (alternateTableLocation < rehashPointer) { + slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + break; + } else if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break MAIN_SEARCH; + } else { + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + // NOTE: if there are other tables adding this row this cycle, we will add these into the slot + // tracker. However, when the modified slots are processed we will identify the output row as new + // for this cycle and ignore the incomplete tracker data. + mainModifiedTrackerCookieSource.set(tableLocation, EMPTY_COOKIE_SLOT); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + protected void remove(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final FloatChunk keyChunk0 = sourceKeyChunks[0].asFloatChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final float k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void shift(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta) { + final FloatChunk keyChunk0 = sourceKeyChunks[0].asFloatChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final float k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void modify(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final FloatChunk keyChunk0 = sourceKeyChunks[0].asFloatChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final float k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + private static int hash(float k0) { + int hash = FloatChunkHasher.hashInitialSingle(k0); + return hash; + } + + private boolean migrateOneLocation(int locationToMigrate) { + final int currentStateValue = alternateSlotToOutputRow.getUnsafe(locationToMigrate); + if (currentStateValue == EMPTY_OUTPUT_ROW) { + return false; + } + final float k0 = alternateKeySource0.getUnsafe(locationToMigrate); + final int hash = hash(k0); + int destinationTableLocation = hashToTableLocation(hash); + while (slotToOutputRow.getUnsafe(destinationTableLocation) != EMPTY_OUTPUT_ROW) { + destinationTableLocation = nextTableLocation(destinationTableLocation); + } + mainKeySource0.set(destinationTableLocation, k0); + slotToOutputRow.set(destinationTableLocation, currentStateValue); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(locationToMigrate); + mainModifiedTrackerCookieSource.set(destinationTableLocation, cookie); + alternateModifiedTrackerCookieSource.set(locationToMigrate, EMPTY_COOKIE_SLOT); + alternateSlotToOutputRow.set(locationToMigrate, EMPTY_OUTPUT_ROW); + return true; + } + + @Override + protected int rehashInternalPartial(int entriesToRehash) { + int rehashedEntries = 0; + while (rehashPointer > 0 && rehashedEntries < entriesToRehash) { + if (migrateOneLocation(--rehashPointer)) { + rehashedEntries++; + } + } + return rehashedEntries; + } + + @Override + protected void newAlternate() { + super.newAlternate(); + this.mainKeySource0 = (ImmutableFloatArraySource)super.mainKeySources[0]; + this.alternateKeySource0 = (ImmutableFloatArraySource)super.alternateKeySources[0]; + } + + @Override + protected void clearAlternate() { + super.clearAlternate(); + this.alternateKeySource0 = null; + } + + @Override + protected void migrateFront() { + int location = 0; + while (migrateOneLocation(location++)); + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final float[] destKeyArray0 = new float[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final float [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + final long [] oldModifiedCookie = mainModifiedTrackerCookieSource.getArray(); + final long [] destModifiedCookie = new long[tableSize]; + mainModifiedTrackerCookieSource.setArray(destModifiedCookie); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final float k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + destModifiedCookie[destinationTableLocation] = oldModifiedCookie[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherInt.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherInt.java new file mode 100644 index 00000000000..9de0920427d --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherInt.java @@ -0,0 +1,346 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.incopen.gen; + +import static io.deephaven.util.compare.IntComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.IntChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableIntArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class IncrementalMultiJoinHasherInt extends IncrementalMultiJoinStateManagerTypedBase { + private ImmutableIntArraySource mainKeySource0; + + private ImmutableIntArraySource alternateKeySource0; + + public IncrementalMultiJoinHasherInt(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableIntArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + private int alternateNextTableLocation(int tableLocation) { + return (tableLocation + 1) & (alternateTableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final IntChunk keyChunk0 = sourceKeyChunks[0].asIntChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final int k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + MAIN_SEARCH: while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + int alternateTableLocation = firstAlternateTableLocation; + while (alternateTableLocation < rehashPointer) { + slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + break; + } else if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break MAIN_SEARCH; + } else { + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + // NOTE: if there are other tables adding this row this cycle, we will add these into the slot + // tracker. However, when the modified slots are processed we will identify the output row as new + // for this cycle and ignore the incomplete tracker data. + mainModifiedTrackerCookieSource.set(tableLocation, EMPTY_COOKIE_SLOT); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + protected void remove(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final IntChunk keyChunk0 = sourceKeyChunks[0].asIntChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final int k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void shift(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta) { + final IntChunk keyChunk0 = sourceKeyChunks[0].asIntChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final int k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void modify(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final IntChunk keyChunk0 = sourceKeyChunks[0].asIntChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final int k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + private static int hash(int k0) { + int hash = IntChunkHasher.hashInitialSingle(k0); + return hash; + } + + private boolean migrateOneLocation(int locationToMigrate) { + final int currentStateValue = alternateSlotToOutputRow.getUnsafe(locationToMigrate); + if (currentStateValue == EMPTY_OUTPUT_ROW) { + return false; + } + final int k0 = alternateKeySource0.getUnsafe(locationToMigrate); + final int hash = hash(k0); + int destinationTableLocation = hashToTableLocation(hash); + while (slotToOutputRow.getUnsafe(destinationTableLocation) != EMPTY_OUTPUT_ROW) { + destinationTableLocation = nextTableLocation(destinationTableLocation); + } + mainKeySource0.set(destinationTableLocation, k0); + slotToOutputRow.set(destinationTableLocation, currentStateValue); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(locationToMigrate); + mainModifiedTrackerCookieSource.set(destinationTableLocation, cookie); + alternateModifiedTrackerCookieSource.set(locationToMigrate, EMPTY_COOKIE_SLOT); + alternateSlotToOutputRow.set(locationToMigrate, EMPTY_OUTPUT_ROW); + return true; + } + + @Override + protected int rehashInternalPartial(int entriesToRehash) { + int rehashedEntries = 0; + while (rehashPointer > 0 && rehashedEntries < entriesToRehash) { + if (migrateOneLocation(--rehashPointer)) { + rehashedEntries++; + } + } + return rehashedEntries; + } + + @Override + protected void newAlternate() { + super.newAlternate(); + this.mainKeySource0 = (ImmutableIntArraySource)super.mainKeySources[0]; + this.alternateKeySource0 = (ImmutableIntArraySource)super.alternateKeySources[0]; + } + + @Override + protected void clearAlternate() { + super.clearAlternate(); + this.alternateKeySource0 = null; + } + + @Override + protected void migrateFront() { + int location = 0; + while (migrateOneLocation(location++)); + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final int[] destKeyArray0 = new int[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final int [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + final long [] oldModifiedCookie = mainModifiedTrackerCookieSource.getArray(); + final long [] destModifiedCookie = new long[tableSize]; + mainModifiedTrackerCookieSource.setArray(destModifiedCookie); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final int k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + destModifiedCookie[destinationTableLocation] = oldModifiedCookie[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherLong.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherLong.java new file mode 100644 index 00000000000..88fe4e58f74 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherLong.java @@ -0,0 +1,345 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.incopen.gen; + +import static io.deephaven.util.compare.LongComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.LongChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableLongArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class IncrementalMultiJoinHasherLong extends IncrementalMultiJoinStateManagerTypedBase { + private ImmutableLongArraySource mainKeySource0; + + private ImmutableLongArraySource alternateKeySource0; + + public IncrementalMultiJoinHasherLong(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableLongArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + private int alternateNextTableLocation(int tableLocation) { + return (tableLocation + 1) & (alternateTableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final LongChunk keyChunk0 = sourceKeyChunks[0].asLongChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final long k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + MAIN_SEARCH: while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + int alternateTableLocation = firstAlternateTableLocation; + while (alternateTableLocation < rehashPointer) { + slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + break; + } else if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break MAIN_SEARCH; + } else { + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + // NOTE: if there are other tables adding this row this cycle, we will add these into the slot + // tracker. However, when the modified slots are processed we will identify the output row as new + // for this cycle and ignore the incomplete tracker data. + mainModifiedTrackerCookieSource.set(tableLocation, EMPTY_COOKIE_SLOT); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + protected void remove(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final LongChunk keyChunk0 = sourceKeyChunks[0].asLongChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final long k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void shift(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta) { + final LongChunk keyChunk0 = sourceKeyChunks[0].asLongChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final long k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void modify(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final LongChunk keyChunk0 = sourceKeyChunks[0].asLongChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final long k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + private static int hash(long k0) { + int hash = LongChunkHasher.hashInitialSingle(k0); + return hash; + } + + private boolean migrateOneLocation(int locationToMigrate) { + final int currentStateValue = alternateSlotToOutputRow.getUnsafe(locationToMigrate); + if (currentStateValue == EMPTY_OUTPUT_ROW) { + return false; + } + final long k0 = alternateKeySource0.getUnsafe(locationToMigrate); + final int hash = hash(k0); + int destinationTableLocation = hashToTableLocation(hash); + while (slotToOutputRow.getUnsafe(destinationTableLocation) != EMPTY_OUTPUT_ROW) { + destinationTableLocation = nextTableLocation(destinationTableLocation); + } + mainKeySource0.set(destinationTableLocation, k0); + slotToOutputRow.set(destinationTableLocation, currentStateValue); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(locationToMigrate); + mainModifiedTrackerCookieSource.set(destinationTableLocation, cookie); + alternateModifiedTrackerCookieSource.set(locationToMigrate, EMPTY_COOKIE_SLOT); + alternateSlotToOutputRow.set(locationToMigrate, EMPTY_OUTPUT_ROW); + return true; + } + + @Override + protected int rehashInternalPartial(int entriesToRehash) { + int rehashedEntries = 0; + while (rehashPointer > 0 && rehashedEntries < entriesToRehash) { + if (migrateOneLocation(--rehashPointer)) { + rehashedEntries++; + } + } + return rehashedEntries; + } + + @Override + protected void newAlternate() { + super.newAlternate(); + this.mainKeySource0 = (ImmutableLongArraySource)super.mainKeySources[0]; + this.alternateKeySource0 = (ImmutableLongArraySource)super.alternateKeySources[0]; + } + + @Override + protected void clearAlternate() { + super.clearAlternate(); + this.alternateKeySource0 = null; + } + + @Override + protected void migrateFront() { + int location = 0; + while (migrateOneLocation(location++)); + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final long[] destKeyArray0 = new long[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final long [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + final long [] oldModifiedCookie = mainModifiedTrackerCookieSource.getArray(); + final long [] destModifiedCookie = new long[tableSize]; + mainModifiedTrackerCookieSource.setArray(destModifiedCookie); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final long k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + destModifiedCookie[destinationTableLocation] = oldModifiedCookie[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherObject.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherObject.java new file mode 100644 index 00000000000..eda05056088 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherObject.java @@ -0,0 +1,348 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.incopen.gen; + +import static io.deephaven.util.compare.ObjectComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.ObjectChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableObjectArraySource; +import java.lang.Object; +import java.lang.Override; +import java.util.Arrays; + +final class IncrementalMultiJoinHasherObject extends IncrementalMultiJoinStateManagerTypedBase { + private ImmutableObjectArraySource mainKeySource0; + + private ImmutableObjectArraySource alternateKeySource0; + + public IncrementalMultiJoinHasherObject(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableObjectArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + private int alternateNextTableLocation(int tableLocation) { + return (tableLocation + 1) & (alternateTableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final ObjectChunk keyChunk0 = sourceKeyChunks[0].asObjectChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final Object k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + MAIN_SEARCH: while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + int alternateTableLocation = firstAlternateTableLocation; + while (alternateTableLocation < rehashPointer) { + slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + break; + } else if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break MAIN_SEARCH; + } else { + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + // NOTE: if there are other tables adding this row this cycle, we will add these into the slot + // tracker. However, when the modified slots are processed we will identify the output row as new + // for this cycle and ignore the incomplete tracker data. + mainModifiedTrackerCookieSource.set(tableLocation, EMPTY_COOKIE_SLOT); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + protected void remove(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final ObjectChunk keyChunk0 = sourceKeyChunks[0].asObjectChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final Object k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void shift(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta) { + final ObjectChunk keyChunk0 = sourceKeyChunks[0].asObjectChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final Object k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void modify(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final ObjectChunk keyChunk0 = sourceKeyChunks[0].asObjectChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final Object k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + private static int hash(Object k0) { + int hash = ObjectChunkHasher.hashInitialSingle(k0); + return hash; + } + + private boolean migrateOneLocation(int locationToMigrate) { + final int currentStateValue = alternateSlotToOutputRow.getUnsafe(locationToMigrate); + if (currentStateValue == EMPTY_OUTPUT_ROW) { + return false; + } + final Object k0 = alternateKeySource0.getUnsafe(locationToMigrate); + final int hash = hash(k0); + int destinationTableLocation = hashToTableLocation(hash); + while (slotToOutputRow.getUnsafe(destinationTableLocation) != EMPTY_OUTPUT_ROW) { + destinationTableLocation = nextTableLocation(destinationTableLocation); + } + mainKeySource0.set(destinationTableLocation, k0); + alternateKeySource0.set(locationToMigrate, null); + slotToOutputRow.set(destinationTableLocation, currentStateValue); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(locationToMigrate); + mainModifiedTrackerCookieSource.set(destinationTableLocation, cookie); + alternateModifiedTrackerCookieSource.set(locationToMigrate, EMPTY_COOKIE_SLOT); + alternateSlotToOutputRow.set(locationToMigrate, EMPTY_OUTPUT_ROW); + return true; + } + + @Override + protected int rehashInternalPartial(int entriesToRehash) { + int rehashedEntries = 0; + while (rehashPointer > 0 && rehashedEntries < entriesToRehash) { + if (migrateOneLocation(--rehashPointer)) { + rehashedEntries++; + } + } + return rehashedEntries; + } + + @Override + protected void newAlternate() { + super.newAlternate(); + this.mainKeySource0 = (ImmutableObjectArraySource)super.mainKeySources[0]; + this.alternateKeySource0 = (ImmutableObjectArraySource)super.alternateKeySources[0]; + } + + @Override + protected void clearAlternate() { + super.clearAlternate(); + this.alternateKeySource0 = null; + } + + @Override + protected void migrateFront() { + int location = 0; + while (migrateOneLocation(location++)); + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final Object[] destKeyArray0 = new Object[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final Object [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + final long [] oldModifiedCookie = mainModifiedTrackerCookieSource.getArray(); + final long [] destModifiedCookie = new long[tableSize]; + mainModifiedTrackerCookieSource.setArray(destModifiedCookie); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final Object k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + destModifiedCookie[destinationTableLocation] = oldModifiedCookie[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherShort.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherShort.java new file mode 100644 index 00000000000..fc38ea3cfe6 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/IncrementalMultiJoinHasherShort.java @@ -0,0 +1,346 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.incopen.gen; + +import static io.deephaven.util.compare.ShortComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.ShortChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.MultiJoinModifiedSlotTracker; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableShortArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class IncrementalMultiJoinHasherShort extends IncrementalMultiJoinStateManagerTypedBase { + private ImmutableShortArraySource mainKeySource0; + + private ImmutableShortArraySource alternateKeySource0; + + public IncrementalMultiJoinHasherShort(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableShortArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + private int alternateNextTableLocation(int tableLocation) { + return (tableLocation + 1) & (alternateTableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final ShortChunk keyChunk0 = sourceKeyChunks[0].asShortChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final short k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + MAIN_SEARCH: while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + int alternateTableLocation = firstAlternateTableLocation; + while (alternateTableLocation < rehashPointer) { + slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + break; + } else if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break MAIN_SEARCH; + } else { + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + // NOTE: if there are other tables adding this row this cycle, we will add these into the slot + // tracker. However, when the modified slots are processed we will identify the output row as new + // for this cycle and ignore the incomplete tracker data. + mainModifiedTrackerCookieSource.set(tableLocation, EMPTY_COOKIE_SLOT); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + if (modifiedSlotTracker != null) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, RowSequence.NULL_ROW_KEY, trackerFlag)); + } + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + protected void remove(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final ShortChunk keyChunk0 = sourceKeyChunks[0].asShortChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final short k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + tableRedirSource.set(slotValue, NO_REDIRECTION); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void shift(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag, long shiftDelta) { + final ShortChunk keyChunk0 = sourceKeyChunks[0].asShortChunk(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final short k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long mappedRowKey = tableRedirSource.getUnsafe(slotValue); + Assert.eq(rowKeyChunk.get(chunkPosition), "rowKey", mappedRowKey, "mappedRowKey"); + tableRedirSource.set(slotValue, mappedRowKey + shiftDelta); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.addSlot(cookie, slotValue, tableNumber, mappedRowKey, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + protected void modify(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, int tableNumber, + MultiJoinModifiedSlotTracker modifiedSlotTracker, byte trackerFlag) { + final ShortChunk keyChunk0 = sourceKeyChunks[0].asShortChunk(); + final int chunkSize = keyChunk0.size(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final short k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + boolean found = false; + int tableLocation = firstTableLocation; + int slotValue; + while ((slotValue = slotToOutputRow.getUnsafe(tableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + final long cookie = mainModifiedTrackerCookieSource.getUnsafe(tableLocation); + mainModifiedTrackerCookieSource.set(tableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + found = true; + break; + } + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + if (!found) { + final int firstAlternateTableLocation = hashToTableLocationAlternate(hash); + boolean alternateFound = false; + if (firstAlternateTableLocation < rehashPointer) { + int alternateTableLocation = firstAlternateTableLocation; + while ((slotValue = alternateSlotToOutputRow.getUnsafe(alternateTableLocation)) != EMPTY_OUTPUT_ROW) { + if (eq(alternateKeySource0.getUnsafe(alternateTableLocation), k0)) { + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(alternateTableLocation); + alternateModifiedTrackerCookieSource.set(alternateTableLocation, modifiedSlotTracker.modifySlot(cookie, slotValue, tableNumber, trackerFlag)); + alternateFound = true; + break; + } + alternateTableLocation = alternateNextTableLocation(alternateTableLocation); + Assert.neq(alternateTableLocation, "alternateTableLocation", firstAlternateTableLocation, "firstAlternateTableLocation"); + } + } + if (!alternateFound) { + throw new IllegalStateException("Matching row not found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + } + } + } + + private static int hash(short k0) { + int hash = ShortChunkHasher.hashInitialSingle(k0); + return hash; + } + + private boolean migrateOneLocation(int locationToMigrate) { + final int currentStateValue = alternateSlotToOutputRow.getUnsafe(locationToMigrate); + if (currentStateValue == EMPTY_OUTPUT_ROW) { + return false; + } + final short k0 = alternateKeySource0.getUnsafe(locationToMigrate); + final int hash = hash(k0); + int destinationTableLocation = hashToTableLocation(hash); + while (slotToOutputRow.getUnsafe(destinationTableLocation) != EMPTY_OUTPUT_ROW) { + destinationTableLocation = nextTableLocation(destinationTableLocation); + } + mainKeySource0.set(destinationTableLocation, k0); + slotToOutputRow.set(destinationTableLocation, currentStateValue); + final long cookie = alternateModifiedTrackerCookieSource.getUnsafe(locationToMigrate); + mainModifiedTrackerCookieSource.set(destinationTableLocation, cookie); + alternateModifiedTrackerCookieSource.set(locationToMigrate, EMPTY_COOKIE_SLOT); + alternateSlotToOutputRow.set(locationToMigrate, EMPTY_OUTPUT_ROW); + return true; + } + + @Override + protected int rehashInternalPartial(int entriesToRehash) { + int rehashedEntries = 0; + while (rehashPointer > 0 && rehashedEntries < entriesToRehash) { + if (migrateOneLocation(--rehashPointer)) { + rehashedEntries++; + } + } + return rehashedEntries; + } + + @Override + protected void newAlternate() { + super.newAlternate(); + this.mainKeySource0 = (ImmutableShortArraySource)super.mainKeySources[0]; + this.alternateKeySource0 = (ImmutableShortArraySource)super.alternateKeySources[0]; + } + + @Override + protected void clearAlternate() { + super.clearAlternate(); + this.alternateKeySource0 = null; + } + + @Override + protected void migrateFront() { + int location = 0; + while (migrateOneLocation(location++)); + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final short[] destKeyArray0 = new short[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final short [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + final long [] oldModifiedCookie = mainModifiedTrackerCookieSource.getArray(); + final long [] destModifiedCookie = new long[tableSize]; + mainModifiedTrackerCookieSource.setArray(destModifiedCookie); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final short k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + destModifiedCookie[destinationTableLocation] = oldModifiedCookie[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/TypedHashDispatcher.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/TypedHashDispatcher.java new file mode 100644 index 00000000000..43af5b40f79 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/incopen/gen/TypedHashDispatcher.java @@ -0,0 +1,44 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.replicators.ReplicateTypedHashers +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.incopen.gen; + +import io.deephaven.chunk.ChunkType; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import java.util.Arrays; + +/** + * The TypedHashDispatcher returns a pre-generated and precompiled hasher instance suitable for the provided column sources, or null if there is not a precompiled hasher suitable for the specified sources. + */ +public class TypedHashDispatcher { + private TypedHashDispatcher() { + // static use only + } + + public static IncrementalMultiJoinStateManagerTypedBase dispatch(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + final ChunkType[] chunkTypes = Arrays.stream(tableKeySources).map(ColumnSource::getChunkType).toArray(ChunkType[]::new);; + if (chunkTypes.length == 1) { + return dispatchSingle(chunkTypes[0], tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + } + return null; + } + + private static IncrementalMultiJoinStateManagerTypedBase dispatchSingle(ChunkType chunkType, + ColumnSource[] tableKeySources, ColumnSource[] originalTableKeySources, int tableSize, + double maximumLoadFactor, double targetLoadFactor) { + switch (chunkType) { + default: throw new UnsupportedOperationException("Invalid chunk type for typed hashers: " + chunkType); + case Char: return new IncrementalMultiJoinHasherChar(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Byte: return new IncrementalMultiJoinHasherByte(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Short: return new IncrementalMultiJoinHasherShort(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Int: return new IncrementalMultiJoinHasherInt(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Long: return new IncrementalMultiJoinHasherLong(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Float: return new IncrementalMultiJoinHasherFloat(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Double: return new IncrementalMultiJoinHasherDouble(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Object: return new IncrementalMultiJoinHasherObject(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherByte.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherByte.java new file mode 100644 index 00000000000..6caae305042 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherByte.java @@ -0,0 +1,106 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen; + +import static io.deephaven.util.compare.ByteComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.ByteChunk; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.ByteChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableByteArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class StaticMultiJoinHasherByte extends StaticMultiJoinStateManagerTypedBase { + private final ImmutableByteArraySource mainKeySource0; + + public StaticMultiJoinHasherByte(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableByteArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, long tableNumber) { + final ByteChunk keyChunk0 = sourceKeyChunks[0].asByteChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final byte k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + private static int hash(byte k0) { + int hash = ByteChunkHasher.hashInitialSingle(k0); + return hash; + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final byte[] destKeyArray0 = new byte[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final byte [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final byte k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherChar.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherChar.java new file mode 100644 index 00000000000..cfc9875695c --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherChar.java @@ -0,0 +1,106 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen; + +import static io.deephaven.util.compare.CharComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.CharChunk; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.CharChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableCharArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class StaticMultiJoinHasherChar extends StaticMultiJoinStateManagerTypedBase { + private final ImmutableCharArraySource mainKeySource0; + + public StaticMultiJoinHasherChar(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableCharArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, long tableNumber) { + final CharChunk keyChunk0 = sourceKeyChunks[0].asCharChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final char k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + private static int hash(char k0) { + int hash = CharChunkHasher.hashInitialSingle(k0); + return hash; + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final char[] destKeyArray0 = new char[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final char [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final char k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherDouble.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherDouble.java new file mode 100644 index 00000000000..89392db9fc8 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherDouble.java @@ -0,0 +1,106 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen; + +import static io.deephaven.util.compare.DoubleComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.DoubleChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.DoubleChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableDoubleArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class StaticMultiJoinHasherDouble extends StaticMultiJoinStateManagerTypedBase { + private final ImmutableDoubleArraySource mainKeySource0; + + public StaticMultiJoinHasherDouble(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableDoubleArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, long tableNumber) { + final DoubleChunk keyChunk0 = sourceKeyChunks[0].asDoubleChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final double k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + private static int hash(double k0) { + int hash = DoubleChunkHasher.hashInitialSingle(k0); + return hash; + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final double[] destKeyArray0 = new double[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final double [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final double k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherFloat.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherFloat.java new file mode 100644 index 00000000000..44be0a065e5 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherFloat.java @@ -0,0 +1,106 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen; + +import static io.deephaven.util.compare.FloatComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.FloatChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.FloatChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableFloatArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class StaticMultiJoinHasherFloat extends StaticMultiJoinStateManagerTypedBase { + private final ImmutableFloatArraySource mainKeySource0; + + public StaticMultiJoinHasherFloat(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableFloatArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, long tableNumber) { + final FloatChunk keyChunk0 = sourceKeyChunks[0].asFloatChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final float k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + private static int hash(float k0) { + int hash = FloatChunkHasher.hashInitialSingle(k0); + return hash; + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final float[] destKeyArray0 = new float[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final float [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final float k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherInt.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherInt.java new file mode 100644 index 00000000000..76b1f54308c --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherInt.java @@ -0,0 +1,106 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen; + +import static io.deephaven.util.compare.IntComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.IntChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableIntArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class StaticMultiJoinHasherInt extends StaticMultiJoinStateManagerTypedBase { + private final ImmutableIntArraySource mainKeySource0; + + public StaticMultiJoinHasherInt(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableIntArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, long tableNumber) { + final IntChunk keyChunk0 = sourceKeyChunks[0].asIntChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final int k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + private static int hash(int k0) { + int hash = IntChunkHasher.hashInitialSingle(k0); + return hash; + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final int[] destKeyArray0 = new int[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final int [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final int k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherLong.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherLong.java new file mode 100644 index 00000000000..dd4a1d30a90 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherLong.java @@ -0,0 +1,105 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen; + +import static io.deephaven.util.compare.LongComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.LongChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableLongArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class StaticMultiJoinHasherLong extends StaticMultiJoinStateManagerTypedBase { + private final ImmutableLongArraySource mainKeySource0; + + public StaticMultiJoinHasherLong(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableLongArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, long tableNumber) { + final LongChunk keyChunk0 = sourceKeyChunks[0].asLongChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final long k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + private static int hash(long k0) { + int hash = LongChunkHasher.hashInitialSingle(k0); + return hash; + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final long[] destKeyArray0 = new long[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final long [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final long k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherObject.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherObject.java new file mode 100644 index 00000000000..b4fef3aa9b4 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherObject.java @@ -0,0 +1,107 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen; + +import static io.deephaven.util.compare.ObjectComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.ObjectChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableObjectArraySource; +import java.lang.Object; +import java.lang.Override; +import java.util.Arrays; + +final class StaticMultiJoinHasherObject extends StaticMultiJoinStateManagerTypedBase { + private final ImmutableObjectArraySource mainKeySource0; + + public StaticMultiJoinHasherObject(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableObjectArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, long tableNumber) { + final ObjectChunk keyChunk0 = sourceKeyChunks[0].asObjectChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final Object k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + private static int hash(Object k0) { + int hash = ObjectChunkHasher.hashInitialSingle(k0); + return hash; + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final Object[] destKeyArray0 = new Object[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final Object [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final Object k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherShort.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherShort.java new file mode 100644 index 00000000000..44d950ee178 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/StaticMultiJoinHasherShort.java @@ -0,0 +1,106 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.engine.table.impl.by.typed.TypedHasherFactory +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen; + +import static io.deephaven.util.compare.ShortComparisons.eq; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.hashing.ShortChunkHasher; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.immutable.ImmutableShortArraySource; +import java.lang.Override; +import java.util.Arrays; + +final class StaticMultiJoinHasherShort extends StaticMultiJoinStateManagerTypedBase { + private final ImmutableShortArraySource mainKeySource0; + + public StaticMultiJoinHasherShort(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + super(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor); + this.mainKeySource0 = (ImmutableShortArraySource) super.mainKeySources[0]; + this.mainKeySource0.ensureCapacity(tableSize); + } + + private int nextTableLocation(int tableLocation) { + return (tableLocation + 1) & (tableSize - 1); + } + + protected void buildFromTable(RowSequence rowSequence, Chunk[] sourceKeyChunks, + LongArraySource tableRedirSource, long tableNumber) { + final ShortChunk keyChunk0 = sourceKeyChunks[0].asShortChunk(); + final int chunkSize = keyChunk0.size(); + final LongChunk rowKeyChunk = rowSequence.asRowKeyChunk(); + for (int chunkPosition = 0; chunkPosition < chunkSize; ++chunkPosition) { + final short k0 = keyChunk0.get(chunkPosition); + final int hash = hash(k0); + final int firstTableLocation = hashToTableLocation(hash); + int tableLocation = firstTableLocation; + while (true) { + int slotValue = slotToOutputRow.getUnsafe(tableLocation); + if (slotValue == EMPTY_OUTPUT_ROW) { + numEntries++; + mainKeySource0.set(tableLocation, k0); + final int outputKey = numEntries - 1; + slotToOutputRow.set(tableLocation, outputKey); + tableRedirSource.set(outputKey, rowKeyChunk.get(chunkPosition)); + outputKeySources[0].set((long)outputKey, k0); + break; + } else if (eq(mainKeySource0.getUnsafe(tableLocation), k0)) { + if (tableRedirSource.getLong(slotValue) != NO_REDIRECTION) { + throw new IllegalStateException("Duplicate key found for " + keyString(sourceKeyChunks, chunkPosition) + " in table " + tableNumber + "."); + } + tableRedirSource.set(slotValue, rowKeyChunk.get(chunkPosition)); + break; + } else { + tableLocation = nextTableLocation(tableLocation); + Assert.neq(tableLocation, "tableLocation", firstTableLocation, "firstTableLocation"); + } + } + } + } + + private static int hash(short k0) { + int hash = ShortChunkHasher.hashInitialSingle(k0); + return hash; + } + + @Override + protected void rehashInternalFull(final int oldSize) { + final short[] destKeyArray0 = new short[tableSize]; + final int[] destState = new int[tableSize]; + Arrays.fill(destState, EMPTY_OUTPUT_ROW); + final short [] originalKeyArray0 = mainKeySource0.getArray(); + mainKeySource0.setArray(destKeyArray0); + final int [] originalStateArray = slotToOutputRow.getArray(); + slotToOutputRow.setArray(destState); + for (int sourceBucket = 0; sourceBucket < oldSize; ++sourceBucket) { + final int currentStateValue = originalStateArray[sourceBucket]; + if (currentStateValue == EMPTY_OUTPUT_ROW) { + continue; + } + final short k0 = originalKeyArray0[sourceBucket]; + final int hash = hash(k0); + final int firstDestinationTableLocation = hashToTableLocation(hash); + int destinationTableLocation = firstDestinationTableLocation; + while (true) { + if (destState[destinationTableLocation] == EMPTY_OUTPUT_ROW) { + destKeyArray0[destinationTableLocation] = k0; + destState[destinationTableLocation] = originalStateArray[sourceBucket]; + break; + } + destinationTableLocation = nextTableLocation(destinationTableLocation); + Assert.neq(destinationTableLocation, "destinationTableLocation", firstDestinationTableLocation, "firstDestinationTableLocation"); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/TypedHashDispatcher.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/TypedHashDispatcher.java new file mode 100644 index 00000000000..8c4899f215f --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/multijoin/typed/staticopen/gen/TypedHashDispatcher.java @@ -0,0 +1,44 @@ +// DO NOT EDIT THIS CLASS, AUTOMATICALLY GENERATED BY io.deephaven.replicators.ReplicateTypedHashers +// Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.multijoin.typed.staticopen.gen; + +import io.deephaven.chunk.ChunkType; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; +import java.util.Arrays; + +/** + * The TypedHashDispatcher returns a pre-generated and precompiled hasher instance suitable for the provided column sources, or null if there is not a precompiled hasher suitable for the specified sources. + */ +public class TypedHashDispatcher { + private TypedHashDispatcher() { + // static use only + } + + public static StaticMultiJoinStateManagerTypedBase dispatch(ColumnSource[] tableKeySources, + ColumnSource[] originalTableKeySources, int tableSize, double maximumLoadFactor, + double targetLoadFactor) { + final ChunkType[] chunkTypes = Arrays.stream(tableKeySources).map(ColumnSource::getChunkType).toArray(ChunkType[]::new);; + if (chunkTypes.length == 1) { + return dispatchSingle(chunkTypes[0], tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + } + return null; + } + + private static StaticMultiJoinStateManagerTypedBase dispatchSingle(ChunkType chunkType, + ColumnSource[] tableKeySources, ColumnSource[] originalTableKeySources, int tableSize, + double maximumLoadFactor, double targetLoadFactor) { + switch (chunkType) { + default: throw new UnsupportedOperationException("Invalid chunk type for typed hashers: " + chunkType); + case Char: return new StaticMultiJoinHasherChar(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Byte: return new StaticMultiJoinHasherByte(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Short: return new StaticMultiJoinHasherShort(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Int: return new StaticMultiJoinHasherInt(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Long: return new StaticMultiJoinHasherLong(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Float: return new StaticMultiJoinHasherFloat(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Double: return new StaticMultiJoinHasherDouble(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + case Object: return new StaticMultiJoinHasherObject(tableKeySources, originalTableKeySources, tableSize, maximumLoadFactor, targetLoadFactor); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanArraySource.java index 9618ebdc540..fff05e16d92 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanArraySource.java @@ -4,7 +4,6 @@ package io.deephaven.engine.table.impl.sources; import gnu.trove.list.array.TIntArrayList; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.WritableSourceWithPrepareForParallelPopulation; @@ -256,7 +255,10 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); indices.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { - ctx.currentBlockNo = getBlockNo(v); + if (v > maxIndex) { + dest.set(ctx.offset++, null); + return; + } ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; ctx.currentBlock = blocks[ctx.currentBlockNo]; } @@ -282,6 +284,10 @@ protected void fillSparsePrevChunk(@NotNull final WritableChunk final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); indices.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { + if (v > maxIndex) { + dest.set(ctx.offset++, null); + return; + } ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; ctx.currentBlock = blocks[ctx.currentBlockNo]; @@ -579,6 +585,10 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); indices.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { + if (v > maxIndex) { + dest.set(ctx.offset++, NULL_BOOLEAN_AS_BYTE); + return; + } ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; ctx.currentBlock = blocks[ctx.currentBlockNo]; @@ -604,6 +614,10 @@ protected void fillSparsePrevChunk(@NotNull final WritableChunk final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); indices.forAllRowKeys((final long v) -> { if (v >= ctx.capForCurrentBlock) { + if (v > maxIndex) { + dest.set(ctx.offset++, NULL_BOOLEAN_AS_BYTE); + return; + } ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; ctx.currentBlock = blocks[ctx.currentBlockNo]; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java index 8742a5c7429..9670417fcb4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteArraySource.java @@ -12,7 +12,6 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; @@ -27,6 +26,7 @@ import org.jetbrains.annotations.NotNull; import java.util.Arrays; +import java.util.function.LongConsumer; import static io.deephaven.util.QueryConstants.NULL_BYTE; import static io.deephaven.util.type.TypeUtils.box; @@ -37,7 +37,7 @@ *

* The C-haracterArraySource is replicated to all other types with * io.deephaven.engine.table.impl.sources.Replicate. - * + *

* (C-haracter is deliberately spelled that way in order to prevent Replicate from altering this very comment). */ public class ByteArraySource extends ArraySourceHelper @@ -326,7 +326,22 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu final WritableByteChunk chunk = destination.asWritableByteChunk(); // endregion chunkDecl MutableInt destOffset = new MutableInt(0); - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } + if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -354,6 +369,11 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu // endregion copyFromArray destOffset.add(restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -402,7 +422,20 @@ private interface CopyFromBlockFunctor { destOffset.add(length); }; - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } else if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -420,6 +453,11 @@ private interface CopyFromBlockFunctor { int restSz = (int) (to & INDEX_MASK) + 1; lambda.copy(toBlock, 0, restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -439,7 +477,7 @@ private interface CopyFromBlockFunctor { final WritableByteChunk chunk = destGeneric.asWritableByteChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -448,7 +486,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, ctx.currentBlock[(int) (v & INDEX_MASK)]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparseChunk @@ -474,7 +525,7 @@ private interface CopyFromBlockFunctor { final WritableByteChunk chunk = destGeneric.asWritableByteChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -482,7 +533,6 @@ private interface CopyFromBlockFunctor { ctx.currentPrevBlock = prevBlocks[ctx.currentBlockNo]; ctx.prevInUseBlock = prevInUse[ctx.currentBlockNo]; } - final int indexWithinBlock = (int) (v & INDEX_MASK); final int indexWithinInUse = indexWithinBlock >> LOG_INUSE_BITSET_SIZE; final long maskWithinInUse = 1L << (indexWithinBlock & IN_USE_MASK); @@ -490,7 +540,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, usePrev ? ctx.currentPrevBlock[indexWithinBlock] : ctx.currentBlock[indexWithinBlock]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparsePrevChunk diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java index ade3bae143b..f88f02b9ae7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterArraySource.java @@ -7,7 +7,6 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; @@ -22,6 +21,7 @@ import org.jetbrains.annotations.NotNull; import java.util.Arrays; +import java.util.function.LongConsumer; import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.type.TypeUtils.box; @@ -32,7 +32,7 @@ *

* The C-haracterArraySource is replicated to all other types with * io.deephaven.engine.table.impl.sources.Replicate. - * + *

* (C-haracter is deliberately spelled that way in order to prevent Replicate from altering this very comment). */ public class CharacterArraySource extends ArraySourceHelper @@ -321,7 +321,22 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu final WritableCharChunk chunk = destination.asWritableCharChunk(); // endregion chunkDecl MutableInt destOffset = new MutableInt(0); - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } + if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -349,6 +364,11 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu // endregion copyFromArray destOffset.add(restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -397,7 +417,20 @@ private interface CopyFromBlockFunctor { destOffset.add(length); }; - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } else if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -415,6 +448,11 @@ private interface CopyFromBlockFunctor { int restSz = (int) (to & INDEX_MASK) + 1; lambda.copy(toBlock, 0, restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -434,7 +472,7 @@ private interface CopyFromBlockFunctor { final WritableCharChunk chunk = destGeneric.asWritableCharChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -443,7 +481,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, ctx.currentBlock[(int) (v & INDEX_MASK)]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparseChunk @@ -469,7 +520,7 @@ private interface CopyFromBlockFunctor { final WritableCharChunk chunk = destGeneric.asWritableCharChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -477,7 +528,6 @@ private interface CopyFromBlockFunctor { ctx.currentPrevBlock = prevBlocks[ctx.currentBlockNo]; ctx.prevInUseBlock = prevInUse[ctx.currentBlockNo]; } - final int indexWithinBlock = (int) (v & INDEX_MASK); final int indexWithinInUse = indexWithinBlock >> LOG_INUSE_BITSET_SIZE; final long maskWithinInUse = 1L << (indexWithinBlock & IN_USE_MASK); @@ -485,7 +535,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, usePrev ? ctx.currentPrevBlock[indexWithinBlock] : ctx.currentBlock[indexWithinBlock]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparsePrevChunk diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java index 7f042028274..4f848a4b46d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleArraySource.java @@ -12,7 +12,6 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; @@ -27,6 +26,7 @@ import org.jetbrains.annotations.NotNull; import java.util.Arrays; +import java.util.function.LongConsumer; import static io.deephaven.util.QueryConstants.NULL_DOUBLE; import static io.deephaven.util.type.TypeUtils.box; @@ -37,7 +37,7 @@ *

* The C-haracterArraySource is replicated to all other types with * io.deephaven.engine.table.impl.sources.Replicate. - * + *

* (C-haracter is deliberately spelled that way in order to prevent Replicate from altering this very comment). */ public class DoubleArraySource extends ArraySourceHelper @@ -326,7 +326,22 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu final WritableDoubleChunk chunk = destination.asWritableDoubleChunk(); // endregion chunkDecl MutableInt destOffset = new MutableInt(0); - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } + if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -354,6 +369,11 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu // endregion copyFromArray destOffset.add(restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -402,7 +422,20 @@ private interface CopyFromBlockFunctor { destOffset.add(length); }; - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } else if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -420,6 +453,11 @@ private interface CopyFromBlockFunctor { int restSz = (int) (to & INDEX_MASK) + 1; lambda.copy(toBlock, 0, restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -439,7 +477,7 @@ private interface CopyFromBlockFunctor { final WritableDoubleChunk chunk = destGeneric.asWritableDoubleChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -448,7 +486,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, ctx.currentBlock[(int) (v & INDEX_MASK)]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparseChunk @@ -474,7 +525,7 @@ private interface CopyFromBlockFunctor { final WritableDoubleChunk chunk = destGeneric.asWritableDoubleChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -482,7 +533,6 @@ private interface CopyFromBlockFunctor { ctx.currentPrevBlock = prevBlocks[ctx.currentBlockNo]; ctx.prevInUseBlock = prevInUse[ctx.currentBlockNo]; } - final int indexWithinBlock = (int) (v & INDEX_MASK); final int indexWithinInUse = indexWithinBlock >> LOG_INUSE_BITSET_SIZE; final long maskWithinInUse = 1L << (indexWithinBlock & IN_USE_MASK); @@ -490,7 +540,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, usePrev ? ctx.currentPrevBlock[indexWithinBlock] : ctx.currentBlock[indexWithinBlock]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparsePrevChunk diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java index 3c7845aa663..2dcaaefff41 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatArraySource.java @@ -12,7 +12,6 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; @@ -27,6 +26,7 @@ import org.jetbrains.annotations.NotNull; import java.util.Arrays; +import java.util.function.LongConsumer; import static io.deephaven.util.QueryConstants.NULL_FLOAT; import static io.deephaven.util.type.TypeUtils.box; @@ -37,7 +37,7 @@ *

* The C-haracterArraySource is replicated to all other types with * io.deephaven.engine.table.impl.sources.Replicate. - * + *

* (C-haracter is deliberately spelled that way in order to prevent Replicate from altering this very comment). */ public class FloatArraySource extends ArraySourceHelper @@ -326,7 +326,22 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu final WritableFloatChunk chunk = destination.asWritableFloatChunk(); // endregion chunkDecl MutableInt destOffset = new MutableInt(0); - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } + if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -354,6 +369,11 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu // endregion copyFromArray destOffset.add(restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -402,7 +422,20 @@ private interface CopyFromBlockFunctor { destOffset.add(length); }; - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } else if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -420,6 +453,11 @@ private interface CopyFromBlockFunctor { int restSz = (int) (to & INDEX_MASK) + 1; lambda.copy(toBlock, 0, restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -439,7 +477,7 @@ private interface CopyFromBlockFunctor { final WritableFloatChunk chunk = destGeneric.asWritableFloatChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -448,7 +486,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, ctx.currentBlock[(int) (v & INDEX_MASK)]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparseChunk @@ -474,7 +525,7 @@ private interface CopyFromBlockFunctor { final WritableFloatChunk chunk = destGeneric.asWritableFloatChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -482,7 +533,6 @@ private interface CopyFromBlockFunctor { ctx.currentPrevBlock = prevBlocks[ctx.currentBlockNo]; ctx.prevInUseBlock = prevInUse[ctx.currentBlockNo]; } - final int indexWithinBlock = (int) (v & INDEX_MASK); final int indexWithinInUse = indexWithinBlock >> LOG_INUSE_BITSET_SIZE; final long maskWithinInUse = 1L << (indexWithinBlock & IN_USE_MASK); @@ -490,7 +540,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, usePrev ? ctx.currentPrevBlock[indexWithinBlock] : ctx.currentBlock[indexWithinBlock]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparsePrevChunk diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java index b0d678656a3..fff12f546cd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerArraySource.java @@ -12,7 +12,6 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; @@ -27,6 +26,7 @@ import org.jetbrains.annotations.NotNull; import java.util.Arrays; +import java.util.function.LongConsumer; import static io.deephaven.util.QueryConstants.NULL_INT; import static io.deephaven.util.type.TypeUtils.box; @@ -37,7 +37,7 @@ *

* The C-haracterArraySource is replicated to all other types with * io.deephaven.engine.table.impl.sources.Replicate. - * + *

* (C-haracter is deliberately spelled that way in order to prevent Replicate from altering this very comment). */ public class IntegerArraySource extends ArraySourceHelper @@ -326,7 +326,22 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu final WritableIntChunk chunk = destination.asWritableIntChunk(); // endregion chunkDecl MutableInt destOffset = new MutableInt(0); - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } + if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -354,6 +369,11 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu // endregion copyFromArray destOffset.add(restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -402,7 +422,20 @@ private interface CopyFromBlockFunctor { destOffset.add(length); }; - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } else if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -420,6 +453,11 @@ private interface CopyFromBlockFunctor { int restSz = (int) (to & INDEX_MASK) + 1; lambda.copy(toBlock, 0, restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -439,7 +477,7 @@ private interface CopyFromBlockFunctor { final WritableIntChunk chunk = destGeneric.asWritableIntChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -448,7 +486,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, ctx.currentBlock[(int) (v & INDEX_MASK)]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparseChunk @@ -474,7 +525,7 @@ private interface CopyFromBlockFunctor { final WritableIntChunk chunk = destGeneric.asWritableIntChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -482,7 +533,6 @@ private interface CopyFromBlockFunctor { ctx.currentPrevBlock = prevBlocks[ctx.currentBlockNo]; ctx.prevInUseBlock = prevInUse[ctx.currentBlockNo]; } - final int indexWithinBlock = (int) (v & INDEX_MASK); final int indexWithinInUse = indexWithinBlock >> LOG_INUSE_BITSET_SIZE; final long maskWithinInUse = 1L << (indexWithinBlock & IN_USE_MASK); @@ -490,7 +540,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, usePrev ? ctx.currentPrevBlock[indexWithinBlock] : ctx.currentBlock[indexWithinBlock]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparsePrevChunk diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongArraySource.java index 96d09cca8a3..29c03927b8b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongArraySource.java @@ -23,7 +23,6 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; @@ -38,6 +37,7 @@ import org.jetbrains.annotations.NotNull; import java.util.Arrays; +import java.util.function.LongConsumer; import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.type.TypeUtils.box; @@ -48,7 +48,7 @@ *

* The C-haracterArraySource is replicated to all other types with * io.deephaven.engine.table.impl.sources.Replicate. - * + *

* (C-haracter is deliberately spelled that way in order to prevent Replicate from altering this very comment). */ public class LongArraySource extends ArraySourceHelper @@ -349,7 +349,22 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu final WritableLongChunk chunk = destination.asWritableLongChunk(); // endregion chunkDecl MutableInt destOffset = new MutableInt(0); - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } + if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -377,6 +392,11 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu // endregion copyFromArray destOffset.add(restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -394,7 +414,22 @@ public void fillChunk( final WritableObjectChunk chunk = destination.asWritableObjectChunk(); // endregion chunkDecl MutableInt destOffset = new MutableInt(0); - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } + if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -442,6 +477,11 @@ public void fillChunk( // endregion copyFromArray destOffset.add(restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -490,7 +530,20 @@ private interface CopyFromBlockFunctor { destOffset.add(length); }; - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } else if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -508,6 +561,11 @@ private interface CopyFromBlockFunctor { int restSz = (int) (to & INDEX_MASK) + 1; lambda.copy(toBlock, 0, restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -578,7 +636,20 @@ public void fillPrevChunk( destOffset.add(length); }; - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } else if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -596,6 +667,11 @@ public void fillPrevChunk( int restSz = (int) (to & INDEX_MASK) + 1; lambda.copy(toBlock, 0, restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -615,7 +691,7 @@ public void fillPrevChunk( final WritableLongChunk chunk = destGeneric.asWritableLongChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -624,7 +700,20 @@ public void fillPrevChunk( // region conversion chunk.set(ctx.offset++, ctx.currentBlock[(int) (v & INDEX_MASK)]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } @@ -640,7 +729,7 @@ protected void fillSparseChunk( final WritableObjectChunk chunk = destGeneric.asWritableObjectChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -649,7 +738,20 @@ protected void fillSparseChunk( // region conversion chunk.set(ctx.offset++,converter.apply( ctx.currentBlock[(int) (v & INDEX_MASK)])); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparseChunk @@ -675,7 +777,7 @@ protected void fillSparseChunk( final WritableLongChunk chunk = destGeneric.asWritableLongChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -683,7 +785,6 @@ protected void fillSparseChunk( ctx.currentPrevBlock = prevBlocks[ctx.currentBlockNo]; ctx.prevInUseBlock = prevInUse[ctx.currentBlockNo]; } - final int indexWithinBlock = (int) (v & INDEX_MASK); final int indexWithinInUse = indexWithinBlock >> LOG_INUSE_BITSET_SIZE; final long maskWithinInUse = 1L << (indexWithinBlock & IN_USE_MASK); @@ -691,7 +792,20 @@ protected void fillSparseChunk( // region conversion chunk.set(ctx.offset++, usePrev ? ctx.currentPrevBlock[indexWithinBlock] : ctx.currentBlock[indexWithinBlock]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } @@ -714,7 +828,7 @@ protected void fillSparsePrevChunk( final WritableObjectChunk chunk = destGeneric.asWritableObjectChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -722,7 +836,6 @@ protected void fillSparsePrevChunk( ctx.currentPrevBlock = prevBlocks[ctx.currentBlockNo]; ctx.prevInUseBlock = prevInUse[ctx.currentBlockNo]; } - final int indexWithinBlock = (int) (v & INDEX_MASK); final int indexWithinInUse = indexWithinBlock >> LOG_INUSE_BITSET_SIZE; final long maskWithinInUse = 1L << (indexWithinBlock & IN_USE_MASK); @@ -730,7 +843,20 @@ protected void fillSparsePrevChunk( // region conversion chunk.set(ctx.offset++,converter.apply( usePrev ? ctx.currentPrevBlock[indexWithinBlock] : ctx.currentBlock[indexWithinBlock])); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparsePrevChunk diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectArraySource.java index b42c4592a1e..221139de49a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectArraySource.java @@ -5,16 +5,19 @@ import gnu.trove.list.array.TIntArrayList; import io.deephaven.base.verify.Assert; -import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.vector.Vector; import io.deephaven.chunk.*; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.util.SoftRecycler; +import org.apache.commons.lang3.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import java.util.Arrays; @@ -236,6 +239,132 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu return capacity; } + @Override + public void fillChunk( + @NotNull final ChunkSource.FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + if (rowSequence.getAverageRunLengthEstimate() < USE_RANGES_AVERAGE_RUN_LENGTH) { + fillSparseChunk(destination, rowSequence); + return; + } + MutableInt destOffset = new MutableInt(0); + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } + if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + + final int fromBlock = getBlockNo(from); + final int toBlock = getBlockNo(to); + final int fromOffsetInBlock = (int) (from & INDEX_MASK); + if (fromBlock == toBlock) { + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.copyFromArray(getBlock(fromBlock), fromOffsetInBlock, destOffset.intValue(), sz); + destOffset.add(sz); + } else { + final int sz = BLOCK_SIZE - fromOffsetInBlock; + destination.copyFromArray(getBlock(fromBlock), fromOffsetInBlock, destOffset.intValue(), sz); + destOffset.add(sz); + for (int blockNo = fromBlock + 1; blockNo < toBlock; ++blockNo) { + destination.copyFromArray(getBlock(blockNo), 0, destOffset.intValue(), BLOCK_SIZE); + destOffset.add(BLOCK_SIZE); + } + int restSz = (int) (to & INDEX_MASK) + 1; + destination.copyFromArray(getBlock(toBlock), 0, destOffset.intValue(), restSz); + destOffset.add(restSz); + } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } + }); + destination.setSize(destOffset.intValue()); + } + + private interface CopyFromBlockFunctor { + void copy(int blockNo, int srcOffset, int length); + } + + @Override + public void fillPrevChunk( + @NotNull final ColumnSource.FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { + if (prevFlusher == null) { + fillChunk(context, destination, rowSequence); + return; + } + + if (rowSequence.getAverageRunLengthEstimate() < USE_RANGES_AVERAGE_RUN_LENGTH) { + fillSparsePrevChunk(destination, rowSequence); + return; + } + + final ArraySourceHelper.FillContext effectiveContext = (ArraySourceHelper.FillContext) context; + final MutableInt destOffset = new MutableInt(0); + + CopyFromBlockFunctor lambda = (blockNo, srcOffset, length) -> { + final long[] inUse = prevInUse[blockNo]; + if (inUse != null) { + effectiveContext.copyKernel.conditionalCopy(destination, getBlock(blockNo), getPrevBlock(blockNo), + inUse, srcOffset, destOffset.intValue(), length); + } else { + destination.copyFromArray(getBlock(blockNo), srcOffset, destOffset.intValue(), length); + } + destOffset.add(length); + }; + + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } else if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + + final int fromBlock = getBlockNo(from); + final int toBlock = getBlockNo(to); + final int fromOffsetInBlock = (int) (from & INDEX_MASK); + if (fromBlock == toBlock) { + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + lambda.copy(fromBlock, fromOffsetInBlock, sz); + } else { + final int sz = BLOCK_SIZE - fromOffsetInBlock; + lambda.copy(fromBlock, fromOffsetInBlock, sz); + + for (int blockNo = fromBlock + 1; blockNo < toBlock; ++blockNo) { + lambda.copy(blockNo, 0, BLOCK_SIZE); + } + + int restSz = (int) (to & INDEX_MASK) + 1; + lambda.copy(toBlock, 0, restSz); + } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } + }); + destination.setSize(destOffset.intValue()); + } @Override protected void fillSparseChunk(@NotNull final WritableChunk destGeneric, @NotNull final RowSequence indices) { @@ -248,6 +377,10 @@ protected void fillSparseChunk(@NotNull final WritableChunk dest final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); indices.forEachRowKey((final long v) -> { if (v >= ctx.capForCurrentBlock) { + if (v > maxIndex) { + dest.set(ctx.offset++, null); + return true; + } ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; ctx.currentBlock = blocks[ctx.currentBlockNo]; @@ -275,6 +408,10 @@ protected void fillSparsePrevChunk(@NotNull final WritableChunk final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); indices.forEachRowKey((final long v) -> { if (v >= ctx.capForCurrentBlock) { + if (v > maxIndex) { + dest.set(ctx.offset++, null); + return true; + } ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; ctx.currentBlock = blocks[ctx.currentBlockNo]; @@ -496,10 +633,10 @@ public void move(long source, long dest, long length) { if (source == dest) { return; } - if ((source - dest) % BLOCK_SIZE == 0) { - // TODO: we can move full blocks! + if (((source - dest) & INDEX_MASK) == 0 && (source & INDEX_MASK) == 0) { + // TODO (#3359): we can move full blocks! } - if (source < dest) { + if (source < dest && source + length >= dest) { for (long ii = length - 1; ii >= 0; ) { final long sourceKey = source + ii; final long destKey = dest + ii; @@ -509,11 +646,11 @@ public void move(long source, long dest, long length) { final int destBlock = (int) (destKey >> LOG_BLOCK_SIZE); final int destIndexWithinBlock = (int) (destKey & INDEX_MASK); - blocks[destBlock][destIndexWithinBlock] = blocks[sourceBlock][sourceIndexWithinBlock]; + final int valuesInBothBlocks = Math.min(destIndexWithinBlock + 1, sourceIndexWithinBlock + 1); + final int toMove = (ii + 1) < valuesInBothBlocks ? (int)(ii + 1): valuesInBothBlocks; - // TODO: figure out the first key in both blocks, and do an array copy - - ii -= 1; + System.arraycopy(blocks[sourceBlock], sourceIndexWithinBlock - toMove + 1, blocks[destBlock], destIndexWithinBlock - toMove + 1, toMove); + ii -= toMove; } } else { for (long ii = 0; ii < length;) { @@ -525,11 +662,11 @@ public void move(long source, long dest, long length) { final int destBlock = (int) (destKey >> LOG_BLOCK_SIZE); final int destIndexWithinBlock = (int) (destKey & INDEX_MASK); - blocks[destBlock][destIndexWithinBlock] = blocks[sourceBlock][sourceIndexWithinBlock]; - - // TODO: figure out the last key in both blocks, and do an array copy + final int valuesInBothBlocks = BLOCK_SIZE - Math.max(destIndexWithinBlock, sourceIndexWithinBlock); + final int toMove = (length - ii < valuesInBothBlocks) ? (int)(length - ii): valuesInBothBlocks; - ii += 1; + System.arraycopy(blocks[sourceBlock], sourceIndexWithinBlock, blocks[destBlock], destIndexWithinBlock, toMove); + ii += toMove; } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java index 0ad59c51294..a7913b97f2d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortArraySource.java @@ -12,7 +12,6 @@ import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; @@ -27,6 +26,7 @@ import org.jetbrains.annotations.NotNull; import java.util.Arrays; +import java.util.function.LongConsumer; import static io.deephaven.util.QueryConstants.NULL_SHORT; import static io.deephaven.util.type.TypeUtils.box; @@ -37,7 +37,7 @@ *

* The C-haracterArraySource is replicated to all other types with * io.deephaven.engine.table.impl.sources.Replicate. - * + *

* (C-haracter is deliberately spelled that way in order to prevent Replicate from altering this very comment). */ public class ShortArraySource extends ArraySourceHelper @@ -326,7 +326,22 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu final WritableShortChunk chunk = destination.asWritableShortChunk(); // endregion chunkDecl MutableInt destOffset = new MutableInt(0); - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } + if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -354,6 +369,11 @@ public long resetWritableChunkToBackingStoreSlice(@NotNull ResettableWritableChu // endregion copyFromArray destOffset.add(restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -402,7 +422,20 @@ private interface CopyFromBlockFunctor { destOffset.add(length); }; - rowSequence.forAllRowKeyRanges((final long from, final long to) -> { + rowSequence.forAllRowKeyRanges((final long from, long to) -> { + int valuesAtEnd = 0; + if (from > maxIndex) { + // the whole region is beyond us + final int sz = LongSizedDataStructure.intSize("int cast", to - from + 1); + destination.fillWithNullValue(destOffset.intValue(), sz); + destOffset.add(sz); + return; + } else if (to > maxIndex) { + // only part of the region is beyond us + valuesAtEnd = LongSizedDataStructure.intSize("int cast", to - maxIndex); + to = maxIndex; + } + final int fromBlock = getBlockNo(from); final int toBlock = getBlockNo(to); final int fromOffsetInBlock = (int) (from & INDEX_MASK); @@ -420,6 +453,11 @@ private interface CopyFromBlockFunctor { int restSz = (int) (to & INDEX_MASK) + 1; lambda.copy(toBlock, 0, restSz); } + + if (valuesAtEnd > 0) { + destination.fillWithNullValue(destOffset.intValue(), valuesAtEnd); + destOffset.add(valuesAtEnd); + } }); destination.setSize(destOffset.intValue()); } @@ -439,7 +477,7 @@ private interface CopyFromBlockFunctor { final WritableShortChunk chunk = destGeneric.asWritableShortChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -448,7 +486,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, ctx.currentBlock[(int) (v & INDEX_MASK)]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparseChunk @@ -474,7 +525,7 @@ private interface CopyFromBlockFunctor { final WritableShortChunk chunk = destGeneric.asWritableShortChunk(); // endregion chunkDecl final FillSparseChunkContext ctx = new FillSparseChunkContext<>(); - rows.forAllRowKeys((final long v) -> { + final LongConsumer normalFiller = (final long v) -> { if (v >= ctx.capForCurrentBlock) { ctx.currentBlockNo = getBlockNo(v); ctx.capForCurrentBlock = (ctx.currentBlockNo + 1L) << LOG_BLOCK_SIZE; @@ -482,7 +533,6 @@ private interface CopyFromBlockFunctor { ctx.currentPrevBlock = prevBlocks[ctx.currentBlockNo]; ctx.prevInUseBlock = prevInUse[ctx.currentBlockNo]; } - final int indexWithinBlock = (int) (v & INDEX_MASK); final int indexWithinInUse = indexWithinBlock >> LOG_INUSE_BITSET_SIZE; final long maskWithinInUse = 1L << (indexWithinBlock & IN_USE_MASK); @@ -490,7 +540,20 @@ private interface CopyFromBlockFunctor { // region conversion chunk.set(ctx.offset++, usePrev ? ctx.currentPrevBlock[indexWithinBlock] : ctx.currentBlock[indexWithinBlock]); // endregion conversion - }); + }; + if (rows.lastRowKey() > maxIndex) { + final long initialPosition, firstNullPosition; + try (final RowSequence.Iterator rowsIter = rows.getRowSequenceIterator()) { + initialPosition = rowsIter.getRelativePosition(); + rowsIter.getNextRowSequenceThrough(maxIndex).forAllRowKeys(normalFiller); + firstNullPosition = rowsIter.getRelativePosition(); + } + final int trailingNullCount = Math.toIntExact(rows.size() - (firstNullPosition - initialPosition)); + chunk.fillWithNullValue(ctx.offset, trailingNullCount); + ctx.offset += trailingNullCount; + } else { + rows.forAllRowKeys(normalFiller); + } chunk.setSize(ctx.offset); } // endregion fillSparsePrevChunk diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableMultiJoinTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableMultiJoinTest.java new file mode 100644 index 00000000000..68e712ce279 --- /dev/null +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableMultiJoinTest.java @@ -0,0 +1,781 @@ +package io.deephaven.engine.table.impl; + +import gnu.trove.map.hash.TIntIntHashMap; +import io.deephaven.chunk.util.pools.ChunkPoolReleaseTracking; +import io.deephaven.datastructures.util.CollectionUtil; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.table.ModifiedColumnSet; +import io.deephaven.engine.table.MultiJoinFactory; +import io.deephaven.engine.table.MultiJoinInput; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.testutil.*; +import io.deephaven.engine.testutil.generator.*; +import io.deephaven.engine.util.PrintListener; +import io.deephaven.engine.util.TableTools; +import io.deephaven.test.types.OutOfBandTest; +import io.deephaven.time.DateTimeUtils; +import io.deephaven.util.SafeCloseable; +import org.apache.commons.lang3.mutable.MutableBoolean; +import org.junit.*; +import org.junit.experimental.categories.Category; + +import java.util.*; +import java.util.function.IntUnaryOperator; +import java.util.stream.Collectors; + +import static io.deephaven.engine.testutil.TstUtils.*; +import static io.deephaven.engine.testutil.junit4.EngineCleanup.printTableUpdates; +import static io.deephaven.engine.util.TableTools.*; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.NULL_INT; + +@Category(OutOfBandTest.class) +public class QueryTableMultiJoinTest extends QueryTableTestBase { + final static JoinControl DEFAULT_JOIN_CONTROL = new JoinControl(); + final static JoinControl REHASH_JOIN_CONTROL = new JoinControl() { + @Override + public int initialBuildSize() { + return 1 << 4; + } + }; + + final static JoinControl HIGH_LOAD_JOIN_CONTROL = new JoinControl() { + @Override + public int initialBuildSize() { + return 1 << 4; + } + + @Override + public double getMaximumLoadFactor() { + return 0.95; + } + + @Override + public double getTargetLoadFactor() { + return 0.9; + } + }; + + @Before + public void before() throws Exception { + ChunkPoolReleaseTracking.enableStrict(); + } + + @After + public void after() throws Exception { + ChunkPoolReleaseTracking.checkAndDisable(); + } + + @Test + public void testSimple() { + final Table t1 = TstUtils.testTable(col("Key", "A", "B", "C"), intCol("S1", 1, 2, 3)); + final Table t2 = TstUtils.testTable(col("Key", "C", "B", "D"), intCol("S2", 3, 4, 5)); + final Table t3 = TstUtils.testTable(col("Key", "F", "A", "B", "E"), intCol("S3", 6, 7, 8, 9), + doubleCol("S3b", 9.9, 10.1, 11.2, 12.3)); + + final Table result = MultiJoinFactory.of(new String[] {"Key"}, t1, t2, t3).table(); + TableTools.showWithRowSet(result); + + final Table expected = TableTools.newTable(col("Key", "A", "B", "C", "D", "F", "E"), + intCol("S1", 1, 2, 3, NULL_INT, NULL_INT, NULL_INT), + intCol("S2", NULL_INT, 4, 3, 5, NULL_INT, NULL_INT), intCol("S3", 7, 8, NULL_INT, NULL_INT, 6, 9), + doubleCol("S3b", 10.1, 11.2, NULL_DOUBLE, NULL_DOUBLE, 9.9, 12.3)); + + TstUtils.assertTableEquals(expected, result); + } + + @Test + public void testSimpleIncremental() { + final QueryTable t1 = TstUtils.testRefreshingTable(col("Key", "A", "B", "C"), intCol("S1", 1, 2, 3)); + final QueryTable t2 = TstUtils.testRefreshingTable(col("Key", "C", "B", "D"), intCol("S2", 3, 4, 5)); + final Table t3 = TstUtils.testTable(col("Key", "F", "A", "B", "E"), intCol("S3", 6, 7, 8, 9), + doubleCol("S3b", 9.9, 10.1, 11.2, 12.3)); + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final Table result = + updateGraph.sharedLock() + .computeLocked(() -> MultiJoinFactory.of(new String[] {"Key"}, t1, t2, t3).table()); + System.out.println("Initial multi-join."); + TableTools.showWithRowSet(result); + final PrintListener printListener = new PrintListener("multiJoin", (QueryTable) result, 10); + final TableUpdateValidator validator = TableUpdateValidator.make("multiJoin", (QueryTable) result); + final FailureListener failureListener = new FailureListener(); + validator.getResultTable().addUpdateListener(failureListener); + + final Table expected = doIterativeMultiJoin(new String[] {"Key"}, Arrays.asList(t1, t2, t3)); + + TstUtils.assertTableEquals(expected, result); + + System.out.println("Adding D, S1=4"); + updateGraph.runWithinUnitTestCycle(() -> { + final RowSet additions = RowSetFactory.fromKeys(5); + TstUtils.addToTable(t1, additions, col("Key", "D"), intCol("S1", 4)); + t1.notifyListeners(additions, RowSetFactory.empty(), RowSetFactory.empty()); + }); + + TstUtils.assertTableEquals(expected, result); + + System.out.println("Removing B, S2=4"); + updateGraph.runWithinUnitTestCycle(() -> { + final RowSet removals = RowSetFactory.fromKeys(1); + TstUtils.removeRows(t2, removals); + t2.notifyListeners(RowSetFactory.empty(), removals, RowSetFactory.empty()); + }); + TstUtils.assertTableEquals(expected, result); + TableTools.showWithRowSet(result); + + System.out.println("Modifying C, S2=3 to G,10 and D, S2=5 to B, 11"); + updateGraph.runWithinUnitTestCycle(() -> { + final RowSet modifications = RowSetFactory.fromKeys(0, 2); + TstUtils.addToTable(t2, modifications, col("Key", "G", "B"), intCol("S2", 10, 11)); + t2.notifyListeners(RowSetFactory.empty(), RowSetFactory.empty(), modifications); + }); + + updateGraph.runWithinUnitTestCycle(() -> { + TableTools.showWithRowSet(t1); + TstUtils.addToTable(t1, i(3, 4), col("Key", "C", "D"), intCol("S1", 3, 4)); + TstUtils.removeRows(t1, i(2, 5)); + + final TableUpdateImpl update = new TableUpdateImpl(); + update.modifiedColumnSet = ModifiedColumnSet.EMPTY; + update.added = RowSetFactory.empty(); + update.removed = RowSetFactory.empty(); + update.modified = RowSetFactory.empty(); + + final RowSetShiftData.Builder shiftBuilder = new RowSetShiftData.Builder(); + shiftBuilder.shiftRange(2, 2, 1); + shiftBuilder.shiftRange(5, 5, -1); + + update.shifted = shiftBuilder.build(); + + TableTools.showWithRowSet(t1); + + t1.notifyListeners(update); + }); + + TstUtils.assertTableEquals(expected, result); + + updateGraph.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(t1, i(3), col("Key", "C"), intCol("S1", 7)); + + final TableUpdateImpl update = new TableUpdateImpl(); + update.modifiedColumnSet = ModifiedColumnSet.EMPTY; + update.added = i(3); + update.removed = i(3); + update.modified = RowSetFactory.empty(); + + update.shifted = RowSetShiftData.EMPTY; + + TableTools.showWithRowSet(t1); + + t1.notifyListeners(update); + }); + + TstUtils.assertTableEquals(expected, result); + + printListener.stop(); + } + + @Test + public void testStatic() { + for (int size = 10; size <= 100_000; size *= 10) { + for (int seed = 0; seed < SEEDCOUNT.applyAsInt(size); ++seed) { + System.out.println("Size = " + size + ", seed =" + seed); + testStatic(DEFAULT_JOIN_CONTROL, size, seed, new String[] {"Key"}, new String[] {"Key2"}); + testStatic(DEFAULT_JOIN_CONTROL, size, seed, new String[] {"Key", "Key2"}, + CollectionUtil.ZERO_LENGTH_STRING_ARRAY); + testStatic(DEFAULT_JOIN_CONTROL, size, seed, new String[] {"Key", "Key2"}, + CollectionUtil.ZERO_LENGTH_STRING_ARRAY); + } + } + } + + @Test + public void testStaticOverflowAndRehash() { + for (int size = 1000; size <= 100_000; size *= 10) { + testStatic(HIGH_LOAD_JOIN_CONTROL, size, 0, new String[] {"Key", "Key2"}, + CollectionUtil.ZERO_LENGTH_STRING_ARRAY); + testStatic(REHASH_JOIN_CONTROL, size, 0, new String[] {"Key", "Key2"}, + CollectionUtil.ZERO_LENGTH_STRING_ARRAY); + } + } + + private void testStatic(JoinControl joinControl, int size, int seed, String[] keys, String[] drops) { + final Random random = new Random(seed); + final int tableCount = random.nextInt(10) + 1; + + final List

inputTables = new ArrayList<>(); + + for (int ii = 0; ii < tableCount; ++ii) { + final Table table = TstUtils + .getTable(false, size, random, + TstUtils.initColumnInfos(new String[] {"Key", "Key2", "I" + ii, "D" + ii, "B" + ii}, + new IntGenerator(0, size), new BooleanGenerator(), new IntGenerator(), + new DoubleGenerator(-1000, 1000), new BooleanGenerator(0.45, 0.1))) + .firstBy(keys).dropColumns(drops); + inputTables.add(table); + } + + if (printTableUpdates()) { + System.out.println("Table Count: " + tableCount); + for (int ii = 0; ii < tableCount; ++ii) { + TableTools.showWithRowSet(inputTables.get(ii)); + } + } + + final Table result = MultiJoinTableImpl.of(joinControl, MultiJoinInput.from(keys, + inputTables.toArray(TableDefaults.ZERO_LENGTH_TABLE_ARRAY))).table(); + final Table expected = doIterativeMultiJoin(keys, inputTables); + + if (printTableUpdates()) { + TableTools.showWithRowSet(result); + } + + TstUtils.assertTableEquals(expected, result); + } + + private static class SeedCount implements IntUnaryOperator { + final static int seedCount = 10; + final static TIntIntHashMap seedCountForSize = new TIntIntHashMap(1, 0.5f, -1, seedCount); + + static { + seedCountForSize.put(100_000, 1); + seedCountForSize.put(10_000, 1); + seedCountForSize.put(1_000, 10); + } + + @Override + public int applyAsInt(int size) { + return seedCountForSize.get(size); + } + } + + private static final SeedCount SEEDCOUNT = new SeedCount(); + + @Test + public void testIncremental() { + final int seedInitial = 0; + final int maxSteps = 20; + + for (int size = 10; size <= 10_000; size *= 10) { + for (int seed = seedInitial; seed < seedInitial + SEEDCOUNT.applyAsInt(size); ++seed) { + System.out.println("Size = " + size + ", seed = " + seed); + try (final SafeCloseable ignored = LivenessScopeStack.open()) { + testIncremental(DEFAULT_JOIN_CONTROL, size, seed, maxSteps, new String[] {"Key"}, + new String[] {"Key2"}); + testIncremental(DEFAULT_JOIN_CONTROL, size, seed, maxSteps, new String[] {"Key", "Key2"}, + CollectionUtil.ZERO_LENGTH_STRING_ARRAY); + } + } + } + } + + @Test + public void testIncrementalWithShifts() { + final int seedInitial = 0; + final int maxSteps = 10; // if we have more steps, we are more likely to run out of unique key space + + for (int size = 10; size <= 10_000; size *= 10) { + for (int seed = seedInitial; seed < seedInitial + SEEDCOUNT.applyAsInt(size); ++seed) { + System.out.println("Size = " + size + ", seed = " + seed); + try (final SafeCloseable ignored = LivenessScopeStack.open()) { + testIncrementalWithShifts(size, seed, maxSteps); + } + } + } + } + + @Test + public void testIncrementalOverflowAndRehash() { + final int seedInitial = 0; + final int maxSteps = 20; + + for (int size = 1_000; size <= 100_000; size *= 10) { + System.out.println("Size = " + size + ", seed = " + seedInitial); + try (final SafeCloseable ignored = LivenessScopeStack.open()) { + testIncremental(HIGH_LOAD_JOIN_CONTROL, size, seedInitial, maxSteps, new String[] {"Key"}, + new String[] {"Key2"}); + testIncremental(REHASH_JOIN_CONTROL, size, seedInitial, maxSteps, new String[] {"Key", "Key2"}, + CollectionUtil.ZERO_LENGTH_STRING_ARRAY); + } + } + } + + @Test + public void testIncrementalZeroKey() { + final int seedCount = 10; + for (int seed = 0; seed < seedCount; ++seed) { + System.out.println("Zero Key, seed = " + seed); + try (final SafeCloseable ignored = LivenessScopeStack.open()) { + testIncremental(DEFAULT_JOIN_CONTROL, 5, seed, 20, CollectionUtil.ZERO_LENGTH_STRING_ARRAY, + new String[] {"Key", "Key2"}); + } + } + } + + private void testIncremental(JoinControl joinControl, int size, int seed, int maxStep, String[] keys, + String[] drops) { + final Random random = new Random(seed); + final int tableCount = random.nextInt(24) + 1; + + final List inputTables = new ArrayList<>(); + final List[]> columnInfos = new ArrayList<>(); + final List printListenersForRetention = new ArrayList<>(); + + for (int ii = 0; ii < tableCount; ++ii) { + final ColumnInfo[] columnInfo; + final QueryTable table = TstUtils.getTable(true, random.nextInt(size), random, + columnInfo = TstUtils.initColumnInfos(new String[] {"Key", "Key2", "I" + ii, "D" + ii, "B" + ii}, + new IntGenerator(0, size), + new SetGenerator<>(DateTimeUtils.parseInstant("2021-11-18T09:30:00 NY"), + DateTimeUtils.parseInstant("2021-11-18T16:15:00 NY"), + DateTimeUtils.parseInstant("2021-11-18T23:59:59.999 NY")), + new IntGenerator(), new DoubleGenerator(-1000, 1000), new BooleanGenerator(0.45, 0.1))); + inputTables.add(table); + columnInfos.add(columnInfo); + } + + final MutableBoolean sortResult = new MutableBoolean(false); + + final List
lastByInputs = new ArrayList<>(); + + for (int tt = 0; tt < inputTables.size(); ++tt) { + final Table in = inputTables.get(tt); + final Table out; + + final double modType = random.nextDouble(); + if (keys.length == 0) { + if (modType < 0.5) { + if (printTableUpdates()) { + System.out.println("Table " + tt + " tail(1)"); + } + out = in.dropColumns(drops).tail(1); + } else { + if (printTableUpdates()) { + System.out.println("Table " + tt + " lastBy()"); + } + out = in.dropColumns(drops).lastBy(keys); + } + } else { + final Table lastByKeys = in.dropColumns(drops).lastBy(keys); + if (modType < 0.5 || lastByKeys.getDefinition().getColumnNames().size() < 2) { + out = lastByKeys; + if (printTableUpdates()) { + System.out.println("Table " + tt + " lastBy()"); + } + } else { + sortResult.setTrue(); + out = lastByKeys.sort(lastByKeys.getDefinition().getColumnNames().get(1)); + if (printTableUpdates()) { + System.out.println("Table " + tt + " lastBy().sort()"); + } + } + } + + lastByInputs.add(out); + } + + if (printTableUpdates()) { + System.out.println("Table Count: " + tableCount); + for (int ii = 0; ii < tableCount; ++ii) { + System.out.println("Input " + ii + ":"); + TableTools.showWithRowSet(inputTables.get(ii)); + } + for (int ii = 0; ii < tableCount; ++ii) { + System.out.println("Last By " + ii + ":"); + TableTools.showWithRowSet(lastByInputs.get(ii)); + printListenersForRetention + .add(new PrintListener("Last By Result " + ii, (QueryTable) lastByInputs.get(ii), 10)); + } + } + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final Table expected = doIterativeMultiJoin(keys, lastByInputs); + final Table result = updateGraph.sharedLock() + .computeLocked(() -> MultiJoinTableImpl + .of(joinControl, + MultiJoinInput.from(keys, lastByInputs.toArray(TableDefaults.ZERO_LENGTH_TABLE_ARRAY))) + .table()); + + if (printTableUpdates()) { + System.out.println("Initial result:"); + TableTools.showWithRowSet(result); + printListenersForRetention.add(new PrintListener("Multi-join result", (QueryTable) result, 10)); + } + + final TableUpdateValidator validator = TableUpdateValidator.make("result validator", (QueryTable) result); + final FailureListener listener = new FailureListener(); + validator.getResultTable().addUpdateListener(listener); + + TstUtils.assertTableEquals(expected, result); + + + for (int step = 0; step < maxStep; ++step) { + if (printTableUpdates()) { + System.out.println("Step = " + step); + } + updateGraph.runWithinUnitTestCycle(() -> { + if (random.nextBoolean()) { + for (int tt = 0; tt < tableCount; ++tt) { + if (random.nextBoolean()) { + GenerateTableUpdates.generateShiftAwareTableUpdates(GenerateTableUpdates.DEFAULT_PROFILE, + size, random, inputTables.get(tt), columnInfos.get(tt)); + } + } + } else { + final int tableToUpdate = random.nextInt(tableCount); + GenerateTableUpdates.generateShiftAwareTableUpdates(GenerateTableUpdates.DEFAULT_PROFILE, size, + random, inputTables.get(tableToUpdate), columnInfos.get(tableToUpdate)); + } + }); + + if (printTableUpdates()) { + System.out.println("Multi-join result step=" + step + ": " + result.size()); + TableTools.showWithRowSet(result, 20); + } + + if (sortResult.booleanValue()) { + TstUtils.assertTableEquals(expected.sort(keys), result.sort(keys)); + } else { + TstUtils.assertTableEquals(expected, result); + } + } + } + + private void testIncrementalWithShifts(int size, int seed, int maxStep) { + final Random random = new Random(seed); + final int tableCount = random.nextInt(24) + 1; + + final List inputTables = new ArrayList<>(); + final List[]> columnInfos = new ArrayList<>(); + + for (int ii = 0; ii < tableCount; ++ii) { + final ColumnInfo[] columnInfo; + final QueryTable table = TstUtils.getTable(true, random.nextInt(size), random, + columnInfo = TstUtils.initColumnInfos(new String[] {"Key", "I" + ii}, + new UniqueIntGenerator(0, size * 8), new IntGenerator())); + inputTables.add(table); + columnInfos.add(columnInfo); + } + + if (printTableUpdates()) { + System.out.println("Table Count: " + tableCount); + for (int ii = 0; ii < tableCount; ++ii) { + System.out.println("Input " + ii + ":"); + TableTools.showWithRowSet(inputTables.get(ii)); + } + } + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final Table expected = doIterativeMultiJoin(new String[] {"Key"}, inputTables); + final Table result = updateGraph.sharedLock().computeLocked(() -> MultiJoinFactory.of(new String[] {"Key"}, + inputTables.toArray(TableDefaults.ZERO_LENGTH_TABLE_ARRAY)).table()); + + + if (printTableUpdates()) { + System.out.println("Initial result:"); + TableTools.showWithRowSet(result); + } + TstUtils.assertTableEquals(expected, result); + + final TableUpdateValidator validator = TableUpdateValidator.make((QueryTable) result); + final FailureListener validatorListener = new FailureListener(); + validator.getResultTable().addUpdateListener(validatorListener); + + final Table expectSorted = expected.sort("Key"); + final Table resultSorted = result.sort("Key"); + + for (int step = 0; step < maxStep; ++step) { + if (printTableUpdates()) { + System.out.println("Step = " + step); + } + updateGraph.runWithinUnitTestCycle(() -> { + if (random.nextBoolean()) { + for (int tt = 0; tt < tableCount; ++tt) { + if (random.nextBoolean()) { + GenerateTableUpdates.generateShiftAwareTableUpdates(GenerateTableUpdates.DEFAULT_PROFILE, + size, random, inputTables.get(tt), columnInfos.get(tt)); + } + } + } else { + final int tableToUpdate = random.nextInt(tableCount); + GenerateTableUpdates.generateShiftAwareTableUpdates(GenerateTableUpdates.DEFAULT_PROFILE, size, + random, inputTables.get(tableToUpdate), columnInfos.get(tableToUpdate)); + } + }); + + if (printTableUpdates()) { + System.out.println("Multi-join result step=" + step + ": " + result.size()); + TableTools.showWithRowSet(result, 20); + } + + TstUtils.assertTableEquals(expectSorted, resultSorted); + } + } + + @Test + public void testZeroKeyStatic() { + int seed = 0; + final Random random = new Random(seed); + final int tableCount = 5; + + final List
inputTables = new ArrayList<>(); + + for (int ii = 0; ii < tableCount; ++ii) { + final Table table = TstUtils + .getTable(false, 10, random, TstUtils.initColumnInfos(new String[] {"I" + ii, "D" + ii, "B" + ii}, + new IntGenerator(), new DoubleGenerator(-1000, 1000), new BooleanGenerator(0.45, 0.1))) + .firstBy(); + inputTables.add(table); + } + + if (printTableUpdates()) { + System.out.println("Table Count: " + tableCount); + for (int ii = 0; ii < tableCount; ++ii) { + TableTools.showWithRowSet(inputTables.get(ii)); + } + } + + final Table result = MultiJoinFactory.of(CollectionUtil.ZERO_LENGTH_STRING_ARRAY, + inputTables.toArray(TableDefaults.ZERO_LENGTH_TABLE_ARRAY)).table(); + final Table expected = doIterativeMultiJoin(CollectionUtil.ZERO_LENGTH_STRING_ARRAY, inputTables); + + if (printTableUpdates()) { + TableTools.showWithRowSet(result); + } + + TstUtils.assertTableEquals(expected, result); + } + + @Test + public void testDuplicateKeys() { + final Table t1 = TableTools.newTable(intCol("C1", 1, 2), intCol("C2", 1, 1), intCol("S1", 10, 11)); + final Table t2 = TableTools.newTable(intCol("C3", 2, 2), intCol("C4", 1, 2)); + + final Table joined = MultiJoinTableImpl.of(MultiJoinInput.of(t1, "Key=C1", "S1")).table(); + assertTableEquals(TableTools.newTable(intCol("Key", 1, 2), intCol("S1", 10, 11)), joined); + + try { + MultiJoinTableImpl.of(MultiJoinInput.of(t1, "Key=C1", "S1"), + MultiJoinInput.of(t2, "Key=C3", "C4")).table(); + Assert.fail("expected exception"); + } catch (IllegalStateException e) { + Assert.assertEquals("Duplicate key found for 2 in table 1.", e.getMessage()); + } + + try { + MultiJoinTableImpl.of(MultiJoinInput.of(t1, "", "S1")).table(); + Assert.fail("expected exception"); + } catch (IllegalStateException e) { + Assert.assertEquals("Duplicate rows for table 0 on zero-key multiJoin.", e.getMessage()); + } + + t1.setRefreshing(true); + + final QueryTable t2r = TstUtils.testRefreshingTable(intCol("C3", 2, 2), intCol("C4", 1, 2), intCol("S2", 20, + 21)); + + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final Table joinedR = updateGraph.sharedLock().computeLocked(() -> MultiJoinTableImpl.of( + MultiJoinInput.of(t1, "Key=C1", "S1"), + MultiJoinInput.of(t2r, "Key=C4", "S2")).table()); + assertTableEquals(TableTools.newTable(intCol("Key", 1, 2), intCol("S1", 10, 11), intCol("S2", 20, 21)), + joinedR); + + try { + updateGraph.sharedLock().doLocked(() -> MultiJoinTableImpl.of( + MultiJoinInput.of(t1, "Key=C1", "S1"), + MultiJoinInput.of(t2r, "Key=C3", "C4")).table()); + Assert.fail("expected exception"); + } catch (IllegalStateException e) { + Assert.assertEquals("Duplicate key found for 2 in table 1.", e.getMessage()); + } + + allowingError(() -> { + updateGraph.runWithinUnitTestCycle(() -> { + addToTable(t2r, i(2), intCol("C3", 1), intCol("C4", 2), intCol("S2", 22)); + t2r.notifyListeners(i(2), i(), i()); + }); + }, (lot) -> { + if (lot == null || lot.size() != 1) { + return false; + } + final Throwable throwable = lot.get(0); + return throwable instanceof IllegalStateException + && throwable.getMessage().equals("Duplicate key found for 2 in table 1."); + }); + } + + @Test + public void testDuplicateZeroKey() { + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final QueryTable t3 = TstUtils.testRefreshingTable(intCol("S1")); + final Table j3 = updateGraph.sharedLock() + .computeLocked(() -> MultiJoinTableImpl.of(MultiJoinInput.of(t3, null, "S1"))) + .table(); + assertTableEquals(TableTools.newTable(intCol("S1")), j3); + updateGraph.runWithinUnitTestCycle(() -> { + addToTable(t3, i(0), intCol("S1", 1)); + t3.notifyListeners(i(0), i(), i()); + }); + assertTableEquals(TableTools.newTable(intCol("S1", 1)), j3); + + allowingError(() -> { + updateGraph.runWithinUnitTestCycle(() -> { + addToTable(t3, i(2), intCol("S1", 2)); + t3.notifyListeners(i(2), i(), i()); + }); + }, (lot) -> { + if (lot == null || lot.size() != 1) { + return false; + } + final Throwable throwable = lot.get(0); + return throwable instanceof IllegalStateException + && throwable.getMessage().equals("Multiple rows in 0 for zero-key multiJoin."); + }); + } + + @Test + public void testZeroKeyTransitions() { + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final QueryTable t3 = TstUtils.testRefreshingTable(intCol("S1")); + final Table j3 = updateGraph.sharedLock().computeLocked(() -> MultiJoinTableImpl + .of(MultiJoinInput.of(t3, "", "S1")).table()); + final TableUpdateValidator validator = TableUpdateValidator.make("testZeroKeyTransitions", (QueryTable) j3); + final FailureListener failureListener = new FailureListener(); + validator.getResultTable().addUpdateListener(failureListener); + + assertTableEquals(TableTools.newTable(intCol("S1")), j3); + updateGraph.runWithinUnitTestCycle(() -> { + addToTable(t3, i(0), intCol("S1", 1)); + t3.notifyListeners(i(0), i(), i()); + }); + assertTableEquals(TableTools.newTable(intCol("S1", 1)), j3); + + updateGraph.runWithinUnitTestCycle(() -> { + addToTable(t3, i(1), intCol("S1", 1)); + removeRows(t3, i(0)); + final RowSetShiftData.Builder builder = new RowSetShiftData.Builder(); + builder.shiftRange(0, 0, 1); + final RowSetShiftData shift = builder.build(); + final ModifiedColumnSet modifiedColumnSet = t3.newModifiedColumnSet("S1"); + t3.notifyListeners(new TableUpdateImpl(i(), i(), i(), shift, modifiedColumnSet)); + }); + assertTableEquals(TableTools.newTable(intCol("S1", 1)), j3); + + updateGraph.runWithinUnitTestCycle(() -> { + addToTable(t3, i(2), intCol("S1", 2)); + removeRows(t3, i(1)); + final RowSetShiftData.Builder builder = new RowSetShiftData.Builder(); + builder.shiftRange(1, 1, 1); + final RowSetShiftData shift = builder.build(); + final ModifiedColumnSet modifiedColumnSet = t3.newModifiedColumnSet("S1"); + t3.notifyListeners(new TableUpdateImpl(i(), i(), i(2), shift, modifiedColumnSet)); + }); + assertTableEquals(TableTools.newTable(intCol("S1", 2)), j3); + + updateGraph.runWithinUnitTestCycle(() -> { + removeRows(t3, i(2)); + t3.notifyListeners(i(), i(2), i()); + }); + assertTableEquals(TableTools.newTable(intCol("S1")), j3); + } + + @Test + public void testColumnConflicts() { + final Table t1 = TableTools.newTable(col("A", "a", "b"), intCol("B", 1, 2)); + final Table t2 = TableTools.newTable(col("A", "a", "b"), intCol("D", 3, 4), doubleCol("C", 3.0, 4.0)); + + try { + MultiJoinFactory.of(); + Assert.fail("expected exception"); + } catch (IllegalArgumentException iae) { + Assert.assertEquals(iae.getMessage(), "At least one table must be included in MultiJoinTable."); + } + + try { + MultiJoinTableImpl.of( + MultiJoinInput.of(t1, "Key=B", "A"), + MultiJoinInput.of(t2, "KeyNotTheSame=C", "A")); + Assert.fail("expected exception"); + } catch (IllegalArgumentException iae) { + Assert.assertEquals(iae.getMessage(), + "Key column mismatch for table 1, first table has key columns=[Key], this table has [KeyNotTheSame]"); + } + + try { + MultiJoinTableImpl.of( + MultiJoinInput.of(t1, "Key=B", "A"), + MultiJoinInput.of(t2, "Key=D", "A")); + Assert.fail("expected exception"); + } catch (IllegalArgumentException iae) { + Assert.assertEquals("Column A defined in table 0 and table 1", iae.getMessage()); + } + + try { + MultiJoinTableImpl.of( + MultiJoinInput.of(t1, "Key=B", "A"), + MultiJoinInput.of(t2, "Key=D", "Key")); + Assert.fail("expected exception"); + } catch (IllegalArgumentException iae) { + Assert.assertEquals("Column Key defined in table key columns and table 1", iae.getMessage()); + } + + try { + MultiJoinTableImpl.of( + MultiJoinInput.of(t1, "Key=B", "A"), + MultiJoinInput.of(t2, "Key=C", "D")); + Assert.fail("expected exception"); + } catch (IllegalArgumentException iae) { + Assert.assertEquals( + "Key column type mismatch for table 1, first table has key column types=[int], this table has [double]", + iae.getMessage()); + } + + try { + MultiJoinTableImpl.of( + MultiJoinInput.of(t1, null, "A"), + MultiJoinInput.of(t2, "Key=C", "D")); + Assert.fail("expected exception"); + } catch (IllegalArgumentException iae) { + Assert.assertEquals( + "Key column mismatch for table 1, first table has key columns=[], this table has [Key]", + iae.getMessage()); + } + + try { + MultiJoinTableImpl.of( + MultiJoinInput.of(t1, null, "A"), + MultiJoinInput.of(t2, null, "A")); + Assert.fail("expected exception"); + } catch (IllegalArgumentException iae) { + Assert.assertEquals("Column A defined in table 0 and table 1", iae.getMessage()); + } + } + + private Table doIterativeMultiJoin(String[] keyColumns, List inputTables) { + final List
keyTables = inputTables.stream() + .map(t -> keyColumns.length == 0 ? t.dropColumns(t.getDefinition().getColumnNames()).view("Dummy=1") + : t.view(keyColumns)) + .collect(Collectors.toList()); + final Table base = TableTools.merge(keyTables).selectDistinct(keyColumns); + final String columnNames = String.join(",", keyColumns); + + Table result = base; + for (Table inputTable : inputTables) { + result = result.naturalJoin(inputTable, columnNames); + } + + return result; + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestBooleanArraySource.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestBooleanArraySource.java index bb5ef0483d6..6e2336f4e0e 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestBooleanArraySource.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestBooleanArraySource.java @@ -39,7 +39,7 @@ import java.util.stream.LongStream; import static io.deephaven.chunk.ArrayGenerator.indexDataGenerator; -import static io.deephaven.util.QueryConstants.NULL_BOOLEAN; +import static io.deephaven.util.QueryConstants.*; import static junit.framework.TestCase.*; public class TestBooleanArraySource { @@ -282,6 +282,7 @@ private void testFillChunkGeneric(byte[] values, byte[] newValues, int chunkSize } } + // region validate with fill private void validateValuesWithFill(int chunkSize, byte[] values, RowSet rowSet, BooleanArraySource source) { try (final RowSequence.Iterator rsIterator = rowSet.getRowSequenceIterator(); final RowSet.Iterator it = rowSet.iterator(); @@ -296,7 +297,7 @@ private void validateValuesWithFill(int chunkSize, byte[] values, RowSet rowSet, assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getBoolean(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : BooleanUtils.NULL_BOOLEAN_AS_BYTE, chunk.get(i)); pos++; } } @@ -318,13 +319,14 @@ private void validatePrevValuesWithFill(int chunkSize, byte[] values, RowSet row assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getPrevBoolean(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : BooleanUtils.NULL_BOOLEAN_AS_BYTE, chunk.get(i)); pos++; } } assertEquals(pos, rowSet.size()); } } + // endregion validate with fill @Test public void testFillChunk() { @@ -344,6 +346,10 @@ public void testFillChunk() { testFillChunkGeneric(ArrayGenerator.randomBooleans(random, 512), ArrayGenerator.randomBooleans(random, 512), 4, RowSetFactory.fromKeys(254, 255, 256, 257)); testFillChunkGeneric(ArrayGenerator.randomBooleans(random, 512), ArrayGenerator.randomBooleans(random, 512), 5, RowSetFactory.fromKeys(254, 255, 256, 257, 258)); + // Test the fill with null behavior when requesting keys outside of source. + testFillChunkGeneric(ArrayGenerator.randomBooleans(random, 512), ArrayGenerator.randomBooleans(random, 4096), 4096, RowSetFactory.fromRange(4096, 8192)); + testFillChunkGeneric(ArrayGenerator.randomBooleans(random, 512), ArrayGenerator.randomBooleans(random, 4096), 4096, RowSetFactory.flat(4096)); + for (int sourceSize = 32; sourceSize < 8192; sourceSize *= 4) { for (int v = -4; v < 5; v += 2) { testSetForGivenSourceSize(random, sourceSize + v); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestByteArraySource.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestByteArraySource.java index de686cfa793..afb3842f7a7 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestByteArraySource.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestByteArraySource.java @@ -34,7 +34,7 @@ import java.util.stream.LongStream; import static io.deephaven.chunk.ArrayGenerator.indexDataGenerator; -import static io.deephaven.util.QueryConstants.NULL_BYTE; +import static io.deephaven.util.QueryConstants.*; import static junit.framework.TestCase.*; public class TestByteArraySource { @@ -284,6 +284,7 @@ private void testFillChunkGeneric(byte[] values, byte[] newValues, int chunkSize } } + // region validate with fill private void validateValuesWithFill(int chunkSize, byte[] values, RowSet rowSet, ByteArraySource source) { try (final RowSequence.Iterator rsIterator = rowSet.getRowSequenceIterator(); final RowSet.Iterator it = rowSet.iterator(); @@ -298,7 +299,7 @@ private void validateValuesWithFill(int chunkSize, byte[] values, RowSet rowSet, assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getByte(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_BYTE, chunk.get(i)); pos++; } } @@ -320,13 +321,14 @@ private void validatePrevValuesWithFill(int chunkSize, byte[] values, RowSet row assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getPrevByte(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_BYTE, chunk.get(i)); pos++; } } assertEquals(pos, rowSet.size()); } } + // endregion validate with fill @Test public void testFillChunk() { @@ -346,6 +348,10 @@ public void testFillChunk() { testFillChunkGeneric(ArrayGenerator.randomBytes(random, 512), ArrayGenerator.randomBytes(random, 512), 4, RowSetFactory.fromKeys(254, 255, 256, 257)); testFillChunkGeneric(ArrayGenerator.randomBytes(random, 512), ArrayGenerator.randomBytes(random, 512), 5, RowSetFactory.fromKeys(254, 255, 256, 257, 258)); + // Test the fill with null behavior when requesting keys outside of source. + testFillChunkGeneric(ArrayGenerator.randomBytes(random, 512), ArrayGenerator.randomBytes(random, 4096), 4096, RowSetFactory.fromRange(4096, 8192)); + testFillChunkGeneric(ArrayGenerator.randomBytes(random, 512), ArrayGenerator.randomBytes(random, 4096), 4096, RowSetFactory.flat(4096)); + for (int sourceSize = 32; sourceSize < 8192; sourceSize *= 4) { for (int v = -4; v < 5; v += 2) { testSetForGivenSourceSize(random, sourceSize + v); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestCharacterArraySource.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestCharacterArraySource.java index 3af3137cca1..2822fe9588f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestCharacterArraySource.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestCharacterArraySource.java @@ -29,7 +29,7 @@ import java.util.stream.LongStream; import static io.deephaven.chunk.ArrayGenerator.indexDataGenerator; -import static io.deephaven.util.QueryConstants.NULL_CHAR; +import static io.deephaven.util.QueryConstants.*; import static junit.framework.TestCase.*; public class TestCharacterArraySource { @@ -279,6 +279,7 @@ private void testFillChunkGeneric(char[] values, char[] newValues, int chunkSize } } + // region validate with fill private void validateValuesWithFill(int chunkSize, char[] values, RowSet rowSet, CharacterArraySource source) { try (final RowSequence.Iterator rsIterator = rowSet.getRowSequenceIterator(); final RowSet.Iterator it = rowSet.iterator(); @@ -293,7 +294,7 @@ private void validateValuesWithFill(int chunkSize, char[] values, RowSet rowSet, assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getChar(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_CHAR, chunk.get(i)); pos++; } } @@ -315,13 +316,14 @@ private void validatePrevValuesWithFill(int chunkSize, char[] values, RowSet row assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getPrevChar(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_CHAR, chunk.get(i)); pos++; } } assertEquals(pos, rowSet.size()); } } + // endregion validate with fill @Test public void testFillChunk() { @@ -341,6 +343,10 @@ public void testFillChunk() { testFillChunkGeneric(ArrayGenerator.randomChars(random, 512), ArrayGenerator.randomChars(random, 512), 4, RowSetFactory.fromKeys(254, 255, 256, 257)); testFillChunkGeneric(ArrayGenerator.randomChars(random, 512), ArrayGenerator.randomChars(random, 512), 5, RowSetFactory.fromKeys(254, 255, 256, 257, 258)); + // Test the fill with null behavior when requesting keys outside of source. + testFillChunkGeneric(ArrayGenerator.randomChars(random, 512), ArrayGenerator.randomChars(random, 4096), 4096, RowSetFactory.fromRange(4096, 8192)); + testFillChunkGeneric(ArrayGenerator.randomChars(random, 512), ArrayGenerator.randomChars(random, 4096), 4096, RowSetFactory.flat(4096)); + for (int sourceSize = 32; sourceSize < 8192; sourceSize *= 4) { for (int v = -4; v < 5; v += 2) { testSetForGivenSourceSize(random, sourceSize + v); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestDoubleArraySource.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestDoubleArraySource.java index 0c39185fbd2..ad1b8d60cb2 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestDoubleArraySource.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestDoubleArraySource.java @@ -34,7 +34,7 @@ import java.util.stream.LongStream; import static io.deephaven.chunk.ArrayGenerator.indexDataGenerator; -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.*; import static junit.framework.TestCase.*; public class TestDoubleArraySource { @@ -284,6 +284,7 @@ private void testFillChunkGeneric(double[] values, double[] newValues, int chunk } } + // region validate with fill private void validateValuesWithFill(int chunkSize, double[] values, RowSet rowSet, DoubleArraySource source) { try (final RowSequence.Iterator rsIterator = rowSet.getRowSequenceIterator(); final RowSet.Iterator it = rowSet.iterator(); @@ -298,7 +299,7 @@ private void validateValuesWithFill(int chunkSize, double[] values, RowSet rowSe assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getDouble(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_DOUBLE, chunk.get(i)); pos++; } } @@ -320,13 +321,14 @@ private void validatePrevValuesWithFill(int chunkSize, double[] values, RowSet r assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getPrevDouble(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_DOUBLE, chunk.get(i)); pos++; } } assertEquals(pos, rowSet.size()); } } + // endregion validate with fill @Test public void testFillChunk() { @@ -346,6 +348,10 @@ public void testFillChunk() { testFillChunkGeneric(ArrayGenerator.randomDoubles(random, 512), ArrayGenerator.randomDoubles(random, 512), 4, RowSetFactory.fromKeys(254, 255, 256, 257)); testFillChunkGeneric(ArrayGenerator.randomDoubles(random, 512), ArrayGenerator.randomDoubles(random, 512), 5, RowSetFactory.fromKeys(254, 255, 256, 257, 258)); + // Test the fill with null behavior when requesting keys outside of source. + testFillChunkGeneric(ArrayGenerator.randomDoubles(random, 512), ArrayGenerator.randomDoubles(random, 4096), 4096, RowSetFactory.fromRange(4096, 8192)); + testFillChunkGeneric(ArrayGenerator.randomDoubles(random, 512), ArrayGenerator.randomDoubles(random, 4096), 4096, RowSetFactory.flat(4096)); + for (int sourceSize = 32; sourceSize < 8192; sourceSize *= 4) { for (int v = -4; v < 5; v += 2) { testSetForGivenSourceSize(random, sourceSize + v); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestFloatArraySource.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestFloatArraySource.java index 21860b0ca81..ee3eca370e3 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestFloatArraySource.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestFloatArraySource.java @@ -34,7 +34,7 @@ import java.util.stream.LongStream; import static io.deephaven.chunk.ArrayGenerator.indexDataGenerator; -import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.*; import static junit.framework.TestCase.*; public class TestFloatArraySource { @@ -284,6 +284,7 @@ private void testFillChunkGeneric(float[] values, float[] newValues, int chunkSi } } + // region validate with fill private void validateValuesWithFill(int chunkSize, float[] values, RowSet rowSet, FloatArraySource source) { try (final RowSequence.Iterator rsIterator = rowSet.getRowSequenceIterator(); final RowSet.Iterator it = rowSet.iterator(); @@ -298,7 +299,7 @@ private void validateValuesWithFill(int chunkSize, float[] values, RowSet rowSet assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getFloat(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_FLOAT, chunk.get(i)); pos++; } } @@ -320,13 +321,14 @@ private void validatePrevValuesWithFill(int chunkSize, float[] values, RowSet ro assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getPrevFloat(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_FLOAT, chunk.get(i)); pos++; } } assertEquals(pos, rowSet.size()); } } + // endregion validate with fill @Test public void testFillChunk() { @@ -346,6 +348,10 @@ public void testFillChunk() { testFillChunkGeneric(ArrayGenerator.randomFloats(random, 512), ArrayGenerator.randomFloats(random, 512), 4, RowSetFactory.fromKeys(254, 255, 256, 257)); testFillChunkGeneric(ArrayGenerator.randomFloats(random, 512), ArrayGenerator.randomFloats(random, 512), 5, RowSetFactory.fromKeys(254, 255, 256, 257, 258)); + // Test the fill with null behavior when requesting keys outside of source. + testFillChunkGeneric(ArrayGenerator.randomFloats(random, 512), ArrayGenerator.randomFloats(random, 4096), 4096, RowSetFactory.fromRange(4096, 8192)); + testFillChunkGeneric(ArrayGenerator.randomFloats(random, 512), ArrayGenerator.randomFloats(random, 4096), 4096, RowSetFactory.flat(4096)); + for (int sourceSize = 32; sourceSize < 8192; sourceSize *= 4) { for (int v = -4; v < 5; v += 2) { testSetForGivenSourceSize(random, sourceSize + v); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestIntegerArraySource.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestIntegerArraySource.java index 7de2644dffe..a456ae2b67c 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestIntegerArraySource.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestIntegerArraySource.java @@ -34,7 +34,7 @@ import java.util.stream.LongStream; import static io.deephaven.chunk.ArrayGenerator.indexDataGenerator; -import static io.deephaven.util.QueryConstants.NULL_INT; +import static io.deephaven.util.QueryConstants.*; import static junit.framework.TestCase.*; public class TestIntegerArraySource { @@ -284,6 +284,7 @@ private void testFillChunkGeneric(int[] values, int[] newValues, int chunkSize, } } + // region validate with fill private void validateValuesWithFill(int chunkSize, int[] values, RowSet rowSet, IntegerArraySource source) { try (final RowSequence.Iterator rsIterator = rowSet.getRowSequenceIterator(); final RowSet.Iterator it = rowSet.iterator(); @@ -298,7 +299,7 @@ private void validateValuesWithFill(int chunkSize, int[] values, RowSet rowSet, assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getInt(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_INT, chunk.get(i)); pos++; } } @@ -320,13 +321,14 @@ private void validatePrevValuesWithFill(int chunkSize, int[] values, RowSet rowS assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getPrevInt(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_INT, chunk.get(i)); pos++; } } assertEquals(pos, rowSet.size()); } } + // endregion validate with fill @Test public void testFillChunk() { @@ -346,6 +348,10 @@ public void testFillChunk() { testFillChunkGeneric(ArrayGenerator.randomInts(random, 512), ArrayGenerator.randomInts(random, 512), 4, RowSetFactory.fromKeys(254, 255, 256, 257)); testFillChunkGeneric(ArrayGenerator.randomInts(random, 512), ArrayGenerator.randomInts(random, 512), 5, RowSetFactory.fromKeys(254, 255, 256, 257, 258)); + // Test the fill with null behavior when requesting keys outside of source. + testFillChunkGeneric(ArrayGenerator.randomInts(random, 512), ArrayGenerator.randomInts(random, 4096), 4096, RowSetFactory.fromRange(4096, 8192)); + testFillChunkGeneric(ArrayGenerator.randomInts(random, 512), ArrayGenerator.randomInts(random, 4096), 4096, RowSetFactory.flat(4096)); + for (int sourceSize = 32; sourceSize < 8192; sourceSize *= 4) { for (int v = -4; v < 5; v += 2) { testSetForGivenSourceSize(random, sourceSize + v); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestLongArraySource.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestLongArraySource.java index 27aa059d851..97862c0d82f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestLongArraySource.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestLongArraySource.java @@ -34,7 +34,7 @@ import java.util.stream.LongStream; import static io.deephaven.chunk.ArrayGenerator.indexDataGenerator; -import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.*; import static junit.framework.TestCase.*; public class TestLongArraySource { @@ -284,6 +284,7 @@ private void testFillChunkGeneric(long[] values, long[] newValues, int chunkSize } } + // region validate with fill private void validateValuesWithFill(int chunkSize, long[] values, RowSet rowSet, LongArraySource source) { try (final RowSequence.Iterator rsIterator = rowSet.getRowSequenceIterator(); final RowSet.Iterator it = rowSet.iterator(); @@ -298,7 +299,7 @@ private void validateValuesWithFill(int chunkSize, long[] values, RowSet rowSet, assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getLong(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_LONG, chunk.get(i)); pos++; } } @@ -320,13 +321,14 @@ private void validatePrevValuesWithFill(int chunkSize, long[] values, RowSet row assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getPrevLong(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_LONG, chunk.get(i)); pos++; } } assertEquals(pos, rowSet.size()); } } + // endregion validate with fill @Test public void testFillChunk() { @@ -346,6 +348,10 @@ public void testFillChunk() { testFillChunkGeneric(ArrayGenerator.randomLongs(random, 512), ArrayGenerator.randomLongs(random, 512), 4, RowSetFactory.fromKeys(254, 255, 256, 257)); testFillChunkGeneric(ArrayGenerator.randomLongs(random, 512), ArrayGenerator.randomLongs(random, 512), 5, RowSetFactory.fromKeys(254, 255, 256, 257, 258)); + // Test the fill with null behavior when requesting keys outside of source. + testFillChunkGeneric(ArrayGenerator.randomLongs(random, 512), ArrayGenerator.randomLongs(random, 4096), 4096, RowSetFactory.fromRange(4096, 8192)); + testFillChunkGeneric(ArrayGenerator.randomLongs(random, 512), ArrayGenerator.randomLongs(random, 4096), 4096, RowSetFactory.flat(4096)); + for (int sourceSize = 32; sourceSize < 8192; sourceSize *= 4) { for (int v = -4; v < 5; v += 2) { testSetForGivenSourceSize(random, sourceSize + v); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestObjectArraySource.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestObjectArraySource.java index bf60bfc9c60..92afc9513bc 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestObjectArraySource.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestObjectArraySource.java @@ -283,6 +283,7 @@ private void testFillChunkGeneric(Object[] values, Object[] newValues, int chunk } } + // region validate with fill private void validateValuesWithFill(int chunkSize, Object[] values, RowSet rowSet, ObjectArraySource source) { try (final RowSequence.Iterator rsIterator = rowSet.getRowSequenceIterator(); final RowSet.Iterator it = rowSet.iterator(); @@ -297,7 +298,7 @@ private void validateValuesWithFill(int chunkSize, Object[] values, RowSet rowSe assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.get(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : null, chunk.get(i)); pos++; } } @@ -319,13 +320,14 @@ private void validatePrevValuesWithFill(int chunkSize, Object[] values, RowSet r assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getPrev(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : null, chunk.get(i)); pos++; } } assertEquals(pos, rowSet.size()); } } + // endregion validate with fill @Test public void testFillChunk() { @@ -345,6 +347,10 @@ public void testFillChunk() { testFillChunkGeneric(ArrayGenerator.randomObjects(random, 512), ArrayGenerator.randomObjects(random, 512), 4, RowSetFactory.fromKeys(254, 255, 256, 257)); testFillChunkGeneric(ArrayGenerator.randomObjects(random, 512), ArrayGenerator.randomObjects(random, 512), 5, RowSetFactory.fromKeys(254, 255, 256, 257, 258)); + // Test the fill with null behavior when requesting keys outside of source. + testFillChunkGeneric(ArrayGenerator.randomObjects(random, 512), ArrayGenerator.randomObjects(random, 4096), 4096, RowSetFactory.fromRange(4096, 8192)); + testFillChunkGeneric(ArrayGenerator.randomObjects(random, 512), ArrayGenerator.randomObjects(random, 4096), 4096, RowSetFactory.flat(4096)); + for (int sourceSize = 32; sourceSize < 8192; sourceSize *= 4) { for (int v = -4; v < 5; v += 2) { testSetForGivenSourceSize(random, sourceSize + v); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestShortArraySource.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestShortArraySource.java index 14b1283498b..1511c5d91ff 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestShortArraySource.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/sources/TestShortArraySource.java @@ -34,7 +34,7 @@ import java.util.stream.LongStream; import static io.deephaven.chunk.ArrayGenerator.indexDataGenerator; -import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.util.QueryConstants.*; import static junit.framework.TestCase.*; public class TestShortArraySource { @@ -284,6 +284,7 @@ private void testFillChunkGeneric(short[] values, short[] newValues, int chunkSi } } + // region validate with fill private void validateValuesWithFill(int chunkSize, short[] values, RowSet rowSet, ShortArraySource source) { try (final RowSequence.Iterator rsIterator = rowSet.getRowSequenceIterator(); final RowSet.Iterator it = rowSet.iterator(); @@ -298,7 +299,7 @@ private void validateValuesWithFill(int chunkSize, short[] values, RowSet rowSet assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getShort(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_SHORT, chunk.get(i)); pos++; } } @@ -320,13 +321,14 @@ private void validatePrevValuesWithFill(int chunkSize, short[] values, RowSet ro assertTrue(it.hasNext()); final long idx = it.nextLong(); checkFromSource(source.getPrevShort(idx), chunk.get(i)); - checkFromValues(values[(int) idx], chunk.get(i)); + checkFromValues(idx < values.length ? values[(int) idx] : NULL_SHORT, chunk.get(i)); pos++; } } assertEquals(pos, rowSet.size()); } } + // endregion validate with fill @Test public void testFillChunk() { @@ -346,6 +348,10 @@ public void testFillChunk() { testFillChunkGeneric(ArrayGenerator.randomShorts(random, 512), ArrayGenerator.randomShorts(random, 512), 4, RowSetFactory.fromKeys(254, 255, 256, 257)); testFillChunkGeneric(ArrayGenerator.randomShorts(random, 512), ArrayGenerator.randomShorts(random, 512), 5, RowSetFactory.fromKeys(254, 255, 256, 257, 258)); + // Test the fill with null behavior when requesting keys outside of source. + testFillChunkGeneric(ArrayGenerator.randomShorts(random, 512), ArrayGenerator.randomShorts(random, 4096), 4096, RowSetFactory.fromRange(4096, 8192)); + testFillChunkGeneric(ArrayGenerator.randomShorts(random, 512), ArrayGenerator.randomShorts(random, 4096), 4096, RowSetFactory.flat(4096)); + for (int sourceSize = 32; sourceSize < 8192; sourceSize *= 4) { for (int v = -4; v < 5; v += 2) { testSetForGivenSourceSize(random, sourceSize + v); diff --git a/replication/reflective/src/main/java/io/deephaven/replicators/ReplicateTypedHashers.java b/replication/reflective/src/main/java/io/deephaven/replicators/ReplicateTypedHashers.java index 614161601cc..0498383e936 100644 --- a/replication/reflective/src/main/java/io/deephaven/replicators/ReplicateTypedHashers.java +++ b/replication/reflective/src/main/java/io/deephaven/replicators/ReplicateTypedHashers.java @@ -11,6 +11,8 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.asofjoin.RightIncrementalAsOfJoinStateManagerTypedBase; import io.deephaven.engine.table.impl.asofjoin.StaticAsOfJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.multijoin.IncrementalMultiJoinStateManagerTypedBase; +import io.deephaven.engine.table.impl.multijoin.StaticMultiJoinStateManagerTypedBase; import io.deephaven.engine.table.impl.naturaljoin.RightIncrementalNaturalJoinStateManagerTypedBase; import io.deephaven.engine.table.impl.by.IncrementalChunkedOperatorAggregationStateManagerOpenAddressedBase; import io.deephaven.engine.table.impl.by.IncrementalChunkedOperatorAggregationStateManagerTypedBase; @@ -40,6 +42,8 @@ public static void main(String[] args) throws IOException { generatePackage(StaticAsOfJoinStateManagerTypedBase.class, false); generatePackage(RightIncrementalAsOfJoinStateManagerTypedBase.class, false); generatePackage(UpdateByStateManagerTypedBase.class, true); + generatePackage(StaticMultiJoinStateManagerTypedBase.class, false); + generatePackage(IncrementalMultiJoinStateManagerTypedBase.class, false); } private static void generatePackage(Class baseClass, boolean doDouble) throws IOException { 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 651b6f933ba..8ef644491f3 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourceAndChunkTests.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourceAndChunkTests.java @@ -124,6 +124,7 @@ private static void fixupBooleanColumnSourceTest(String booleanPath) throws IOEx lines = simpleFixup(lines, "arrayFill", "NULL_BOOLEAN", "BooleanUtils.NULL_BOOLEAN_AS_BYTE"); lines = simpleFixup(lines, "testsourcesink", "ChunkType.Boolean", "ChunkType.Object"); lines = simpleFixup(lines, "null unordered check", "NULL_BOOLEAN", "BooleanUtils.NULL_BOOLEAN_AS_BYTE"); + lines = simpleFixup(lines, "validate with fill", "NULL_BOOLEAN", "BooleanUtils.NULL_BOOLEAN_AS_BYTE"); lines = applyFixup(lines, "fromsource", "(.*)checkFromSource\\((.*)byte fromSource(.*)\\) \\{", m -> Collections.singletonList( @@ -141,6 +142,7 @@ private static void fixupBooleanColumnSourceTest(String booleanPath) throws IOEx lines = addImport(lines, "import io.deephaven.util.BooleanUtils;"); lines = addImport(lines, "import io.deephaven.chunk.WritableObjectChunk;"); lines = addImport(lines, "import io.deephaven.chunk.ObjectChunk;"); + if (!booleanPath.contains("Abstract") && !booleanPath.contains("Sparse")) { lines = removeImport(lines, "import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeyRanges;"); } diff --git a/table-api/src/main/java/io/deephaven/api/JoinAddition.java b/table-api/src/main/java/io/deephaven/api/JoinAddition.java index 150e5cd7b5d..fcfbe9f6d92 100644 --- a/table-api/src/main/java/io/deephaven/api/JoinAddition.java +++ b/table-api/src/main/java/io/deephaven/api/JoinAddition.java @@ -5,6 +5,7 @@ import io.deephaven.api.snapshot.SnapshotWhenOptions; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.stream.Collectors; @@ -38,6 +39,10 @@ static JoinAddition parse(String x) { return of(newColumn, existingColumn); } + static List from(String... values) { + return JoinAddition.from(Arrays.asList(values)); + } + static List from(Collection values) { return values.stream().map(JoinAddition::parse).collect(Collectors.toList()); }