Skip to content

Commit 46579d3

Browse files
authored
feat: add LastBy / FirstBy features to NaturalJoin (deephaven#6604)
Allows user to specify the row to include when there are duplicates of RHS rows. Choices are: * ERROR_ON_DUPLICATES - original NJ behavior * FIRST_MATCH - equivalent to running FirstBy + NaturalJoin * LAST_MATCH - equivalent to running LastBy + NaturalJoin * EXACT_MATCH - equivalent of `exactJoin`, requires exactly one RHS row for every LHS row
1 parent e1dff59 commit 46579d3

File tree

66 files changed

+6126
-2962
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

66 files changed

+6126
-2962
lines changed

engine/table/src/main/java/io/deephaven/engine/table/impl/BothIncrementalNaturalJoinStateManager.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -21,11 +21,11 @@ public interface BothIncrementalNaturalJoinStateManager extends IncrementalNatur
2121

2222
void compactAll();
2323

24-
WritableRowRedirection buildIndexedRowRedirection(QueryTable leftTable, boolean exactMatch, InitialBuildContext ibc,
24+
WritableRowRedirection buildIndexedRowRedirection(QueryTable leftTable, InitialBuildContext ibc,
2525
ColumnSource<RowSet> indexRowSets, JoinControl.RedirectionType redirectionType);
2626

27-
WritableRowRedirection buildRowRedirectionFromRedirections(QueryTable leftTable, boolean exactMatch,
28-
InitialBuildContext ibc, JoinControl.RedirectionType redirectionType);
27+
WritableRowRedirection buildRowRedirectionFromRedirections(QueryTable leftTable, InitialBuildContext ibc,
28+
JoinControl.RedirectionType redirectionType);
2929

3030
Context makeProbeContext(ColumnSource<?>[] probeSources, long maxSize);
3131

engine/table/src/main/java/io/deephaven/engine/table/impl/IncrementalNaturalJoinStateManager.java

+47-3
Original file line numberDiff line numberDiff line change
@@ -3,14 +3,58 @@
33
//
44
package io.deephaven.engine.table.impl;
55

6+
import io.deephaven.api.NaturalJoinType;
7+
import io.deephaven.base.verify.Assert;
68
import io.deephaven.engine.rowset.RowSet;
9+
import io.deephaven.engine.rowset.WritableRowSet;
710

811
public interface IncrementalNaturalJoinStateManager {
9-
long getRightIndex(int slot);
12+
long getRightRowKey(int slot);
1013

11-
RowSet getLeftIndex(int slot);
14+
RowSet getRightRowSet(int slot);
15+
16+
RowSet getLeftRowSet(int slot);
1217

1318
String keyString(int slot);
1419

15-
void checkExactMatch(boolean exactMatch, long leftKeyIndex, long rightSide);
20+
void checkExactMatch(long leftKeyIndex, long rightSide);
21+
22+
/**
23+
* Given the join type, return the correct row key from the set of RHS duplicates.
24+
*/
25+
default long getRightRowKeyFromDuplicates(final WritableRowSet duplicates, final NaturalJoinType joinType) {
26+
if (joinType == NaturalJoinType.LAST_MATCH) {
27+
return duplicates.lastRowKey();
28+
}
29+
return duplicates.firstRowKey();
30+
}
31+
32+
/**
33+
* Add a key to the RHS duplicates, return the appropriate row key from this set *AFTER* the addition.
34+
*/
35+
default long addRightRowKeyToDuplicates(final WritableRowSet duplicates, final long keyToRemove,
36+
final NaturalJoinType joinType) {
37+
duplicates.insert(keyToRemove);
38+
return getRightRowKeyFromDuplicates(duplicates, joinType);
39+
}
40+
41+
/**
42+
* Remove the key from the RHS duplicates, return the appropriate row key from this set *BEFORE* the removal.
43+
*/
44+
default long removeRightRowKeyFromDuplicates(final WritableRowSet duplicates, final long keyToRemove,
45+
final NaturalJoinType joinType) {
46+
final long originalRowKey = getRightRowKeyFromDuplicates(duplicates, joinType);
47+
duplicates.remove(keyToRemove);
48+
return originalRowKey;
49+
}
50+
51+
/**
52+
* Shift a key in the RHS duplicate row set.
53+
*/
54+
default void shiftOneKey(WritableRowSet duplicates, long shiftedKey, long shiftDelta) {
55+
final long sizeBefore = duplicates.size();
56+
duplicates.remove(shiftedKey - shiftDelta);
57+
duplicates.insert(shiftedKey);
58+
Assert.eq(duplicates.size(), "duplicates.size()", sizeBefore, "sizeBefore");
59+
}
1660
}

engine/table/src/main/java/io/deephaven/engine/table/impl/NaturalJoinHelper.java

+130-92
Large diffs are not rendered by default.

engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java

+19-9
Original file line numberDiff line numberDiff line change
@@ -768,7 +768,7 @@ private Table exactJoinImpl(Table table, MatchPair[] columnsToMatch, MatchPair[]
768768
"exactJoin(" + table + "," + Arrays.toString(columnsToMatch) + "," + Arrays.toString(columnsToMatch)
769769
+ ")",
770770
sizeForInstrumentation(),
771-
() -> naturalJoinInternal(table, columnsToMatch, columnsToAdd, true));
771+
() -> naturalJoinInternal(table, columnsToMatch, columnsToAdd, NaturalJoinType.EXACTLY_ONE_MATCH));
772772
}
773773
}
774774

@@ -2263,29 +2263,38 @@ private Table ajInternal(Table rightTable, MatchPair[] columnsToMatch, MatchPair
22632263
public Table naturalJoin(
22642264
Table rightTable,
22652265
Collection<? extends JoinMatch> columnsToMatch,
2266-
Collection<? extends JoinAddition> columnsToAdd) {
2266+
Collection<? extends JoinAddition> columnsToAdd,
2267+
NaturalJoinType joinType) {
22672268
return naturalJoinImpl(
22682269
rightTable,
22692270
MatchPair.fromMatches(columnsToMatch),
2270-
MatchPair.fromAddition(columnsToAdd));
2271+
MatchPair.fromAddition(columnsToAdd),
2272+
joinType);
22712273
}
22722274

2273-
private Table naturalJoinImpl(final Table rightTable, final MatchPair[] columnsToMatch, MatchPair[] columnsToAdd) {
2275+
private Table naturalJoinImpl(
2276+
final Table rightTable,
2277+
final MatchPair[] columnsToMatch,
2278+
final MatchPair[] columnsToAdd,
2279+
final NaturalJoinType joinType) {
22742280
final UpdateGraph updateGraph = getUpdateGraph(rightTable);
22752281
try (final SafeCloseable ignored = ExecutionContext.getContext().withUpdateGraph(updateGraph).open()) {
22762282
return QueryPerformanceRecorder.withNugget(
22772283
"naturalJoin(" + matchString(columnsToMatch) + ", " + matchString(columnsToAdd) + ")",
2278-
() -> naturalJoinInternal(rightTable, columnsToMatch, columnsToAdd, false));
2284+
() -> naturalJoinInternal(rightTable, columnsToMatch, columnsToAdd, joinType));
22792285
}
22802286
}
22812287

2282-
private Table naturalJoinInternal(final Table rightTable, final MatchPair[] columnsToMatch,
2283-
MatchPair[] columnsToAdd, boolean exactMatch) {
2288+
private Table naturalJoinInternal(
2289+
final Table rightTable,
2290+
final MatchPair[] columnsToMatch,
2291+
MatchPair[] columnsToAdd,
2292+
final NaturalJoinType joinType) {
22842293
columnsToAdd = createColumnsToAddIfMissing(rightTable, columnsToMatch, columnsToAdd);
22852294

22862295
final QueryTable rightTableCoalesced = (QueryTable) rightTable.coalesce();
22872296

2288-
return NaturalJoinHelper.naturalJoin(this, rightTableCoalesced, columnsToMatch, columnsToAdd, exactMatch);
2297+
return NaturalJoinHelper.naturalJoin(this, rightTableCoalesced, columnsToMatch, columnsToAdd, joinType);
22892298
}
22902299

22912300
private MatchPair[] createColumnsToAddIfMissing(Table rightTable, MatchPair[] columnsToMatch,
@@ -2393,7 +2402,8 @@ private Table joinNoMemo(
23932402
final Table rightGrouped = rightTable.groupBy(rightColumnsToMatch)
23942403
.view(columnsToAddSelectColumns.values());
23952404
final Table naturalJoinResult = naturalJoinImpl(rightGrouped, columnsToMatch,
2396-
columnsToAddAfterRename.toArray(MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY));
2405+
columnsToAddAfterRename.toArray(MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY),
2406+
NaturalJoinType.ERROR_ON_DUPLICATE);
23972407
final QueryTable ungroupedResult = (QueryTable) naturalJoinResult
23982408
.ungroup(columnsToUngroupBy.toArray(String[]::new));
23992409

engine/table/src/main/java/io/deephaven/engine/table/impl/RightIncrementalNaturalJoinStateManager.java

+9-5
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
//
44
package io.deephaven.engine.table.impl;
55

6+
import io.deephaven.api.NaturalJoinType;
67
import io.deephaven.engine.rowset.RowSequence;
78
import io.deephaven.engine.rowset.RowSet;
89
import io.deephaven.engine.table.ColumnSource;
@@ -15,8 +16,11 @@
1516
public abstract class RightIncrementalNaturalJoinStateManager extends StaticNaturalJoinStateManager
1617
implements IncrementalNaturalJoinStateManager {
1718

18-
protected RightIncrementalNaturalJoinStateManager(ColumnSource<?>[] keySourcesForErrorMessages) {
19-
super(keySourcesForErrorMessages);
19+
protected RightIncrementalNaturalJoinStateManager(
20+
ColumnSource<?>[] keySourcesForErrorMessages,
21+
NaturalJoinType joinType,
22+
boolean addOnly) {
23+
super(keySourcesForErrorMessages, joinType, addOnly);
2024
}
2125

2226
public abstract void buildFromLeftSide(final Table leftTable, ColumnSource<?>[] leftSources,
@@ -27,12 +31,12 @@ public abstract void convertLeftDataIndex(int groupingSize, InitialBuildContext
2731

2832
public abstract void addRightSide(RowSequence rightIndex, ColumnSource<?>[] rightSources);
2933

30-
public abstract WritableRowRedirection buildRowRedirectionFromHashSlot(QueryTable leftTable, boolean exactMatch,
34+
public abstract WritableRowRedirection buildRowRedirectionFromHashSlot(QueryTable leftTable,
3135
InitialBuildContext initialBuildContext, JoinControl.RedirectionType redirectionType);
3236

3337
public abstract WritableRowRedirection buildRowRedirectionFromHashSlotIndexed(QueryTable leftTable,
34-
ColumnSource<RowSet> rowSetSource, int groupingSize, boolean exactMatch,
35-
InitialBuildContext initialBuildContext, JoinControl.RedirectionType redirectionType);
38+
ColumnSource<RowSet> rowSetSource, int groupingSize, InitialBuildContext initialBuildContext,
39+
JoinControl.RedirectionType redirectionType);
3640

3741
// modification probes
3842
public abstract void applyRightShift(Context pc, ColumnSource<?>[] rightSources, RowSet shiftedRowSet,

engine/table/src/main/java/io/deephaven/engine/table/impl/SimpleUniqueStaticNaturalJoinStateManager.java

+16-7
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
//
44
package io.deephaven.engine.table.impl;
55

6+
import io.deephaven.api.NaturalJoinType;
67
import io.deephaven.base.verify.Assert;
78
import io.deephaven.base.verify.Require;
89
import io.deephaven.chunk.util.hashing.ToIntFunctor;
@@ -32,9 +33,13 @@ class SimpleUniqueStaticNaturalJoinStateManager extends StaticNaturalJoinStateMa
3233

3334
private final LongArraySource rightRowSetSource = new LongArraySource();
3435

35-
SimpleUniqueStaticNaturalJoinStateManager(ColumnSource<?>[] tableKeySources, int tableSize,
36-
ToIntFunctor<Values> transform) {
37-
super(tableKeySources);
36+
SimpleUniqueStaticNaturalJoinStateManager(
37+
ColumnSource<?>[] tableKeySources,
38+
int tableSize,
39+
ToIntFunctor<Values> transform,
40+
NaturalJoinType joinType,
41+
boolean addOnly) {
42+
super(tableKeySources, joinType, addOnly);
3843
this.tableSize = Require.gtZero(tableSize, "tableSize");
3944
this.transform = transform;
4045
rightRowSetSource.ensureCapacity(tableSize);
@@ -61,10 +66,14 @@ void setRightSide(RowSet rightRowSet, ColumnSource<?> valueSource) {
6166
return true;
6267
}
6368
final long existingRight = rightRowSetSource.getLong(tableLocation);
64-
if (existingRight == RowSequence.NULL_ROW_KEY) {
69+
if (existingRight == RowSequence.NULL_ROW_KEY || joinType == NaturalJoinType.LAST_MATCH) {
6570
rightRowSetSource.set(tableLocation, keyIndex);
6671
} else {
67-
rightRowSetSource.set(tableLocation, DUPLICATE_RIGHT_VALUE);
72+
if (joinType == NaturalJoinType.FIRST_MATCH) {
73+
// no-op, already have the first match
74+
} else {
75+
rightRowSetSource.set(tableLocation, DUPLICATE_RIGHT_VALUE);
76+
}
6877
}
6978
return true;
7079
});
@@ -112,8 +121,8 @@ protected void decorateLeftSide(RowSet leftRowSet, ColumnSource<?>[] valueSource
112121
}
113122

114123
@NotNull
115-
WritableRowRedirection buildRowRedirection(QueryTable leftTable, boolean exactMatch,
124+
WritableRowRedirection buildRowRedirection(QueryTable leftTable,
116125
LongArraySource leftRedirections, JoinControl.RedirectionType redirectionType) {
117-
return buildRowRedirection(leftTable, exactMatch, leftRedirections::getLong, redirectionType);
126+
return buildRowRedirection(leftTable, leftRedirections::getLong, redirectionType);
118127
}
119128
}

engine/table/src/main/java/io/deephaven/engine/table/impl/StaticNaturalJoinStateManager.java

+16-8
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
//
44
package io.deephaven.engine.table.impl;
55

6+
import io.deephaven.api.NaturalJoinType;
67
import io.deephaven.engine.rowset.RowSequence;
78
import io.deephaven.engine.rowset.RowSet;
89
import io.deephaven.engine.table.ColumnSource;
@@ -20,14 +21,21 @@ public abstract class StaticNaturalJoinStateManager {
2021
public static final long NO_RIGHT_ENTRY_VALUE = RowSequence.NULL_ROW_KEY;
2122

2223
protected final ColumnSource<?>[] keySourcesForErrorMessages;
24+
protected final NaturalJoinType joinType;
25+
protected final boolean addOnly;
2326

24-
protected StaticNaturalJoinStateManager(ColumnSource<?>[] keySourcesForErrorMessages) {
27+
protected StaticNaturalJoinStateManager(
28+
ColumnSource<?>[] keySourcesForErrorMessages,
29+
NaturalJoinType joinType,
30+
boolean addOnly) {
2531
this.keySourcesForErrorMessages = keySourcesForErrorMessages;
32+
this.joinType = joinType;
33+
this.addOnly = addOnly;
2634
}
2735

2836
@SuppressWarnings("WeakerAccess")
29-
public void checkExactMatch(boolean exactMatch, long leftKeyIndex, long rightSide) {
30-
if (exactMatch && rightSide == NO_RIGHT_ENTRY_VALUE) {
37+
public void checkExactMatch(long leftKeyIndex, long rightSide) {
38+
if (joinType == NaturalJoinType.EXACTLY_ONE_MATCH && rightSide == NO_RIGHT_ENTRY_VALUE) {
3139
throw new RuntimeException("Tables don't have one-to-one mapping - no mappings for key "
3240
+ extractKeyStringFromSourceTable(leftKeyIndex) + ".");
3341
}
@@ -42,8 +50,8 @@ protected String extractKeyStringFromSourceTable(long leftKey) {
4250
.collect(Collectors.joining(", ")) + "]";
4351
}
4452

45-
public WritableRowRedirection buildRowRedirection(QueryTable leftTable, boolean exactMatch,
46-
LongUnaryOperator rightSideFromSlot, JoinControl.RedirectionType redirectionType) {
53+
public WritableRowRedirection buildRowRedirection(QueryTable leftTable, LongUnaryOperator rightSideFromSlot,
54+
JoinControl.RedirectionType redirectionType) {
4755
switch (redirectionType) {
4856
case Contiguous: {
4957
if (!leftTable.isFlat()) {
@@ -53,7 +61,7 @@ public WritableRowRedirection buildRowRedirection(QueryTable leftTable, boolean
5361
final long[] innerIndex = new long[leftTable.intSize("contiguous redirection build")];
5462
for (int ii = 0; ii < innerIndex.length; ++ii) {
5563
final long rightSide = rightSideFromSlot.applyAsLong(ii);
56-
checkExactMatch(exactMatch, leftTable.getRowSet().get(ii), rightSide);
64+
checkExactMatch(leftTable.getRowSet().get(ii), rightSide);
5765
innerIndex[ii] = rightSide;
5866
}
5967
return new ContiguousWritableRowRedirection(innerIndex);
@@ -65,7 +73,7 @@ public WritableRowRedirection buildRowRedirection(QueryTable leftTable, boolean
6573
for (final RowSet.Iterator it = leftTable.getRowSet().iterator(); it.hasNext();) {
6674
final long next = it.nextLong();
6775
final long rightSide = rightSideFromSlot.applyAsLong(leftPosition++);
68-
checkExactMatch(exactMatch, leftTable.getRowSet().get(next), rightSide);
76+
checkExactMatch(leftTable.getRowSet().get(next), rightSide);
6977
if (rightSide != NO_RIGHT_ENTRY_VALUE) {
7078
sparseRedirections.set(next, rightSide);
7179
}
@@ -80,7 +88,7 @@ public WritableRowRedirection buildRowRedirection(QueryTable leftTable, boolean
8088
for (final RowSet.Iterator it = leftTable.getRowSet().iterator(); it.hasNext();) {
8189
final long next = it.nextLong();
8290
final long rightSide = rightSideFromSlot.applyAsLong(leftPosition++);
83-
checkExactMatch(exactMatch, leftTable.getRowSet().get(next), rightSide);
91+
checkExactMatch(leftTable.getRowSet().get(next), rightSide);
8492
if (rightSide != NO_RIGHT_ENTRY_VALUE) {
8593
rowRedirection.put(next, rightSide);
8694
}

engine/table/src/main/java/io/deephaven/engine/table/impl/TableAdapter.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -7,11 +7,12 @@
77
import io.deephaven.api.ColumnName;
88
import io.deephaven.api.JoinAddition;
99
import io.deephaven.api.JoinMatch;
10+
import io.deephaven.api.NaturalJoinType;
11+
import io.deephaven.api.Pair;
1012
import io.deephaven.api.RangeJoinMatch;
1113
import io.deephaven.api.Selectable;
1214
import io.deephaven.api.SortColumn;
1315
import io.deephaven.api.agg.Aggregation;
14-
import io.deephaven.api.Pair;
1516
import io.deephaven.api.agg.spec.AggSpec;
1617
import io.deephaven.api.filter.Filter;
1718
import io.deephaven.api.snapshot.SnapshotWhenOptions;
@@ -266,7 +267,7 @@ default Table asOfJoin(Table rightTable, Collection<? extends JoinMatch> exactMa
266267

267268
@Override
268269
default Table naturalJoin(Table rightTable, Collection<? extends JoinMatch> columnsToMatch,
269-
Collection<? extends JoinAddition> columnsToAdd) {
270+
Collection<? extends JoinAddition> columnsToAdd, NaturalJoinType joinType) {
270271
return throwUnsupported();
271272
}
272273

engine/table/src/main/java/io/deephaven/engine/table/impl/UncoalescedTable.java

+5-3
Original file line numberDiff line numberDiff line change
@@ -7,11 +7,12 @@
77
import io.deephaven.api.ColumnName;
88
import io.deephaven.api.JoinAddition;
99
import io.deephaven.api.JoinMatch;
10+
import io.deephaven.api.NaturalJoinType;
11+
import io.deephaven.api.Pair;
1012
import io.deephaven.api.RangeJoinMatch;
1113
import io.deephaven.api.Selectable;
1214
import io.deephaven.api.SortColumn;
1315
import io.deephaven.api.agg.Aggregation;
14-
import io.deephaven.api.Pair;
1516
import io.deephaven.api.agg.spec.AggSpec;
1617
import io.deephaven.api.filter.Filter;
1718
import io.deephaven.api.snapshot.SnapshotWhenOptions;
@@ -335,8 +336,9 @@ public Table asOfJoin(Table rightTable, Collection<? extends JoinMatch> exactMat
335336
public Table naturalJoin(
336337
Table rightTable,
337338
Collection<? extends JoinMatch> columnsToMatch,
338-
Collection<? extends JoinAddition> columnsToAdd) {
339-
return coalesce().naturalJoin(rightTable, columnsToMatch, columnsToAdd);
339+
Collection<? extends JoinAddition> columnsToAdd,
340+
NaturalJoinType joinType) {
341+
return coalesce().naturalJoin(rightTable, columnsToMatch, columnsToAdd, joinType);
340342
}
341343

342344
@Override

0 commit comments

Comments
 (0)