diff --git a/core/trino-main/src/main/java/io/trino/operator/ChannelSet.java b/core/trino-main/src/main/java/io/trino/operator/ChannelSet.java index e23308b091e4..45d099e04368 100644 --- a/core/trino-main/src/main/java/io/trino/operator/ChannelSet.java +++ b/core/trino-main/src/main/java/io/trino/operator/ChannelSet.java @@ -24,7 +24,6 @@ import static io.trino.spi.function.InvocationConvention.InvocationReturnConvention.FAIL_ON_NULL; import static io.trino.spi.function.InvocationConvention.InvocationReturnConvention.FLAT_RETURN; import static io.trino.spi.function.InvocationConvention.simpleConvention; -import static io.trino.spi.type.BigintType.BIGINT; import static java.util.Objects.requireNonNull; public class ChannelSet @@ -88,24 +87,14 @@ public ChannelSet build() return new ChannelSet(set); } - public void addAll(Block valueBlock, Block hashBlock) + public void addAll(Block valueBlock) { if (valueBlock.getPositionCount() == 0) { return; } if (valueBlock instanceof RunLengthEncodedBlock rleBlock) { - if (hashBlock != null) { - set.add(rleBlock.getValue(), 0, BIGINT.getLong(hashBlock, 0)); - } - else { - set.add(rleBlock.getValue(), 0); - } - } - else if (hashBlock != null) { - for (int position = 0; position < valueBlock.getPositionCount(); position++) { - set.add(valueBlock, position, BIGINT.getLong(hashBlock, position)); - } + set.add(rleBlock.getValue(), 0); } else { for (int position = 0; position < valueBlock.getPositionCount(); position++) { diff --git a/core/trino-main/src/main/java/io/trino/operator/HashSemiJoinOperator.java b/core/trino-main/src/main/java/io/trino/operator/HashSemiJoinOperator.java index 7d5a9a19031e..05d57c00e2ba 100644 --- a/core/trino-main/src/main/java/io/trino/operator/HashSemiJoinOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/HashSemiJoinOperator.java @@ -29,7 +29,6 @@ import jakarta.annotation.Nullable; import java.util.List; -import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -40,7 +39,6 @@ import static io.trino.operator.WorkProcessor.TransformationState.finished; import static io.trino.operator.WorkProcessor.TransformationState.ofResult; import static io.trino.operator.WorkProcessorOperatorAdapter.createAdapterOperatorFactory; -import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; import static java.util.Objects.requireNonNull; @@ -52,10 +50,9 @@ public static OperatorFactory createOperatorFactory( PlanNodeId planNodeId, SetSupplier setSupplier, List probeTypes, - int probeJoinChannel, - Optional probeJoinHashChannel) + int probeJoinChannel) { - return createAdapterOperatorFactory(new Factory(operatorId, planNodeId, setSupplier, probeTypes, probeJoinChannel, probeJoinHashChannel)); + return createAdapterOperatorFactory(new Factory(operatorId, planNodeId, setSupplier, probeTypes, probeJoinChannel)); } private static class Factory @@ -66,10 +63,9 @@ private static class Factory private final SetSupplier setSupplier; private final List probeTypes; private final int probeJoinChannel; - private final Optional probeJoinHashChannel; private boolean closed; - private Factory(int operatorId, PlanNodeId planNodeId, SetSupplier setSupplier, List probeTypes, int probeJoinChannel, Optional probeJoinHashChannel) + private Factory(int operatorId, PlanNodeId planNodeId, SetSupplier setSupplier, List probeTypes, int probeJoinChannel) { this.operatorId = operatorId; this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); @@ -77,14 +73,13 @@ private Factory(int operatorId, PlanNodeId planNodeId, SetSupplier setSupplier, this.probeTypes = ImmutableList.copyOf(probeTypes); checkArgument(probeJoinChannel >= 0, "probeJoinChannel is negative"); this.probeJoinChannel = probeJoinChannel; - this.probeJoinHashChannel = probeJoinHashChannel; } @Override public WorkProcessorOperator create(ProcessorContext processorContext, WorkProcessor sourcePages) { checkState(!closed, "Factory is already closed"); - return new HashSemiJoinOperator(sourcePages, setSupplier, probeJoinChannel, probeJoinHashChannel, processorContext.getMemoryTrackingContext()); + return new HashSemiJoinOperator(sourcePages, setSupplier, probeJoinChannel, processorContext.getMemoryTrackingContext()); } @Override @@ -114,7 +109,7 @@ public void close() @Override public Factory duplicate() { - return new Factory(operatorId, planNodeId, setSupplier, probeTypes, probeJoinChannel, probeJoinHashChannel); + return new Factory(operatorId, planNodeId, setSupplier, probeTypes, probeJoinChannel); } } @@ -124,14 +119,12 @@ private HashSemiJoinOperator( WorkProcessor sourcePages, SetSupplier channelSetFuture, int probeJoinChannel, - Optional probeHashChannel, MemoryTrackingContext memoryTrackingContext) { pages = sourcePages .transform(new SemiJoinPages( channelSetFuture, probeJoinChannel, - probeHashChannel, memoryTrackingContext.aggregateUserMemoryContext())); } @@ -144,23 +137,19 @@ public WorkProcessor getOutputPages() private static class SemiJoinPages implements WorkProcessor.Transformation { - private static final int NO_PRECOMPUTED_HASH_CHANNEL = -1; - private final int probeJoinChannel; - private final int probeHashChannel; // when >= 0, this is the precomputed hash channel private final ListenableFuture channelSetFuture; private final LocalMemoryContext localMemoryContext; @Nullable private ChannelSet channelSet; - public SemiJoinPages(SetSupplier channelSetFuture, int probeJoinChannel, Optional probeHashChannel, AggregatedMemoryContext aggregatedMemoryContext) + public SemiJoinPages(SetSupplier channelSetFuture, int probeJoinChannel, AggregatedMemoryContext aggregatedMemoryContext) { checkArgument(probeJoinChannel >= 0, "probeJoinChannel is negative"); this.channelSetFuture = channelSetFuture.getChannelSet(); this.probeJoinChannel = probeJoinChannel; - this.probeHashChannel = probeHashChannel.orElse(NO_PRECOMPUTED_HASH_CHANNEL); this.localMemoryContext = aggregatedMemoryContext.newLocalMemoryContext(SemiJoinPages.class.getSimpleName()); } @@ -190,7 +179,6 @@ public TransformationState process(Page inputPage) Block probeBlock = inputPage.getBlock(probeJoinChannel).copyRegion(0, inputPage.getPositionCount()); boolean probeMayHaveNull = probeBlock.mayHaveNull(); - Block hashBlock = probeHashChannel >= 0 ? inputPage.getBlock(probeHashChannel).copyRegion(0, inputPage.getPositionCount()) : null; // update hashing strategy to use probe cursor for (int position = 0; position < inputPage.getPositionCount(); position++) { @@ -203,14 +191,7 @@ public TransformationState process(Page inputPage) } } else { - boolean contains; - if (hashBlock != null) { - long rawHash = BIGINT.getLong(hashBlock, position); - contains = channelSet.contains(probeBlock, position, rawHash); - } - else { - contains = channelSet.contains(probeBlock, position); - } + boolean contains = channelSet.contains(probeBlock, position); if (!contains && channelSet.containsNull()) { blockBuilder.appendNull(); } diff --git a/core/trino-main/src/main/java/io/trino/operator/OperatorFactories.java b/core/trino-main/src/main/java/io/trino/operator/OperatorFactories.java index baeb53d3d78a..0e1e7d6d5c48 100644 --- a/core/trino-main/src/main/java/io/trino/operator/OperatorFactories.java +++ b/core/trino-main/src/main/java/io/trino/operator/OperatorFactories.java @@ -44,7 +44,6 @@ public static OperatorFactory join( boolean hasFilter, List probeTypes, List probeJoinChannel, - OptionalInt probeHashChannel, Optional> probeOutputChannelsOptional) { List probeOutputChannels = probeOutputChannelsOptional.orElseGet(() -> rangeList(probeTypes.size())); @@ -60,7 +59,7 @@ public static OperatorFactory join( probeOutputChannelTypes, lookupSourceFactory.getBuildOutputTypes(), joinType, - new JoinProbe.JoinProbeFactory(probeOutputChannels, probeJoinChannel, probeHashChannel, hasFilter))); + new JoinProbe.JoinProbeFactory(probeOutputChannels, probeJoinChannel, hasFilter))); } public static OperatorFactory spillingJoin( @@ -70,7 +69,6 @@ public static OperatorFactory spillingJoin( JoinBridgeManager lookupSourceFactory, List probeTypes, List probeJoinChannel, - OptionalInt probeHashChannel, Optional> probeOutputChannelsOptional, OptionalInt totalOperatorsCount, PartitioningSpillerFactory partitioningSpillerFactory, @@ -89,11 +87,10 @@ public static OperatorFactory spillingJoin( probeOutputChannelTypes, lookupSourceFactory.getBuildOutputTypes(), joinType, - new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel, probeHashChannel), + new JoinProbeFactory(probeOutputChannels.stream().mapToInt(i -> i).toArray(), probeJoinChannel), typeOperators, totalOperatorsCount, probeJoinChannel, - probeHashChannel, partitioningSpillerFactory)); } diff --git a/core/trino-main/src/main/java/io/trino/operator/PagesIndex.java b/core/trino-main/src/main/java/io/trino/operator/PagesIndex.java index 7269055b5012..373d031f4a3c 100644 --- a/core/trino-main/src/main/java/io/trino/operator/PagesIndex.java +++ b/core/trino-main/src/main/java/io/trino/operator/PagesIndex.java @@ -456,19 +456,19 @@ private PagesIndexOrdering createPagesIndexComparator(List sortChannels public Supplier createLookupSourceSupplier(Session session, List joinChannels) { - return createLookupSourceSupplier(session, joinChannels, OptionalInt.empty(), Optional.empty(), Optional.empty(), ImmutableList.of()); + return createLookupSourceSupplier(session, joinChannels, Optional.empty(), Optional.empty(), ImmutableList.of()); } - public PagesHashStrategy createPagesHashStrategy(List joinChannels, OptionalInt hashChannel) + public PagesHashStrategy createPagesHashStrategy(List joinChannels) { - return createPagesHashStrategy(joinChannels, hashChannel, Optional.empty()); + return createPagesHashStrategy(joinChannels, Optional.empty()); } - private PagesHashStrategy createPagesHashStrategy(List joinChannels, OptionalInt hashChannel, Optional> outputChannels) + private PagesHashStrategy createPagesHashStrategy(List joinChannels, Optional> outputChannels) { try { return joinCompiler.compilePagesHashStrategyFactory(types, joinChannels, outputChannels) - .createPagesHashStrategy(ImmutableList.copyOf(channels), hashChannel); + .createPagesHashStrategy(ImmutableList.copyOf(channels)); } catch (Exception e) { log.error(e, "Lookup source compile failed for types=%s error=%s", types, e); @@ -480,7 +480,6 @@ private PagesHashStrategy createPagesHashStrategy(List joinChannels, Op outputChannels.orElseGet(() -> rangeList(types.size())), ImmutableList.copyOf(channels), joinChannels, - hashChannel, Optional.empty(), blockTypeOperators); } @@ -494,12 +493,11 @@ public PagesIndexComparator createChannelComparator(int leftChannel, int rightCh public LookupSourceSupplier createLookupSourceSupplier( Session session, List joinChannels, - OptionalInt hashChannel, Optional filterFunctionFactory, Optional sortChannel, List searchFunctionFactories) { - return createLookupSourceSupplier(session, joinChannels, hashChannel, filterFunctionFactory, sortChannel, searchFunctionFactories, Optional.empty(), defaultHashArraySizeSupplier()); + return createLookupSourceSupplier(session, joinChannels, filterFunctionFactory, sortChannel, searchFunctionFactories, Optional.empty(), defaultHashArraySizeSupplier()); } public PagesSpatialIndexSupplier createPagesSpatialIndex( @@ -521,7 +519,6 @@ public PagesSpatialIndexSupplier createPagesSpatialIndex( public LookupSourceSupplier createLookupSourceSupplier( Session session, List joinChannels, - OptionalInt hashChannel, Optional filterFunctionFactory, Optional sortChannel, List searchFunctionFactories, @@ -539,7 +536,6 @@ public LookupSourceSupplier createLookupSourceSupplier( session, valueAddresses, channels, - hashChannel, filterFunctionFactory, sortChannel, searchFunctionFactories, @@ -551,7 +547,6 @@ public LookupSourceSupplier createLookupSourceSupplier( outputChannels.orElseGet(() -> rangeList(types.size())), channels, joinChannels, - hashChannel, sortChannel, blockTypeOperators); diff --git a/core/trino-main/src/main/java/io/trino/operator/SetBuilderOperator.java b/core/trino-main/src/main/java/io/trino/operator/SetBuilderOperator.java index 6345a68ccd95..a74242cfd571 100644 --- a/core/trino-main/src/main/java/io/trino/operator/SetBuilderOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/SetBuilderOperator.java @@ -23,8 +23,6 @@ import io.trino.sql.gen.JoinCompiler; import io.trino.sql.planner.plan.PlanNodeId; -import java.util.Optional; - import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; @@ -65,7 +63,6 @@ public static class SetBuilderOperatorFactory { private final int operatorId; private final PlanNodeId planNodeId; - private final Optional hashChannel; private final SetSupplier setProvider; private final int setChannel; private final int expectedPositions; @@ -78,7 +75,6 @@ public SetBuilderOperatorFactory( PlanNodeId planNodeId, Type type, int setChannel, - Optional hashChannel, int expectedPositions, JoinCompiler joinCompiler, TypeOperators typeOperators) @@ -88,7 +84,6 @@ public SetBuilderOperatorFactory( checkArgument(setChannel >= 0, "setChannel is negative"); this.setProvider = new SetSupplier(requireNonNull(type, "type is null")); this.setChannel = setChannel; - this.hashChannel = requireNonNull(hashChannel, "hashChannel is null"); this.expectedPositions = expectedPositions; this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null"); this.typeOperators = requireNonNull(typeOperators, "blockTypeOperators is null"); @@ -104,7 +99,7 @@ public Operator createOperator(DriverContext driverContext) { checkState(!closed, "Factory is already closed"); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, SetBuilderOperator.class.getSimpleName()); - return new SetBuilderOperator(operatorContext, setProvider, setChannel, hashChannel, expectedPositions, joinCompiler, typeOperators); + return new SetBuilderOperator(operatorContext, setProvider, setChannel, expectedPositions, joinCompiler, typeOperators); } @Override @@ -116,14 +111,13 @@ public void noMoreOperators() @Override public OperatorFactory duplicate() { - return new SetBuilderOperatorFactory(operatorId, planNodeId, setProvider.getType(), setChannel, hashChannel, expectedPositions, joinCompiler, typeOperators); + return new SetBuilderOperatorFactory(operatorId, planNodeId, setProvider.getType(), setChannel, expectedPositions, joinCompiler, typeOperators); } } private final OperatorContext operatorContext; private final SetSupplier setSupplier; private final int setChannel; - private final int hashChannel; private final ChannelSetBuilder channelSetBuilder; @@ -133,7 +127,6 @@ public SetBuilderOperator( OperatorContext operatorContext, SetSupplier setSupplier, int setChannel, - Optional hashChannel, int expectedPositions, JoinCompiler joinCompiler, TypeOperators typeOperators) @@ -142,7 +135,6 @@ public SetBuilderOperator( this.setSupplier = requireNonNull(setSupplier, "setSupplier is null"); this.setChannel = setChannel; - this.hashChannel = hashChannel.orElse(-1); // Set builder has a single channel which goes in channel 0, if hash is present, add a hashBlock to channel 1 this.channelSetBuilder = new ChannelSetBuilder( @@ -189,7 +181,7 @@ public void addInput(Page page) requireNonNull(page, "page is null"); checkState(!isFinished(), "Operator is already finished"); - channelSetBuilder.addAll(page.getBlock(setChannel), hashChannel == -1 ? null : page.getBlock(hashChannel)); + channelSetBuilder.addAll(page.getBlock(setChannel)); } @Override diff --git a/core/trino-main/src/main/java/io/trino/operator/SimplePagesHashStrategy.java b/core/trino-main/src/main/java/io/trino/operator/SimplePagesHashStrategy.java index 02a57a0f445d..5083ac0238da 100644 --- a/core/trino-main/src/main/java/io/trino/operator/SimplePagesHashStrategy.java +++ b/core/trino-main/src/main/java/io/trino/operator/SimplePagesHashStrategy.java @@ -27,13 +27,11 @@ import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.ImmutableList.toImmutableList; import static io.airlift.slice.SizeOf.instanceSize; import static io.airlift.slice.SizeOf.sizeOf; -import static io.trino.spi.type.BigintType.BIGINT; import static java.util.Objects.requireNonNull; public class SimplePagesHashStrategy @@ -45,7 +43,6 @@ public class SimplePagesHashStrategy private final List outputChannels; private final List> channels; private final List hashChannels; - private final List precomputedHashChannel; private final Optional sortChannel; private final List equalOperators; private final List hashCodeOperators; @@ -56,7 +53,6 @@ public SimplePagesHashStrategy( List outputChannels, List> channels, List hashChannels, - OptionalInt precomputedHashChannel, Optional sortChannel, BlockTypeOperators blockTypeOperators) { @@ -69,12 +65,6 @@ public SimplePagesHashStrategy( checkArgument(types.size() == channels.size(), "Expected types and channels to be the same length"); this.hashChannels = ImmutableList.copyOf(requireNonNull(hashChannels, "hashChannels is null")); - if (precomputedHashChannel.isPresent()) { - this.precomputedHashChannel = channels.get(precomputedHashChannel.getAsInt()); - } - else { - this.precomputedHashChannel = null; - } this.sortChannel = requireNonNull(sortChannel, "sortChannel is null"); this.equalOperators = hashChannels.stream() @@ -123,9 +113,6 @@ public void appendTo(int blockIndex, int position, PageBuilder pageBuilder, int @Override public long hashPosition(int blockIndex, int position) { - if (precomputedHashChannel != null) { - return BIGINT.getLong(precomputedHashChannel.get(blockIndex), position); - } long result = 0; for (int i = 0; i < hashChannels.size(); i++) { Block block = channels.get(hashChannels.get(i)).get(blockIndex); diff --git a/core/trino-main/src/main/java/io/trino/operator/StreamingAggregationOperator.java b/core/trino-main/src/main/java/io/trino/operator/StreamingAggregationOperator.java index dcbbf7adbdbd..e7249016fbe9 100644 --- a/core/trino-main/src/main/java/io/trino/operator/StreamingAggregationOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/StreamingAggregationOperator.java @@ -34,7 +34,6 @@ import java.util.LinkedList; import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -210,7 +209,7 @@ private StreamingAggregation( .createPagesHashStrategy( sourceTypes.stream() .map(type -> new ObjectArrayList()) - .collect(toImmutableList()), OptionalInt.empty()); + .collect(toImmutableList())); this.aggregationMetrics = requireNonNull(aggregationMetrics, "aggregationMetrics is null"); } diff --git a/core/trino-main/src/main/java/io/trino/operator/WindowOperator.java b/core/trino-main/src/main/java/io/trino/operator/WindowOperator.java index 7e0d5a72024f..da404d435346 100644 --- a/core/trino-main/src/main/java/io/trino/operator/WindowOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/WindowOperator.java @@ -43,7 +43,6 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.OptionalInt; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -448,10 +447,10 @@ private static class PagesIndexWithHashStrategies List windowFunctionDefinitions) { this.pagesIndex = pagesIndexFactory.newPagesIndex(sourceTypes, expectedPositions); - this.preGroupedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preGroupedPartitionChannels, OptionalInt.empty()); - this.unGroupedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(unGroupedPartitionChannels, OptionalInt.empty()); - this.preSortedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preSortedChannels, OptionalInt.empty()); - this.peerGroupHashStrategy = pagesIndex.createPagesHashStrategy(sortChannels, OptionalInt.empty()); + this.preGroupedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preGroupedPartitionChannels); + this.unGroupedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(unGroupedPartitionChannels); + this.preSortedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preSortedChannels); + this.peerGroupHashStrategy = pagesIndex.createPagesHashStrategy(sortChannels); this.preGroupedPartitionChannels = Ints.toArray(preGroupedPartitionChannels); this.frameBoundComparators = createFrameBoundComparators(pagesIndex, windowFunctionDefinitions); } diff --git a/core/trino-main/src/main/java/io/trino/operator/function/TableFunctionOperator.java b/core/trino-main/src/main/java/io/trino/operator/function/TableFunctionOperator.java index 0143ff022f6f..26a62e30d420 100644 --- a/core/trino-main/src/main/java/io/trino/operator/function/TableFunctionOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/function/TableFunctionOperator.java @@ -41,7 +41,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.OptionalInt; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkPositionIndex; @@ -373,17 +372,17 @@ public HashStrategies( List sortOrders, int preSortedPrefix) { - this.prePartitionedStrategy = pagesIndex.createPagesHashStrategy(prePartitionedChannels, OptionalInt.empty()); + this.prePartitionedStrategy = pagesIndex.createPagesHashStrategy(prePartitionedChannels); List remainingPartitionChannels = partitionChannels.stream() .filter(channel -> !prePartitionedChannels.contains(channel)) .collect(toImmutableList()); - this.remainingPartitionStrategy = pagesIndex.createPagesHashStrategy(remainingPartitionChannels, OptionalInt.empty()); + this.remainingPartitionStrategy = pagesIndex.createPagesHashStrategy(remainingPartitionChannels); List preSortedChannels = sortChannels.stream() .limit(preSortedPrefix) .collect(toImmutableList()); - this.preSortedStrategy = pagesIndex.createPagesHashStrategy(preSortedChannels, OptionalInt.empty()); + this.preSortedStrategy = pagesIndex.createPagesHashStrategy(preSortedChannels); if (preSortedPrefix > 0) { // preSortedPrefix > 0 implies that all partition channels are already pre-partitioned (enforced by check in the constructor), so we only need to do the remaining sort diff --git a/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java b/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java index 3a7bb3e5e997..a411ea32764e 100644 --- a/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java +++ b/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java @@ -42,7 +42,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.OptionalInt; import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -70,7 +69,6 @@ public class IndexLoader private final AtomicReference taskContextReference = new AtomicReference<>(); private final Set lookupSourceInputChannels; private final List keyOutputChannels; - private final OptionalInt keyOutputHashChannel; private final List keyTypes; private final List keyEqualOperators; private final PagesIndex.Factory pagesIndexFactory; @@ -87,7 +85,6 @@ public class IndexLoader public IndexLoader( Set lookupSourceInputChannels, List keyOutputChannels, - OptionalInt keyOutputHashChannel, List outputTypes, IndexBuildDriverFactoryProvider indexBuildDriverFactoryProvider, int expectedPositions, @@ -101,7 +98,6 @@ public IndexLoader( checkArgument(!lookupSourceInputChannels.isEmpty(), "lookupSourceInputChannels must not be empty"); requireNonNull(keyOutputChannels, "keyOutputChannels is null"); checkArgument(!keyOutputChannels.isEmpty(), "keyOutputChannels must not be empty"); - requireNonNull(keyOutputHashChannel, "keyOutputHashChannel is null"); checkArgument(lookupSourceInputChannels.size() <= keyOutputChannels.size(), "Lookup channels must supply a subset of the actual index columns"); requireNonNull(outputTypes, "outputTypes is null"); requireNonNull(indexBuildDriverFactoryProvider, "indexBuildDriverFactoryProvider is null"); @@ -112,7 +108,6 @@ public IndexLoader( this.lookupSourceInputChannels = ImmutableSet.copyOf(lookupSourceInputChannels); this.keyOutputChannels = ImmutableList.copyOf(keyOutputChannels); - this.keyOutputHashChannel = keyOutputHashChannel; this.outputTypes = ImmutableList.copyOf(outputTypes); this.indexBuildDriverFactoryProvider = indexBuildDriverFactoryProvider; this.expectedPositions = expectedPositions; @@ -260,7 +255,6 @@ private synchronized void initializeStateIfNecessary() lookupSourceInputChannels, keyTypes, keyOutputChannels, - keyOutputHashChannel, expectedPositions, maxIndexMemorySize, pagesIndexFactory, @@ -288,7 +282,6 @@ private IndexSnapshotLoader( Set lookupSourceInputChannels, List indexTypes, List keyOutputChannels, - OptionalInt keyOutputHashChannel, int expectedPositions, DataSize maxIndexMemorySize, PagesIndex.Factory pagesIndexFactory, @@ -304,7 +297,6 @@ private IndexSnapshotLoader( this.indexSnapshotBuilder = new IndexSnapshotBuilder( outputTypes, keyOutputChannels, - keyOutputHashChannel, pipelineContext.addDriverContext(), maxIndexMemorySize, expectedPositions, diff --git a/core/trino-main/src/main/java/io/trino/operator/index/IndexLookupSourceFactory.java b/core/trino-main/src/main/java/io/trino/operator/index/IndexLookupSourceFactory.java index 66d3115a8f16..b8592da77876 100644 --- a/core/trino-main/src/main/java/io/trino/operator/index/IndexLookupSourceFactory.java +++ b/core/trino-main/src/main/java/io/trino/operator/index/IndexLookupSourceFactory.java @@ -29,7 +29,6 @@ import io.trino.type.BlockTypeOperators; import java.util.List; -import java.util.OptionalInt; import java.util.Set; import java.util.function.Supplier; @@ -49,7 +48,6 @@ public class IndexLookupSourceFactory public IndexLookupSourceFactory( Set lookupSourceInputChannels, List keyOutputChannels, - OptionalInt keyOutputHashChannel, List outputTypes, IndexBuildDriverFactoryProvider indexBuildDriverFactoryProvider, DataSize maxIndexMemorySize, @@ -65,7 +63,6 @@ public IndexLookupSourceFactory( IndexLoader shared = new IndexLoader( lookupSourceInputChannels, keyOutputChannels, - keyOutputHashChannel, outputTypes, indexBuildDriverFactoryProvider, 10_000, @@ -80,7 +77,6 @@ public IndexLookupSourceFactory( this.indexLoaderSupplier = () -> new IndexLoader( lookupSourceInputChannels, keyOutputChannels, - keyOutputHashChannel, outputTypes, indexBuildDriverFactoryProvider, 10_000, diff --git a/core/trino-main/src/main/java/io/trino/operator/index/IndexSnapshotBuilder.java b/core/trino-main/src/main/java/io/trino/operator/index/IndexSnapshotBuilder.java index c0e13b7f0748..129161625eea 100644 --- a/core/trino-main/src/main/java/io/trino/operator/index/IndexSnapshotBuilder.java +++ b/core/trino-main/src/main/java/io/trino/operator/index/IndexSnapshotBuilder.java @@ -28,7 +28,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -42,7 +41,6 @@ public class IndexSnapshotBuilder private final List outputTypes; private final List missingKeysTypes; private final List keyOutputChannels; - private final OptionalInt keyOutputHashChannel; private final List missingKeysChannels; private final PagesIndex.Factory pagesIndexFactory; @@ -58,7 +56,6 @@ public class IndexSnapshotBuilder public IndexSnapshotBuilder(List outputTypes, List keyOutputChannels, - OptionalInt keyOutputHashChannel, DriverContext driverContext, DataSize maxMemoryInBytes, int expectedPositions, @@ -66,7 +63,6 @@ public IndexSnapshotBuilder(List outputTypes, { requireNonNull(outputTypes, "outputTypes is null"); requireNonNull(keyOutputChannels, "keyOutputChannels is null"); - requireNonNull(keyOutputHashChannel, "keyOutputHashChannel is null"); requireNonNull(driverContext, "driverContext is null"); requireNonNull(maxMemoryInBytes, "maxMemoryInBytes is null"); requireNonNull(pagesIndexFactory, "pagesIndexFactory is null"); @@ -77,7 +73,6 @@ public IndexSnapshotBuilder(List outputTypes, this.outputTypes = ImmutableList.copyOf(outputTypes); this.expectedPositions = expectedPositions; this.keyOutputChannels = ImmutableList.copyOf(keyOutputChannels); - this.keyOutputHashChannel = keyOutputHashChannel; this.maxMemoryInBytes = maxMemoryInBytes.toBytes(); ImmutableList.Builder missingKeysTypes = ImmutableList.builder(); @@ -131,7 +126,7 @@ public IndexSnapshot createIndexSnapshot(UnloadedIndexKeyRecordSet indexKeysReco } pages.clear(); - LookupSource lookupSource = outputPagesIndex.createLookupSourceSupplier(session, keyOutputChannels, keyOutputHashChannel, Optional.empty(), Optional.empty(), ImmutableList.of()).get(); + LookupSource lookupSource = outputPagesIndex.createLookupSourceSupplier(session, keyOutputChannels, Optional.empty(), Optional.empty(), ImmutableList.of()).get(); // Build a page containing the keys that produced no output rows, so in future requests can skip these keys verify(missingKeysPageBuilder.isEmpty()); diff --git a/core/trino-main/src/main/java/io/trino/operator/join/HashBuilderOperator.java b/core/trino-main/src/main/java/io/trino/operator/join/HashBuilderOperator.java index 52bd2d6bd548..d329b386aa04 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/HashBuilderOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/HashBuilderOperator.java @@ -44,7 +44,6 @@ import java.util.ArrayDeque; import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Queue; @@ -76,7 +75,6 @@ public static class HashBuilderOperatorFactory private final JoinBridgeManager lookupSourceFactoryManager; private final List outputChannels; private final List hashChannels; - private final OptionalInt preComputedHashChannel; private final Optional filterFunctionFactory; private final Optional sortChannel; private final List searchFunctionFactories; @@ -97,7 +95,6 @@ public HashBuilderOperatorFactory( JoinBridgeManager lookupSourceFactoryManager, List outputChannels, List hashChannels, - OptionalInt preComputedHashChannel, Optional filterFunctionFactory, Optional sortChannel, List searchFunctionFactories, @@ -116,7 +113,6 @@ public HashBuilderOperatorFactory( this.outputChannels = ImmutableList.copyOf(requireNonNull(outputChannels, "outputChannels is null")); this.hashChannels = ImmutableList.copyOf(requireNonNull(hashChannels, "hashChannels is null")); - this.preComputedHashChannel = requireNonNull(preComputedHashChannel, "preComputedHashChannel is null"); this.filterFunctionFactory = requireNonNull(filterFunctionFactory, "filterFunctionFactory is null"); this.sortChannel = sortChannel; this.searchFunctionFactories = ImmutableList.copyOf(searchFunctionFactories); @@ -143,7 +139,6 @@ public HashBuilderOperator createOperator(DriverContext driverContext) partitionIndex - 1, outputChannels, hashChannels, - preComputedHashChannel, filterFunctionFactory, sortChannel, searchFunctionFactories, @@ -218,7 +213,6 @@ public enum State private final List outputChannels; private final List hashChannels; - private final OptionalInt preComputedHashChannel; private final Optional filterFunctionFactory; private final Optional sortChannel; private final List searchFunctionFactories; @@ -251,7 +245,6 @@ public HashBuilderOperator( int partitionIndex, List outputChannels, List hashChannels, - OptionalInt preComputedHashChannel, Optional filterFunctionFactory, Optional sortChannel, List searchFunctionFactories, @@ -278,7 +271,6 @@ public HashBuilderOperator( this.outputChannels = outputChannels; this.hashChannels = hashChannels; - this.preComputedHashChannel = preComputedHashChannel; this.spillEnabled = spillEnabled; this.singleStreamSpillerFactory = requireNonNull(singleStreamSpillerFactory, "singleStreamSpillerFactory is null"); @@ -656,7 +648,7 @@ private void disposeUnspilledLookupSourceIfRequested() private LookupSourceSupplier buildLookupSource() { - LookupSourceSupplier partition = index.createLookupSourceSupplier(operatorContext.getSession(), hashChannels, preComputedHashChannel, filterFunctionFactory, sortChannel, searchFunctionFactories, Optional.of(outputChannels), hashArraySizeSupplier); + LookupSourceSupplier partition = index.createLookupSourceSupplier(operatorContext.getSession(), hashChannels, filterFunctionFactory, sortChannel, searchFunctionFactories, Optional.of(outputChannels), hashArraySizeSupplier); checkState(lookupSourceSupplier == null, "lookupSourceSupplier is already set"); this.lookupSourceSupplier = partition; return partition; diff --git a/core/trino-main/src/main/java/io/trino/operator/join/JoinProbe.java b/core/trino-main/src/main/java/io/trino/operator/join/JoinProbe.java index 9480ad472e36..ae226a6b6c58 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/JoinProbe.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/JoinProbe.java @@ -15,14 +15,10 @@ import com.google.common.primitives.Ints; import io.trino.spi.Page; -import io.trino.spi.block.Block; -import jakarta.annotation.Nullable; import java.util.List; -import java.util.OptionalInt; import static com.google.common.base.Verify.verify; -import static io.trino.spi.type.BigintType.BIGINT; public class JoinProbe { @@ -30,19 +26,17 @@ public static class JoinProbeFactory { private final int[] probeOutputChannels; private final int[] probeJoinChannels; - private final int probeHashChannel; // only valid when >= 0 - public JoinProbeFactory(int[] probeOutputChannels, List probeJoinChannels, OptionalInt probeHashChannel) + public JoinProbeFactory(int[] probeOutputChannels, List probeJoinChannels) { this.probeOutputChannels = probeOutputChannels; this.probeJoinChannels = Ints.toArray(probeJoinChannels); - this.probeHashChannel = probeHashChannel.orElse(-1); } public JoinProbe createJoinProbe(Page page) { Page probePage = page.getColumns(probeJoinChannels); - return new JoinProbe(probeOutputChannels, page, probePage, probeHashChannel >= 0 ? page.getBlock(probeHashChannel) : null); + return new JoinProbe(probeOutputChannels, page, probePage); } } @@ -50,18 +44,15 @@ public JoinProbe createJoinProbe(Page page) private final int positionCount; private final Page page; private final Page probePage; - @Nullable - private final Block probeHashBlock; private final boolean probeMayHaveNull; private int position = -1; - private JoinProbe(int[] probeOutputChannels, Page page, Page probePage, @Nullable Block probeHashBlock) + private JoinProbe(int[] probeOutputChannels, Page page, Page probePage) { this.probeOutputChannels = probeOutputChannels; this.positionCount = page.getPositionCount(); this.page = page; this.probePage = probePage; - this.probeHashBlock = probeHashBlock; this.probeMayHaveNull = probeMayHaveNull(probePage); } @@ -86,10 +77,6 @@ public long getCurrentJoinPosition(LookupSource lookupSource) if (probeMayHaveNull && currentRowContainsNull()) { return -1; } - if (probeHashBlock != null) { - long rawHash = BIGINT.getLong(probeHashBlock, position); - return lookupSource.getJoinPosition(position, probePage, page, rawHash); - } return lookupSource.getJoinPosition(position, probePage, page); } diff --git a/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java index ccadb0a92f04..0b65be198371 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java @@ -18,7 +18,6 @@ import io.trino.operator.HashGenerator; import io.trino.operator.JoinOperatorType; import io.trino.operator.OperatorFactory; -import io.trino.operator.PrecomputedHashGenerator; import io.trino.operator.ProcessorContext; import io.trino.operator.WorkProcessor; import io.trino.operator.WorkProcessorOperator; @@ -82,7 +81,6 @@ public LookupJoinOperatorFactory( TypeOperators typeOperators, OptionalInt totalOperatorsCount, List probeJoinChannels, - OptionalInt probeHashChannel, PartitioningSpillerFactory partitioningSpillerFactory) { this.operatorId = operatorId; @@ -110,17 +108,11 @@ public LookupJoinOperatorFactory( } this.totalOperatorsCount = requireNonNull(totalOperatorsCount, "totalOperatorsCount is null"); - requireNonNull(probeHashChannel, "probeHashChannel is null"); - if (probeHashChannel.isPresent()) { - this.probeHashGenerator = new PrecomputedHashGenerator(probeHashChannel.getAsInt()); - } - else { - requireNonNull(probeJoinChannels, "probeJoinChannels is null"); - List hashTypes = probeJoinChannels.stream() - .map(probeTypes::get) - .collect(toImmutableList()); - this.probeHashGenerator = createChannelsHashGenerator(hashTypes, Ints.toArray(probeJoinChannels), typeOperators); - } + requireNonNull(probeJoinChannels, "probeJoinChannels is null"); + List hashTypes = probeJoinChannels.stream() + .map(probeTypes::get) + .collect(toImmutableList()); + this.probeHashGenerator = createChannelsHashGenerator(hashTypes, Ints.toArray(probeJoinChannels), typeOperators); this.partitioningSpillerFactory = requireNonNull(partitioningSpillerFactory, "partitioningSpillerFactory is null"); } diff --git a/core/trino-main/src/main/java/io/trino/operator/join/unspilled/HashBuilderOperator.java b/core/trino-main/src/main/java/io/trino/operator/join/unspilled/HashBuilderOperator.java index f752248bf502..e3115dcf9e19 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/unspilled/HashBuilderOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/unspilled/HashBuilderOperator.java @@ -33,7 +33,6 @@ import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -57,7 +56,6 @@ public static class HashBuilderOperatorFactory private final JoinBridgeManager lookupSourceFactoryManager; private final List outputChannels; private final List hashChannels; - private final OptionalInt preComputedHashChannel; private final Optional filterFunctionFactory; private final Optional sortChannel; private final List searchFunctionFactories; @@ -76,7 +74,6 @@ public HashBuilderOperatorFactory( JoinBridgeManager lookupSourceFactoryManager, List outputChannels, List hashChannels, - OptionalInt preComputedHashChannel, Optional filterFunctionFactory, Optional sortChannel, List searchFunctionFactories, @@ -93,7 +90,6 @@ public HashBuilderOperatorFactory( this.outputChannels = ImmutableList.copyOf(requireNonNull(outputChannels, "outputChannels is null")); this.hashChannels = ImmutableList.copyOf(requireNonNull(hashChannels, "hashChannels is null")); - this.preComputedHashChannel = requireNonNull(preComputedHashChannel, "preComputedHashChannel is null"); this.filterFunctionFactory = requireNonNull(filterFunctionFactory, "filterFunctionFactory is null"); this.sortChannel = sortChannel; this.searchFunctionFactories = ImmutableList.copyOf(searchFunctionFactories); @@ -118,7 +114,6 @@ public HashBuilderOperator createOperator(DriverContext driverContext) partitionIndex - 1, outputChannels, hashChannels, - preComputedHashChannel, filterFunctionFactory, sortChannel, searchFunctionFactories, @@ -167,7 +162,6 @@ public enum State private final List outputChannels; private final List hashChannels; - private final OptionalInt preComputedHashChannel; private final Optional filterFunctionFactory; private final Optional sortChannel; private final List searchFunctionFactories; @@ -186,7 +180,6 @@ public HashBuilderOperator( int partitionIndex, List outputChannels, List hashChannels, - OptionalInt preComputedHashChannel, Optional filterFunctionFactory, Optional sortChannel, List searchFunctionFactories, @@ -194,7 +187,7 @@ public HashBuilderOperator( PagesIndex.Factory pagesIndexFactory, HashArraySizeSupplier hashArraySizeSupplier) { - this(operatorContext, lookupSourceFactory, partitionIndex, outputChannels, hashChannels, preComputedHashChannel, filterFunctionFactory, sortChannel, searchFunctionFactories, expectedPositions, pagesIndexFactory, hashArraySizeSupplier, DEFAULT_GRANULARITY); + this(operatorContext, lookupSourceFactory, partitionIndex, outputChannels, hashChannels, filterFunctionFactory, sortChannel, searchFunctionFactories, expectedPositions, pagesIndexFactory, hashArraySizeSupplier, DEFAULT_GRANULARITY); } @VisibleForTesting @@ -204,7 +197,6 @@ public HashBuilderOperator( int partitionIndex, List outputChannels, List hashChannels, - OptionalInt preComputedHashChannel, Optional filterFunctionFactory, Optional sortChannel, List searchFunctionFactories, @@ -228,7 +220,6 @@ public HashBuilderOperator( this.outputChannels = outputChannels; this.hashChannels = hashChannels; - this.preComputedHashChannel = preComputedHashChannel; this.hashArraySizeSupplier = requireNonNull(hashArraySizeSupplier, "hashArraySizeSupplier is null"); } @@ -354,7 +345,7 @@ private void disposeLookupSourceIfRequested() private LookupSourceSupplier buildLookupSource() { checkState(index != null, "index is null"); - LookupSourceSupplier partition = index.createLookupSourceSupplier(operatorContext.getSession(), hashChannels, preComputedHashChannel, filterFunctionFactory, sortChannel, searchFunctionFactories, Optional.of(outputChannels), hashArraySizeSupplier); + LookupSourceSupplier partition = index.createLookupSourceSupplier(operatorContext.getSession(), hashChannels, filterFunctionFactory, sortChannel, searchFunctionFactories, Optional.of(outputChannels), hashArraySizeSupplier); checkState(lookupSourceSupplier == null, "lookupSourceSupplier is already set"); this.lookupSourceSupplier = partition; return partition; diff --git a/core/trino-main/src/main/java/io/trino/operator/join/unspilled/JoinProbe.java b/core/trino-main/src/main/java/io/trino/operator/join/unspilled/JoinProbe.java index b97bef1b3146..037417c60824 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/unspilled/JoinProbe.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/unspilled/JoinProbe.java @@ -18,14 +18,11 @@ import io.trino.spi.Page; import io.trino.spi.block.Block; import io.trino.spi.block.RunLengthEncodedBlock; -import jakarta.annotation.Nullable; import java.util.Arrays; import java.util.List; -import java.util.OptionalInt; import static com.google.common.base.Verify.verify; -import static io.trino.spi.type.BigintType.BIGINT; import static java.util.Objects.requireNonNull; /** @@ -38,21 +35,19 @@ public static class JoinProbeFactory { private final int[] probeOutputChannels; private final int[] probeJoinChannels; - private final int probeHashChannel; // only valid when >= 0 private final boolean hasFilter; - public JoinProbeFactory(List probeOutputChannels, List probeJoinChannels, OptionalInt probeHashChannel, boolean hasFilter) + public JoinProbeFactory(List probeOutputChannels, List probeJoinChannels, boolean hasFilter) { this.probeOutputChannels = Ints.toArray(requireNonNull(probeOutputChannels, "probeOutputChannels is null")); this.probeJoinChannels = Ints.toArray(requireNonNull(probeJoinChannels, "probeJoinChannels is null")); - this.probeHashChannel = requireNonNull(probeHashChannel, "probeHashChannel is null").orElse(-1); this.hasFilter = hasFilter; } public JoinProbe createJoinProbe(Page page, LookupSource lookupSource) { Page probePage = page.getColumns(probeJoinChannels); - return new JoinProbe(probeOutputChannels, page, probePage, lookupSource, probeHashChannel >= 0 ? page.getBlock(probeHashChannel) : null, hasFilter); + return new JoinProbe(probeOutputChannels, page, probePage, lookupSource, hasFilter); } } @@ -62,7 +57,7 @@ public JoinProbe createJoinProbe(Page page, LookupSource lookupSource) private final boolean isRle; private int position = -1; - private JoinProbe(int[] probeOutputChannels, Page page, Page probePage, LookupSource lookupSource, @Nullable Block probeHashBlock, boolean hasFilter) + private JoinProbe(int[] probeOutputChannels, Page page, Page probePage, LookupSource lookupSource, boolean hasFilter) { this.probeOutputChannels = requireNonNull(probeOutputChannels, "probeOutputChannels is null"); this.page = requireNonNull(page, "page is null"); @@ -70,7 +65,7 @@ private JoinProbe(int[] probeOutputChannels, Page page, Page probePage, LookupSo // if filter channels are not RLE encoded, then every probe // row might be unique and must be matched independently this.isRle = !hasFilter && hasOnlyRleBlocks(probePage); - joinPositionCache = fillCache(lookupSource, page, probeHashBlock, probePage, isRle); + joinPositionCache = fillCache(lookupSource, page, probePage, isRle); } public int[] getOutputChannels() @@ -117,7 +112,6 @@ public Page getPage() private static long[] fillCache( LookupSource lookupSource, Page page, - Block probeHashBlock, Page probePage, boolean isRle) { @@ -172,16 +166,7 @@ private static long[] fillCache( // This way less code is in the if branch and CPU should be able to optimize branch prediction better nonNullCount += isNull[i] ? 0 : 1; } - if (probeHashBlock != null) { - long[] hashes = new long[positionCount]; - for (int i = 0; i < positionCount; i++) { - hashes[i] = BIGINT.getLong(probeHashBlock, i); - } - lookupSource.getJoinPosition(positions, probePage, page, hashes, joinPositionCache); - } - else { - lookupSource.getJoinPosition(positions, probePage, page, joinPositionCache); - } + lookupSource.getJoinPosition(positions, probePage, page, joinPositionCache); return joinPositionCache; } // else fall back to non-null path } @@ -189,16 +174,7 @@ private static long[] fillCache( for (int i = 0; i < positionCount; i++) { positions[i] = i; } - if (probeHashBlock != null) { - long[] hashes = new long[positionCount]; - for (int i = 0; i < positionCount; i++) { - hashes[i] = BIGINT.getLong(probeHashBlock, i); - } - lookupSource.getJoinPosition(positions, probePage, page, hashes, joinPositionCache); - } - else { - lookupSource.getJoinPosition(positions, probePage, page, joinPositionCache); - } + lookupSource.getJoinPosition(positions, probePage, page, joinPositionCache); return joinPositionCache; } diff --git a/core/trino-main/src/main/java/io/trino/sql/gen/JoinCompiler.java b/core/trino-main/src/main/java/io/trino/sql/gen/JoinCompiler.java index dce5a83b10c8..90d0dc52cdf6 100644 --- a/core/trino-main/src/main/java/io/trino/sql/gen/JoinCompiler.java +++ b/core/trino-main/src/main/java/io/trino/sql/gen/JoinCompiler.java @@ -79,13 +79,11 @@ import static io.airlift.bytecode.expression.BytecodeExpressions.constantFalse; import static io.airlift.bytecode.expression.BytecodeExpressions.constantInt; import static io.airlift.bytecode.expression.BytecodeExpressions.constantLong; -import static io.airlift.bytecode.expression.BytecodeExpressions.constantNull; import static io.airlift.bytecode.expression.BytecodeExpressions.constantTrue; import static io.airlift.bytecode.expression.BytecodeExpressions.getStatic; import static io.airlift.bytecode.expression.BytecodeExpressions.invokeDynamic; import static io.airlift.bytecode.expression.BytecodeExpressions.invokeStatic; import static io.airlift.bytecode.expression.BytecodeExpressions.newInstance; -import static io.airlift.bytecode.expression.BytecodeExpressions.notEqual; import static io.airlift.bytecode.expression.BytecodeExpressions.setStatic; import static io.trino.cache.SafeCaches.buildNonEvictableCache; import static io.trino.operator.join.JoinUtils.getSingleBigintJoinChannel; @@ -94,7 +92,6 @@ import static io.trino.spi.function.InvocationConvention.InvocationReturnConvention.DEFAULT_ON_NULL; import static io.trino.spi.function.InvocationConvention.InvocationReturnConvention.FAIL_ON_NULL; import static io.trino.spi.function.InvocationConvention.simpleConvention; -import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.sql.gen.Bootstrap.BOOTSTRAP_METHOD; import static io.trino.sql.gen.SqlTypeBytecodeExpression.constantType; import static io.trino.util.CompilerUtils.defineClass; @@ -236,13 +233,12 @@ private Class internalCompileHashStrategy(List channelFields, - List joinChannelFields, - FieldDefinition hashChannelField) + List joinChannelFields) { Parameter channels = arg("channels", type(List.class, type(List.class, Block.class))); - Parameter hashChannel = arg("hashChannel", type(OptionalInt.class)); - MethodDefinition constructorDefinition = classDefinition.declareConstructor(a(PUBLIC), channels, hashChannel); + MethodDefinition constructorDefinition = classDefinition.declareConstructor(a(PUBLIC), channels); Variable thisVariable = constructorDefinition.getThis(); @@ -304,16 +298,6 @@ private static void generateConstructor( constructor.append(thisVariable.setField(joinChannelFields.get(index), joinChannel)); } - - constructor.comment("Set hashChannel"); - constructor.append(new IfStatement() - .condition(hashChannel.invoke("isPresent", boolean.class)) - .ifTrue(thisVariable.setField( - hashChannelField, - channels.invoke("get", Object.class, hashChannel.invoke("getAsInt", int.class)))) - .ifFalse(thisVariable.setField( - hashChannelField, - constantNull(hashChannelField.getType())))); constructor.ret(); } @@ -406,7 +390,7 @@ private static void generateIsPositionNull(ClassDefinition classDefinition, List .append(constantFalse().ret()); } - private void generateHashPositionMethod(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, List joinChannelTypes, List joinChannelFields, FieldDefinition hashChannelField) + private void generateHashPositionMethod(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, List joinChannelTypes, List joinChannelFields) { Parameter blockIndex = arg("blockIndex", int.class); Parameter blockPosition = arg("blockPosition", int.class); @@ -417,24 +401,6 @@ private void generateHashPositionMethod(ClassDefinition classDefinition, CallSit blockIndex, blockPosition); - Variable thisVariable = hashPositionMethod.getThis(); - BytecodeExpression hashChannel = thisVariable.getField(hashChannelField); - BytecodeExpression bigintType = constantType(callSiteBinder, BIGINT); - - IfStatement ifStatement = new IfStatement(); - ifStatement.condition(notEqual(hashChannel, constantNull(hashChannelField.getType()))); - ifStatement.ifTrue( - bigintType.invoke( - "getLong", - long.class, - hashChannel.invoke("get", Object.class, blockIndex).cast(Block.class), - blockPosition) - .ret()); - - hashPositionMethod - .getBody() - .append(ifStatement); - Variable resultVariable = hashPositionMethod.getScope().declareVariable(long.class, "result"); hashPositionMethod.getBody().push(0L).putVariable(resultVariable); @@ -1052,13 +1018,12 @@ public LookupSourceSupplier createLookupSourceSupplier( Session session, LongArrayList addresses, List> channels, - OptionalInt hashChannel, Optional filterFunctionFactory, Optional sortChannel, List searchFunctionFactories, HashArraySizeSupplier hashArraySizeSupplier) { - PagesHashStrategy pagesHashStrategy = pagesHashStrategyFactory.createPagesHashStrategy(channels, hashChannel); + PagesHashStrategy pagesHashStrategy = pagesHashStrategyFactory.createPagesHashStrategy(channels); try { return constructor.newInstance(session, pagesHashStrategy, addresses, channels, filterFunctionFactory, sortChannel, searchFunctionFactories, hashArraySizeSupplier, singleBigintJoinChannel); } @@ -1075,17 +1040,17 @@ public static class PagesHashStrategyFactory public PagesHashStrategyFactory(Class pagesHashStrategyClass) { try { - constructor = pagesHashStrategyClass.getConstructor(List.class, OptionalInt.class); + constructor = pagesHashStrategyClass.getConstructor(List.class); } catch (NoSuchMethodException e) { throw new RuntimeException(e); } } - public PagesHashStrategy createPagesHashStrategy(List> channels, OptionalInt hashChannel) + public PagesHashStrategy createPagesHashStrategy(List> channels) { try { - return constructor.newInstance(channels, hashChannel); + return constructor.newInstance(channels); } catch (ReflectiveOperationException e) { throw new RuntimeException(e); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java index f587ccf2f63e..33ea7030aa9d 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java @@ -2439,8 +2439,6 @@ public PhysicalOperation visitIndexJoin(IndexJoinNode node, LocalExecutionPlanCo // Plan probe side PhysicalOperation probeSource = node.getProbeSource().accept(this, context); List probeChannels = getChannelsForSymbols(probeSymbols, probeSource.getLayout()); - OptionalInt probeHashChannel = node.getProbeHashSymbol().map(channelGetter(probeSource)) - .map(OptionalInt::of).orElse(OptionalInt.empty()); // The probe key channels will be handed to the index according to probeSymbol order Map probeKeyLayout = new HashMap<>(); @@ -2454,8 +2452,6 @@ public PhysicalOperation visitIndexJoin(IndexJoinNode node, LocalExecutionPlanCo LocalExecutionPlanContext indexContext = context.createIndexSourceSubContext(new IndexSourceContext(indexLookupToProbeInput)); PhysicalOperation indexSource = node.getIndexSource().accept(this, indexContext); List indexOutputChannels = getChannelsForSymbols(indexSymbols, indexSource.getLayout()); - OptionalInt indexHashChannel = node.getIndexHashSymbol().map(channelGetter(indexSource)) - .map(OptionalInt::of).orElse(OptionalInt.empty()); // Identify just the join keys/channels needed for lookup by the index source (does not have to use all of them). Set indexSymbolsNeededBySource = IndexJoinOptimizer.IndexKeyTracer.trace(node.getIndexSource(), ImmutableSet.copyOf(indexSymbols)).keySet(); @@ -2502,7 +2498,6 @@ public PhysicalOperation visitIndexJoin(IndexJoinNode node, LocalExecutionPlanCo IndexLookupSourceFactory indexLookupSourceFactory = new IndexLookupSourceFactory( lookupSourceInputChannels, indexOutputChannels, - indexHashChannel, indexSource.getTypes(), indexBuildDriverFactoryProvider, maxIndexMemorySize, @@ -2540,7 +2535,6 @@ public PhysicalOperation visitIndexJoin(IndexJoinNode node, LocalExecutionPlanCo lookupSourceFactoryManager, probeSource.getTypes(), probeChannels, - probeHashChannel, Optional.empty(), totalOperatorsCount, unsupportedPartitioningSpillerFactory(), @@ -2552,7 +2546,6 @@ public PhysicalOperation visitIndexJoin(IndexJoinNode node, LocalExecutionPlanCo lookupSourceFactoryManager, probeSource.getTypes(), probeChannels, - probeHashChannel, Optional.empty(), totalOperatorsCount, unsupportedPartitioningSpillerFactory(), @@ -2582,7 +2575,7 @@ public PhysicalOperation visitJoin(JoinNode node, LocalExecutionPlanContext cont return switch (node.getType()) { case INNER, LEFT, RIGHT, FULL -> - createLookupJoin(node, node.getLeft(), leftSymbols, node.getLeftHashSymbol(), node.getRight(), rightSymbols, node.getRightHashSymbol(), localDynamicFilters, context); + createLookupJoin(node, node.getLeft(), leftSymbols, node.getRight(), rightSymbols, localDynamicFilters, context); }; } @@ -2900,10 +2893,8 @@ private PhysicalOperation createLookupJoin( JoinNode node, PlanNode probeNode, List probeSymbols, - Optional probeHashSymbol, PlanNode buildNode, List buildSymbols, - Optional buildHashSymbol, Set localDynamicFilters, LocalExecutionPlanContext context) { @@ -2920,8 +2911,6 @@ private PhysicalOperation createLookupJoin( List probeTypes = probeSource.getTypes(); List probeOutputChannels = ImmutableList.copyOf(getChannelsForSymbols(node.getLeftOutputSymbols(), probeSource.getLayout())); List probeJoinChannels = ImmutableList.copyOf(getChannelsForSymbols(probeSymbols, probeSource.getLayout())); - OptionalInt probeHashChannel = probeHashSymbol.map(channelGetter(probeSource)) - .map(OptionalInt::of).orElse(OptionalInt.empty()); OptionalInt totalOperatorsCount = OptionalInt.empty(); if (spillEnabled) { totalOperatorsCount = context.getDriverInstanceCount(); @@ -2943,9 +2932,6 @@ private PhysicalOperation createLookupJoin( List buildOutputChannels = ImmutableList.copyOf(getChannelsForSymbols(node.getRightOutputSymbols(), buildSource.getLayout())); List buildChannels = ImmutableList.copyOf(getChannelsForSymbols(buildSymbols, buildSource.getLayout())); - OptionalInt buildHashChannel = buildHashSymbol.map(channelGetter(buildSource)) - .map(OptionalInt::of).orElse(OptionalInt.empty()); - int partitionCount = buildContext.getDriverInstanceCount().orElse(1); Map buildLayout = buildSource.getLayout(); @@ -3016,7 +3002,6 @@ private PhysicalOperation createLookupJoin( lookupSourceFactory, buildOutputChannels, buildChannels, - buildHashChannel, filterFunctionFactory, sortChannel, searchFunctionFactories, @@ -3043,7 +3028,6 @@ private PhysicalOperation createLookupJoin( lookupSourceFactory, probeTypes, probeJoinChannels, - probeHashChannel, Optional.of(probeOutputChannels), totalOperatorsCount, partitioningSpillerFactory, @@ -3069,7 +3053,6 @@ private PhysicalOperation createLookupJoin( lookupSourceFactory, buildOutputChannels, buildChannels, - buildHashChannel, filterFunctionFactory, sortChannel, searchFunctionFactories, @@ -3095,7 +3078,6 @@ private PhysicalOperation createLookupJoin( node.getFilter().isPresent(), probeTypes, probeJoinChannels, - probeHashChannel, Optional.of(probeOutputChannels)); } @@ -3306,15 +3288,11 @@ public PhysicalOperation visitSemiJoin(SemiJoinNode node, LocalExecutionPlanCont buildSource); } - Optional buildHashChannel = node.getFilteringSourceHashSymbol().map(channelGetter(buildSource)); - Optional probeHashChannel = node.getSourceHashSymbol().map(channelGetter(probeSource)); - SetBuilderOperatorFactory setBuilderOperatorFactory = new SetBuilderOperatorFactory( buildContext.getNextOperatorId(), node.getId(), buildSource.getTypes().get(buildChannel), buildChannel, - buildHashChannel, 10_000, joinCompiler, typeOperators); @@ -3330,7 +3308,7 @@ public PhysicalOperation visitSemiJoin(SemiJoinNode node, LocalExecutionPlanCont .put(node.getSemiJoinOutput(), probeSource.getLayout().size()) .buildOrThrow(); - OperatorFactory operator = HashSemiJoinOperator.createOperatorFactory(context.getNextOperatorId(), node.getId(), setProvider, probeSource.getTypes(), probeChannel, probeHashChannel); + OperatorFactory operator = HashSemiJoinOperator.createOperatorFactory(context.getNextOperatorId(), node.getId(), setProvider, probeSource.getTypes(), probeChannel); return new PhysicalOperation(operator, outputMappings, probeSource); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/PlanCopier.java b/core/trino-main/src/main/java/io/trino/sql/planner/PlanCopier.java index 079f64482063..08f1eaf97860 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/PlanCopier.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/PlanCopier.java @@ -169,8 +169,6 @@ public PlanNode visitJoin(JoinNode node, RewriteContext context) node.getRightOutputSymbols(), node.isMaySkipOutputDuplicates(), node.getFilter(), - node.getLeftHashSymbol(), - node.getRightHashSymbol(), node.getDistributionType(), node.isSpillable(), node.getDynamicFilters(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/RelationPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/RelationPlanner.java index 22caa77d1aab..724bea63b14a 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/RelationPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/RelationPlanner.java @@ -1011,8 +1011,6 @@ else if (firstDependencies.stream().allMatch(right::canResolve) && secondDepende Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); @@ -1053,8 +1051,6 @@ else if (firstDependencies.stream().allMatch(right::canResolve) && secondDepende .collect(Collectors.toList()))), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); } @@ -1167,8 +1163,6 @@ If casts are redundant (due to column type and common type being equal), Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/AdaptiveReorderPartitionedJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/AdaptiveReorderPartitionedJoin.java index 91e550de055c..bb56c151d88c 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/AdaptiveReorderPartitionedJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/AdaptiveReorderPartitionedJoin.java @@ -89,7 +89,7 @@ public class AdaptiveReorderPartitionedJoin { private static final Capture LOCAL_EXCHANGE_NODE = Capture.newCapture(); private static final Pattern PATTERN = join() - .matching(AdaptiveReorderPartitionedJoin::isPartitionedJoinWithNoHashSymbols) + .matching(AdaptiveReorderPartitionedJoin::isPartitionedJoin) .or( // In case partial aggregation is missing prev -> prev.with(right().matching( @@ -113,14 +113,10 @@ public AdaptiveReorderPartitionedJoin(Metadata metadata) this.metadata = requireNonNull(metadata, "metadata is null"); } - private static boolean isPartitionedJoinWithNoHashSymbols(JoinNode joinNode) + private static boolean isPartitionedJoin(JoinNode joinNode) { // Check if the join is partitioned and does not have hash symbols - return joinNode.getDistributionType().equals(Optional.of(PARTITIONED)) - // TODO: Add support for hash symbols. For now, it's not important since hash optimization - // is disabled by default - && joinNode.getRightHashSymbol().isEmpty() - && joinNode.getLeftHashSymbol().isEmpty(); + return joinNode.getDistributionType().equals(Optional.of(PARTITIONED)); } @Override @@ -206,8 +202,6 @@ private static JoinNode flipJoinAndFixLocalExchanges( flippedJoinNode.getRightOutputSymbols(), flippedJoinNode.isMaySkipOutputDuplicates(), flippedJoinNode.getFilter(), - flippedJoinNode.getLeftHashSymbol(), - flippedJoinNode.getRightHashSymbol(), flippedJoinNode.getDistributionType(), flippedJoinNode.isSpillable(), flippedJoinNode.getDynamicFilters(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/EliminateCrossJoins.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/EliminateCrossJoins.java index 1f31de091b6c..87b5adb78661 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/EliminateCrossJoins.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/EliminateCrossJoins.java @@ -183,8 +183,6 @@ public static PlanNode buildJoinTree(List expectedOutputSymbols, JoinGra Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExpressionRewriteRuleSet.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExpressionRewriteRuleSet.java index 3367d8a38385..9659f5c09a69 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExpressionRewriteRuleSet.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExpressionRewriteRuleSet.java @@ -258,8 +258,6 @@ public Result apply(JoinNode joinNode, Captures captures, Context context) joinNode.getRightOutputSymbols(), joinNode.isMaySkipOutputDuplicates(), filter, - joinNode.getLeftHashSymbol(), - joinNode.getRightHashSymbol(), joinNode.getDistributionType(), joinNode.isSpillable(), joinNode.getDynamicFilters(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExtractSpatialJoins.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExtractSpatialJoins.java index 47edb054a474..d6c5ebdfb5fd 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExtractSpatialJoins.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExtractSpatialJoins.java @@ -338,8 +338,6 @@ private static Result tryCreateSpatialJoin( joinNode.getRightOutputSymbols(), joinNode.isMaySkipOutputDuplicates(), Optional.of(newFilter), - joinNode.getLeftHashSymbol(), - joinNode.getRightHashSymbol(), joinNode.getDistributionType(), joinNode.isSpillable(), joinNode.getDynamicFilters(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ImplementTableFunctionSource.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ImplementTableFunctionSource.java index f874cb4e0a8d..418825f20251 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ImplementTableFunctionSource.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ImplementTableFunctionSource.java @@ -467,8 +467,6 @@ else if (right.pruneWhenEmpty()) { Optional.of(joinCondition), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()), left.rowNumber(), @@ -602,8 +600,6 @@ else if (right.pruneWhenEmpty()) { Optional.of(joinCondition), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()), left.rowNumber(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/MultipleDistinctAggregationsToSubqueries.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/MultipleDistinctAggregationsToSubqueries.java index 4e2ae1ed7f23..ba0617b79d2e 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/MultipleDistinctAggregationsToSubqueries.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/MultipleDistinctAggregationsToSubqueries.java @@ -200,8 +200,6 @@ private JoinNode buildJoin(PlanNode left, List leftJoinSymbols, PlanNode Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneIndexJoinColumns.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneIndexJoinColumns.java index 828ad226ed72..b236a436f377 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneIndexJoinColumns.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneIndexJoinColumns.java @@ -41,14 +41,12 @@ protected Optional pushDownProjectOff(Context context, IndexJoinNode i .addAll(indexJoinNode.getCriteria().stream() .map(IndexJoinNode.EquiJoinClause::getProbe) .collect(toImmutableList())); - indexJoinNode.getProbeHashSymbol().ifPresent(probeInputs::add); ImmutableSet.Builder indexInputs = ImmutableSet.builder() .addAll(referencedOutputs) .addAll(indexJoinNode.getCriteria().stream() .map(IndexJoinNode.EquiJoinClause::getIndex) .collect(toImmutableList())); - indexJoinNode.getIndexHashSymbol().ifPresent(indexInputs::add); return restrictChildOutputs(context.getIdAllocator(), indexJoinNode, probeInputs.build(), indexInputs.build()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneJoinChildrenColumns.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneJoinChildrenColumns.java index 93a5e31f6463..5419bcdfb91c 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneJoinChildrenColumns.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneJoinChildrenColumns.java @@ -57,7 +57,6 @@ public Result apply(JoinNode joinNode, Captures captures, Context context) joinNode.getCriteria().stream() .map(JoinNode.EquiJoinClause::getLeft) .iterator()) - .addAll(joinNode.getLeftHashSymbol().map(ImmutableSet::of).orElse(ImmutableSet.of())) .build(); Set rightUsableInputs = ImmutableSet.builder() @@ -66,7 +65,6 @@ public Result apply(JoinNode joinNode, Captures captures, Context context) joinNode.getCriteria().stream() .map(JoinNode.EquiJoinClause::getRight) .iterator()) - .addAll(joinNode.getRightHashSymbol().map(ImmutableSet::of).orElse(ImmutableSet.of())) .build(); return restrictChildOutputs(context.getIdAllocator(), joinNode, leftUsableInputs, rightUsableInputs) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneJoinColumns.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneJoinColumns.java index c1e56a7a4f6d..5c5cefd29aa0 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneJoinColumns.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneJoinColumns.java @@ -48,8 +48,6 @@ protected Optional pushDownProjectOff(Context context, JoinNode joinNo filteredCopy(joinNode.getRightOutputSymbols(), referencedOutputs::contains), joinNode.isMaySkipOutputDuplicates(), joinNode.getFilter(), - joinNode.getLeftHashSymbol(), - joinNode.getRightHashSymbol(), joinNode.getDistributionType(), joinNode.isSpillable(), joinNode.getDynamicFilters(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneSemiJoinColumns.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneSemiJoinColumns.java index 44dd7b45ca9a..bc710858e835 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneSemiJoinColumns.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneSemiJoinColumns.java @@ -45,8 +45,7 @@ protected Optional pushDownProjectOff(Context context, SemiJoinNode se Set requiredSourceInputs = Streams.concat( referencedOutputs.stream() .filter(symbol -> !symbol.equals(semiJoinNode.getSemiJoinOutput())), - Stream.of(semiJoinNode.getSourceJoinSymbol()), - semiJoinNode.getSourceHashSymbol().stream()) + Stream.of(semiJoinNode.getSourceJoinSymbol())) .collect(toImmutableSet()); return restrictOutputs(context.getIdAllocator(), semiJoinNode.getSource(), requiredSourceInputs) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneSemiJoinFilteringSourceColumns.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneSemiJoinFilteringSourceColumns.java index f3a279a89d9d..a76055b54da0 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneSemiJoinFilteringSourceColumns.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PruneSemiJoinFilteringSourceColumns.java @@ -14,7 +14,7 @@ package io.trino.sql.planner.iterative.rule; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Streams; +import com.google.common.collect.ImmutableSet; import io.trino.matching.Captures; import io.trino.matching.Pattern; import io.trino.sql.planner.Symbol; @@ -22,9 +22,7 @@ import io.trino.sql.planner.plan.SemiJoinNode; import java.util.Set; -import java.util.stream.Stream; -import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.trino.sql.planner.iterative.rule.Util.restrictOutputs; import static io.trino.sql.planner.plan.Patterns.semiJoin; @@ -42,10 +40,7 @@ public Pattern getPattern() @Override public Result apply(SemiJoinNode semiJoinNode, Captures captures, Context context) { - Set requiredFilteringSourceInputs = Streams.concat( - Stream.of(semiJoinNode.getFilteringSourceJoinSymbol()), - semiJoinNode.getFilteringSourceHashSymbol().stream()) - .collect(toImmutableSet()); + Set requiredFilteringSourceInputs = ImmutableSet.of(semiJoinNode.getFilteringSourceJoinSymbol()); return restrictOutputs(context.getIdAllocator(), semiJoinNode.getFilteringSource(), requiredFilteringSourceInputs) .map(newFilteringSource -> diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushAggregationThroughOuterJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushAggregationThroughOuterJoin.java index b8bd9f75d301..2c282819c3e6 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushAggregationThroughOuterJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushAggregationThroughOuterJoin.java @@ -156,8 +156,6 @@ public Result apply(AggregationNode aggregation, Captures captures, Context cont // there are no duplicate rows possible since outer rows were guaranteed to be distinct false, join.getFilter(), - join.getLeftHashSymbol(), - join.getRightHashSymbol(), join.getDistributionType(), join.isSpillable(), join.getDynamicFilters(), @@ -175,8 +173,6 @@ public Result apply(AggregationNode aggregation, Captures captures, Context cont // there are no duplicate rows possible since outer rows were guaranteed to be distinct false, join.getFilter(), - join.getLeftHashSymbol(), - join.getRightHashSymbol(), join.getDistributionType(), join.isSpillable(), join.getDynamicFilters(), @@ -251,8 +247,6 @@ private Optional coalesceWithNullAggregation(AggregationNode aggregati Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushDownDereferenceThroughJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushDownDereferenceThroughJoin.java index 85ca829ebeb9..a9818e30707a 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushDownDereferenceThroughJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushDownDereferenceThroughJoin.java @@ -168,8 +168,6 @@ else if (joinNode.getRight().getOutputSymbols().contains(baseSymbol)) { joinNode.isMaySkipOutputDuplicates(), // Use newly created symbols in filter joinNode.getFilter().map(expression -> replaceExpression(expression, mappings)), - joinNode.getLeftHashSymbol(), - joinNode.getRightHashSymbol(), joinNode.getDistributionType(), joinNode.isSpillable(), joinNode.getDynamicFilters(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushInequalityFilterExpressionBelowJoinRuleSet.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushInequalityFilterExpressionBelowJoinRuleSet.java index b4d32efdf920..7761531150eb 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushInequalityFilterExpressionBelowJoinRuleSet.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushInequalityFilterExpressionBelowJoinRuleSet.java @@ -255,8 +255,6 @@ private JoinNode constructModifiedJoin( concatToList(originalJoinNode.getRightOutputSymbols(), newJoinRightOutputSymbols), originalJoinNode.isMaySkipOutputDuplicates(), newJoinFilter, - originalJoinNode.getLeftHashSymbol(), - originalJoinNode.getRightHashSymbol(), originalJoinNode.getDistributionType(), originalJoinNode.isSpillable(), originalJoinNode.getDynamicFilters(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughJoin.java index 2962fea0b82c..9b9561c522ab 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughJoin.java @@ -254,9 +254,7 @@ private Set getJoinRequiredSymbols(JoinNode node) return Streams.concat( node.getCriteria().stream().map(JoinNode.EquiJoinClause::getLeft), node.getCriteria().stream().map(JoinNode.EquiJoinClause::getRight), - node.getFilter().map(SymbolsExtractor::extractUnique).orElse(ImmutableSet.of()).stream(), - node.getLeftHashSymbol().map(ImmutableSet::of).orElse(ImmutableSet.of()).stream(), - node.getRightHashSymbol().map(ImmutableSet::of).orElse(ImmutableSet.of()).stream()) + node.getFilter().map(SymbolsExtractor::extractUnique).orElse(ImmutableSet.of()).stream()) .collect(toImmutableSet()); } @@ -310,8 +308,6 @@ private PlanNode replaceJoin( rightChild.getOutputSymbols(), child.isMaySkipOutputDuplicates(), child.getFilter(), - child.getLeftHashSymbol(), - child.getRightHashSymbol(), child.getDistributionType(), child.isSpillable(), child.getDynamicFilters(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushProjectionThroughJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushProjectionThroughJoin.java index 0df6f82f6b6b..661dec12e149 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushProjectionThroughJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushProjectionThroughJoin.java @@ -118,8 +118,6 @@ else if (rightChild.getOutputSymbols().containsAll(symbols)) { rightOutputSymbols, joinNode.isMaySkipOutputDuplicates(), joinNode.getFilter(), - joinNode.getLeftHashSymbol(), - joinNode.getRightHashSymbol(), joinNode.getDistributionType(), joinNode.isSpillable(), joinNode.getDynamicFilters(), @@ -146,9 +144,7 @@ private static Set getJoinRequiredSymbols(JoinNode node) return Streams.concat( node.getCriteria().stream().map(JoinNode.EquiJoinClause::getLeft), node.getCriteria().stream().map(JoinNode.EquiJoinClause::getRight), - node.getFilter().map(SymbolsExtractor::extractUnique).orElse(ImmutableSet.of()).stream(), - node.getLeftHashSymbol().map(ImmutableSet::of).orElse(ImmutableSet.of()).stream(), - node.getRightHashSymbol().map(ImmutableSet::of).orElse(ImmutableSet.of()).stream()) + node.getFilter().map(SymbolsExtractor::extractUnique).orElse(ImmutableSet.of()).stream()) .collect(toImmutableSet()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/RemoveUnsupportedDynamicFilters.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/RemoveUnsupportedDynamicFilters.java index f9f9b041703a..3ceda8084325 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/RemoveUnsupportedDynamicFilters.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/RemoveUnsupportedDynamicFilters.java @@ -156,8 +156,6 @@ public PlanWithConsumedDynamicFilters visitJoin(JoinNode node, Set leftSources, Li joinFilters.isEmpty() ? Optional.empty() : Optional.of(and(joinFilters)), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty())); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithProjection.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithProjection.java index fd9698d1eb2f..55c631c72bff 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithProjection.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithProjection.java @@ -133,8 +133,6 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co decorrelatedSource.get().getCorrelatedPredicates(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithoutProjection.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithoutProjection.java index 10c78e323a6f..d3aa9161e934 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithoutProjection.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedDistinctAggregationWithoutProjection.java @@ -124,8 +124,6 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co decorrelatedSource.get().getCorrelatedPredicates(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java index 70613deb860a..1c4355200c2e 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithProjection.java @@ -215,8 +215,6 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co decorrelatedSource.get().getCorrelatedPredicates(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java index 959f8a8983a7..7c09f8c76336 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGlobalAggregationWithoutProjection.java @@ -196,8 +196,6 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co decorrelatedSource.get().getCorrelatedPredicates(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithProjection.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithProjection.java index 54f465ddc11b..d61515eddde1 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithProjection.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithProjection.java @@ -183,8 +183,6 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co decorrelatedSource.get().getCorrelatedPredicates(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithoutProjection.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithoutProjection.java index 15aff01e8852..21b92b355f84 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithoutProjection.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedGroupedAggregationWithoutProjection.java @@ -174,8 +174,6 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co decorrelatedSource.get().getCorrelatedPredicates(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java index a2e7cf6c0fa9..1de8015e3a0d 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedInPredicateToJoin.java @@ -244,8 +244,6 @@ private static JoinNode leftOuterJoin(PlanNodeIdAllocator idAllocator, AssignUni Optional.of(joinExpression), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedJoinToJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedJoinToJoin.java index 9da0239a5233..1c8ef610b8a9 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedJoinToJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformCorrelatedJoinToJoin.java @@ -90,8 +90,6 @@ public Result apply(CorrelatedJoinNode correlatedJoinNode, Captures captures, Co filter.equals(TRUE) ? Optional.empty() : Optional.of(filter), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty())); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformFilteringSemiJoinToInnerJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformFilteringSemiJoinToInnerJoin.java index be2191b10258..1fda5541967f 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformFilteringSemiJoinToInnerJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformFilteringSemiJoinToInnerJoin.java @@ -141,8 +141,6 @@ public Result apply(FilterNode filterNode, Captures captures, Context context) joinFilter, Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), semiJoin.getDynamicFilterId() .map(id -> ImmutableMap.of(id, semiJoin.getFilteringSourceJoinSymbol())) .orElse(ImmutableMap.of()), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformUncorrelatedInPredicateSubqueryToSemiJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformUncorrelatedInPredicateSubqueryToSemiJoin.java index 481d00968517..97f518537f07 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformUncorrelatedInPredicateSubqueryToSemiJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformUncorrelatedInPredicateSubqueryToSemiJoin.java @@ -83,8 +83,6 @@ public Result apply(ApplyNode applyNode, Captures captures, Context context) inPredicate.reference(), semiJoinSymbol, Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.empty()); return Result.ofPlanNode(replacement); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformUncorrelatedSubqueryToJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformUncorrelatedSubqueryToJoin.java index ff8c1f84a9a0..a324ab93cebb 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformUncorrelatedSubqueryToJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/TransformUncorrelatedSubqueryToJoin.java @@ -129,8 +129,6 @@ private JoinNode rewriteToJoin(CorrelatedJoinNode parent, JoinType type, Express filter.equals(TRUE) ? Optional.empty() : Optional.of(filter), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java index 20e882c00f5c..c49b59c25f4f 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddExchanges.java @@ -1060,8 +1060,6 @@ private PlanWithProperties buildJoin(JoinNode node, PlanWithProperties newLeft, node.getRightOutputSymbols(), node.isMaySkipOutputDuplicates(), node.getFilter(), - node.getLeftHashSymbol(), - node.getRightHashSymbol(), Optional.of(newDistributionType), node.isSpillable(), node.getDynamicFilters(), diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/IndexJoinOptimizer.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/IndexJoinOptimizer.java index 1a1de843c9ad..3a28ab728da4 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/IndexJoinOptimizer.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/IndexJoinOptimizer.java @@ -138,10 +138,10 @@ public PlanNode visitJoin(JoinNode node, RewriteContext context) // Prefer the right candidate over the left candidate PlanNode indexJoinNode = null; if (rightIndexCandidate.isPresent()) { - indexJoinNode = new IndexJoinNode(idAllocator.getNextId(), IndexJoinNode.Type.INNER, leftRewritten, rightIndexCandidate.get(), createEquiJoinClause(leftJoinSymbols, rightJoinSymbols), Optional.empty(), Optional.empty()); + indexJoinNode = new IndexJoinNode(idAllocator.getNextId(), IndexJoinNode.Type.INNER, leftRewritten, rightIndexCandidate.get(), createEquiJoinClause(leftJoinSymbols, rightJoinSymbols)); } else if (leftIndexCandidate.isPresent()) { - indexJoinNode = new IndexJoinNode(idAllocator.getNextId(), IndexJoinNode.Type.INNER, rightRewritten, leftIndexCandidate.get(), createEquiJoinClause(rightJoinSymbols, leftJoinSymbols), Optional.empty(), Optional.empty()); + indexJoinNode = new IndexJoinNode(idAllocator.getNextId(), IndexJoinNode.Type.INNER, rightRewritten, leftIndexCandidate.get(), createEquiJoinClause(rightJoinSymbols, leftJoinSymbols)); } if (indexJoinNode != null) { @@ -193,8 +193,6 @@ else if (leftIndexCandidate.isPresent()) { node.getRightOutputSymbols(), node.isMaySkipOutputDuplicates(), node.getFilter(), - node.getLeftHashSymbol(), - node.getRightHashSymbol(), node.getDistributionType(), node.isSpillable(), node.getDynamicFilters(), @@ -205,7 +203,7 @@ else if (leftIndexCandidate.isPresent()) { private static PlanNode createIndexJoinWithExpectedOutputs(List expectedOutputs, IndexJoinNode.Type type, PlanNode probe, PlanNode index, List equiJoinClause, PlanNodeIdAllocator idAllocator) { - PlanNode result = new IndexJoinNode(idAllocator.getNextId(), type, probe, index, equiJoinClause, Optional.empty(), Optional.empty()); + PlanNode result = new IndexJoinNode(idAllocator.getNextId(), type, probe, index, equiJoinClause); if (!result.getOutputSymbols().equals(expectedOutputs)) { result = new ProjectNode( idAllocator.getNextId(), @@ -401,7 +399,7 @@ public PlanNode visitIndexJoin(IndexJoinNode node, RewriteContext conte PlanNode source = node; if (rewrittenProbeSource != node.getProbeSource()) { - source = new IndexJoinNode(node.getId(), node.getType(), rewrittenProbeSource, node.getIndexSource(), node.getCriteria(), node.getProbeHashSymbol(), node.getIndexHashSymbol()); + source = new IndexJoinNode(node.getId(), node.getType(), rewrittenProbeSource, node.getIndexSource(), node.getCriteria()); } return source; diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/LimitPushDown.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/LimitPushDown.java index a12b8b699c38..cee02a466ab2 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/LimitPushDown.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/LimitPushDown.java @@ -227,8 +227,6 @@ public PlanNode visitSemiJoin(SemiJoinNode node, RewriteContext co node.getSourceJoinSymbol(), node.getFilteringSourceJoinSymbol(), node.getSemiJoinOutput(), - node.getSourceHashSymbol(), - node.getFilteringSourceHashSymbol(), node.getDistributionType(), node.getDynamicFilterId()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PredicatePushDown.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PredicatePushDown.java index 198c57197f4d..f66dc4d0fc83 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PredicatePushDown.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/PredicatePushDown.java @@ -575,8 +575,6 @@ public PlanNode visitJoin(JoinNode node, RewriteContext context) rightSource.getOutputSymbols(), node.isMaySkipOutputDuplicates(), newJoinFilter, - node.getLeftHashSymbol(), - node.getRightHashSymbol(), node.getDistributionType(), node.isSpillable(), dynamicFilters, @@ -1180,8 +1178,6 @@ private JoinNode tryNormalizeToOuterToInnerJoin(JoinNode node, Expression inheri node.getRightOutputSymbols(), node.isMaySkipOutputDuplicates(), node.getFilter(), - node.getLeftHashSymbol(), - node.getRightHashSymbol(), node.getDistributionType(), node.isSpillable(), node.getDynamicFilters(), @@ -1197,8 +1193,6 @@ private JoinNode tryNormalizeToOuterToInnerJoin(JoinNode node, Expression inheri node.getRightOutputSymbols(), node.isMaySkipOutputDuplicates(), node.getFilter(), - node.getLeftHashSymbol(), - node.getRightHashSymbol(), node.getDistributionType(), node.isSpillable(), node.getDynamicFilters(), @@ -1219,8 +1213,6 @@ private JoinNode tryNormalizeToOuterToInnerJoin(JoinNode node, Expression inheri node.getRightOutputSymbols(), node.isMaySkipOutputDuplicates(), node.getFilter(), - node.getLeftHashSymbol(), - node.getRightHashSymbol(), node.getDistributionType(), node.isSpillable(), node.getDynamicFilters(), @@ -1362,8 +1354,6 @@ private PlanNode visitNonFilteringSemiJoin(SemiJoinNode node, RewriteContext newCriteria = builder.build(); Optional newFilter = node.getFilter().map(mapper::map); - Optional newLeftHashSymbol = node.getLeftHashSymbol().map(mapper::map); - Optional newRightHashSymbol = node.getRightHashSymbol().map(mapper::map); // rewrite dynamic filters Map canonicalDynamicFilters = new HashMap<>(); @@ -1140,8 +1138,6 @@ public PlanAndMappings visitJoin(JoinNode node, UnaliasContext context) newRightOutputSymbols, node.isMaySkipOutputDuplicates(), newFilter, - newLeftHashSymbol, - newRightHashSymbol, node.getDistributionType(), node.isSpillable(), newDynamicFilters, @@ -1165,8 +1161,6 @@ public PlanAndMappings visitSemiJoin(SemiJoinNode node, UnaliasContext context) Symbol newSourceJoinSymbol = mapper.map(node.getSourceJoinSymbol()); Symbol newFilteringSourceJoinSymbol = mapper.map(node.getFilteringSourceJoinSymbol()); Symbol newSemiJoinOutput = mapper.map(node.getSemiJoinOutput()); - Optional newSourceHashSymbol = node.getSourceHashSymbol().map(mapper::map); - Optional newFilteringSourceHashSymbol = node.getFilteringSourceHashSymbol().map(mapper::map); return new PlanAndMappings( new SemiJoinNode( @@ -1176,8 +1170,6 @@ public PlanAndMappings visitSemiJoin(SemiJoinNode node, UnaliasContext context) newSourceJoinSymbol, newFilteringSourceJoinSymbol, newSemiJoinOutput, - newSourceHashSymbol, - newFilteringSourceHashSymbol, node.getDistributionType(), node.getDynamicFilterId()), outputMapping); @@ -1226,11 +1218,8 @@ public PlanAndMappings visitIndexJoin(IndexJoinNode node, UnaliasContext context } List newEquiCriteria = builder.build(); - Optional newProbeHashSymbol = node.getProbeHashSymbol().map(mapper::map); - Optional newIndexHashSymbol = node.getIndexHashSymbol().map(mapper::map); - return new PlanAndMappings( - new IndexJoinNode(node.getId(), node.getType(), rewrittenProbe.getRoot(), rewrittenIndex.getRoot(), newEquiCriteria, newProbeHashSymbol, newIndexHashSymbol), + new IndexJoinNode(node.getId(), node.getType(), rewrittenProbe.getRoot(), rewrittenIndex.getRoot(), newEquiCriteria), outputMapping); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/plan/IndexJoinNode.java b/core/trino-main/src/main/java/io/trino/sql/planner/plan/IndexJoinNode.java index 100f9eae6d96..9b5f3b388e76 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/plan/IndexJoinNode.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/plan/IndexJoinNode.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Objects; -import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; import static java.lang.String.format; @@ -35,8 +34,6 @@ public class IndexJoinNode private final PlanNode probeSource; private final PlanNode indexSource; private final List criteria; - private final Optional probeHashSymbol; - private final Optional indexHashSymbol; @JsonCreator public IndexJoinNode( @@ -44,17 +41,13 @@ public IndexJoinNode( @JsonProperty("type") Type type, @JsonProperty("probeSource") PlanNode probeSource, @JsonProperty("indexSource") PlanNode indexSource, - @JsonProperty("criteria") List criteria, - @JsonProperty("probeHashSymbol") Optional probeHashSymbol, - @JsonProperty("indexHashSymbol") Optional indexHashSymbol) + @JsonProperty("criteria") List criteria) { super(id); this.type = requireNonNull(type, "type is null"); this.probeSource = requireNonNull(probeSource, "probeSource is null"); this.indexSource = requireNonNull(indexSource, "indexSource is null"); this.criteria = ImmutableList.copyOf(requireNonNull(criteria, "criteria is null")); - this.probeHashSymbol = requireNonNull(probeHashSymbol, "probeHashSymbol is null"); - this.indexHashSymbol = requireNonNull(indexHashSymbol, "indexHashSymbol is null"); } public enum Type @@ -99,18 +92,6 @@ public List getCriteria() return criteria; } - @JsonProperty("probeHashSymbol") - public Optional getProbeHashSymbol() - { - return probeHashSymbol; - } - - @JsonProperty("indexHashSymbol") - public Optional getIndexHashSymbol() - { - return indexHashSymbol; - } - @Override public List getSources() { @@ -136,7 +117,7 @@ public R accept(PlanVisitor visitor, C context) public PlanNode replaceChildren(List newChildren) { checkArgument(newChildren.size() == 2, "expected newChildren to contain 2 nodes"); - return new IndexJoinNode(getId(), type, newChildren.get(0), newChildren.get(1), criteria, probeHashSymbol, indexHashSymbol); + return new IndexJoinNode(getId(), type, newChildren.get(0), newChildren.get(1), criteria); } public static class EquiJoinClause diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/plan/JoinNode.java b/core/trino-main/src/main/java/io/trino/sql/planner/plan/JoinNode.java index e66a9b9104f8..49b383198a03 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/plan/JoinNode.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/plan/JoinNode.java @@ -60,8 +60,6 @@ public enum DistributionType private final List rightOutputSymbols; private final boolean maySkipOutputDuplicates; private final Optional filter; - private final Optional leftHashSymbol; - private final Optional rightHashSymbol; private final Optional distributionType; private final Optional spillable; private final Map dynamicFilters; @@ -80,8 +78,6 @@ public JoinNode( @JsonProperty("rightOutputSymbols") List rightOutputSymbols, @JsonProperty("maySkipOutputDuplicates") boolean maySkipOutputDuplicates, @JsonProperty("filter") Optional filter, - @JsonProperty("leftHashSymbol") Optional leftHashSymbol, - @JsonProperty("rightHashSymbol") Optional rightHashSymbol, @JsonProperty("distributionType") Optional distributionType, @JsonProperty("spillable") Optional spillable, @JsonProperty("dynamicFilters") Map dynamicFilters, @@ -95,8 +91,6 @@ public JoinNode( requireNonNull(leftOutputSymbols, "leftOutputSymbols is null"); requireNonNull(rightOutputSymbols, "rightOutputSymbols is null"); requireNonNull(filter, "filter is null"); - requireNonNull(leftHashSymbol, "leftHashSymbol is null"); - requireNonNull(rightHashSymbol, "rightHashSymbol is null"); requireNonNull(distributionType, "distributionType is null"); requireNonNull(spillable, "spillable is null"); @@ -108,8 +102,6 @@ public JoinNode( this.rightOutputSymbols = ImmutableList.copyOf(rightOutputSymbols); this.maySkipOutputDuplicates = maySkipOutputDuplicates; this.filter = filter; - this.leftHashSymbol = leftHashSymbol; - this.rightHashSymbol = rightHashSymbol; this.distributionType = distributionType; this.spillable = spillable; this.dynamicFilters = ImmutableMap.copyOf(requireNonNull(dynamicFilters, "dynamicFilters is null")); @@ -130,9 +122,6 @@ public JoinNode( rightSymbols); }); - checkArgument(!(criteria.isEmpty() && leftHashSymbol.isPresent()), "Left hash symbol is only valid in an equijoin"); - checkArgument(!(criteria.isEmpty() && rightHashSymbol.isPresent()), "Right hash symbol is only valid in an equijoin"); - criteria.forEach(equiJoinClause -> checkArgument( leftSymbols.contains(equiJoinClause.getLeft()) && @@ -165,8 +154,6 @@ public JoinNode flipChildren() leftOutputSymbols, maySkipOutputDuplicates, filter, - rightHashSymbol, - leftHashSymbol, distributionType, spillable, ImmutableMap.of(), // dynamicFilters are invalid after flipping children @@ -232,18 +219,6 @@ public Optional getFilter() return filter; } - @JsonProperty("leftHashSymbol") - public Optional getLeftHashSymbol() - { - return leftHashSymbol; - } - - @JsonProperty("rightHashSymbol") - public Optional getRightHashSymbol() - { - return rightHashSymbol; - } - @Override public List getSources() { @@ -299,32 +274,32 @@ public R accept(PlanVisitor visitor, C context) public PlanNode replaceChildren(List newChildren) { checkArgument(newChildren.size() == 2, "expected newChildren to contain 2 nodes"); - return new JoinNode(getId(), type, newChildren.get(0), newChildren.get(1), criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, leftHashSymbol, rightHashSymbol, distributionType, spillable, dynamicFilters, reorderJoinStatsAndCost); + return new JoinNode(getId(), type, newChildren.get(0), newChildren.get(1), criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, distributionType, spillable, dynamicFilters, reorderJoinStatsAndCost); } public JoinNode withDistributionType(DistributionType distributionType) { - return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, leftHashSymbol, rightHashSymbol, Optional.of(distributionType), spillable, dynamicFilters, reorderJoinStatsAndCost); + return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, Optional.of(distributionType), spillable, dynamicFilters, reorderJoinStatsAndCost); } public JoinNode withSpillable(boolean spillable) { - return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, leftHashSymbol, rightHashSymbol, distributionType, Optional.of(spillable), dynamicFilters, reorderJoinStatsAndCost); + return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, distributionType, Optional.of(spillable), dynamicFilters, reorderJoinStatsAndCost); } public JoinNode withMaySkipOutputDuplicates() { - return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, true, filter, leftHashSymbol, rightHashSymbol, distributionType, spillable, dynamicFilters, reorderJoinStatsAndCost); + return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, true, filter, distributionType, spillable, dynamicFilters, reorderJoinStatsAndCost); } public JoinNode withReorderJoinStatsAndCost(PlanNodeStatsAndCostSummary statsAndCost) { - return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, leftHashSymbol, rightHashSymbol, distributionType, spillable, dynamicFilters, Optional.of(statsAndCost)); + return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, distributionType, spillable, dynamicFilters, Optional.of(statsAndCost)); } public JoinNode withoutDynamicFilters() { - return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, leftHashSymbol, rightHashSymbol, distributionType, spillable, ImmutableMap.of(), reorderJoinStatsAndCost); + return new JoinNode(getId(), type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, maySkipOutputDuplicates, filter, distributionType, spillable, ImmutableMap.of(), reorderJoinStatsAndCost); } public boolean isCrossJoin() diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/plan/SemiJoinNode.java b/core/trino-main/src/main/java/io/trino/sql/planner/plan/SemiJoinNode.java index 45f2fe33a925..53f2e81bdf05 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/plan/SemiJoinNode.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/plan/SemiJoinNode.java @@ -34,8 +34,6 @@ public class SemiJoinNode private final Symbol sourceJoinSymbol; private final Symbol filteringSourceJoinSymbol; private final Symbol semiJoinOutput; - private final Optional sourceHashSymbol; - private final Optional filteringSourceHashSymbol; private final Optional distributionType; private final Optional dynamicFilterId; @@ -46,8 +44,6 @@ public SemiJoinNode(@JsonProperty("id") PlanNodeId id, @JsonProperty("sourceJoinSymbol") Symbol sourceJoinSymbol, @JsonProperty("filteringSourceJoinSymbol") Symbol filteringSourceJoinSymbol, @JsonProperty("semiJoinOutput") Symbol semiJoinOutput, - @JsonProperty("sourceHashSymbol") Optional sourceHashSymbol, - @JsonProperty("filteringSourceHashSymbol") Optional filteringSourceHashSymbol, @JsonProperty("distributionType") Optional distributionType, @JsonProperty("dynamicFilterId") Optional dynamicFilterId) { @@ -57,8 +53,6 @@ public SemiJoinNode(@JsonProperty("id") PlanNodeId id, this.sourceJoinSymbol = requireNonNull(sourceJoinSymbol, "sourceJoinSymbol is null"); this.filteringSourceJoinSymbol = requireNonNull(filteringSourceJoinSymbol, "filteringSourceJoinSymbol is null"); this.semiJoinOutput = requireNonNull(semiJoinOutput, "semiJoinOutput is null"); - this.sourceHashSymbol = requireNonNull(sourceHashSymbol, "sourceHashSymbol is null"); - this.filteringSourceHashSymbol = requireNonNull(filteringSourceHashSymbol, "filteringSourceHashSymbol is null"); this.distributionType = requireNonNull(distributionType, "distributionType is null"); this.dynamicFilterId = requireNonNull(dynamicFilterId, "dynamicFilterId is null"); @@ -102,18 +96,6 @@ public Symbol getSemiJoinOutput() return semiJoinOutput; } - @JsonProperty("sourceHashSymbol") - public Optional getSourceHashSymbol() - { - return sourceHashSymbol; - } - - @JsonProperty("filteringSourceHashSymbol") - public Optional getFilteringSourceHashSymbol() - { - return filteringSourceHashSymbol; - } - @JsonProperty("distributionType") public Optional getDistributionType() { @@ -158,8 +140,6 @@ public PlanNode replaceChildren(List newChildren) sourceJoinSymbol, filteringSourceJoinSymbol, semiJoinOutput, - sourceHashSymbol, - filteringSourceHashSymbol, distributionType, dynamicFilterId); } @@ -173,8 +153,6 @@ public SemiJoinNode withDistributionType(DistributionType distributionType) sourceJoinSymbol, filteringSourceJoinSymbol, semiJoinOutput, - sourceHashSymbol, - filteringSourceHashSymbol, Optional.of(distributionType), dynamicFilterId); } @@ -188,8 +166,6 @@ public SemiJoinNode withoutDynamicFilter() sourceJoinSymbol, filteringSourceJoinSymbol, semiJoinOutput, - sourceHashSymbol, - filteringSourceHashSymbol, distributionType, Optional.empty()); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/PlanPrinter.java b/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/PlanPrinter.java index c97d31d8d98e..b916bf5d9fee 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/PlanPrinter.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/PlanPrinter.java @@ -694,7 +694,6 @@ public Void visitJoin(JoinNode node, Context context) ImmutableMap.Builder descriptor = ImmutableMap.builder() .put("criteria", Joiner.on(" AND ").join(anonymizeExpressions(criteriaExpressions))); node.getFilter().ifPresent(filter -> descriptor.put("filter", formatFilter(filter))); - descriptor.put("hash", formatHash(node.getLeftHashSymbol(), node.getRightHashSymbol())); node.getDistributionType().ifPresent(distribution -> descriptor.put("distribution", distribution.name())); nodeOutput = addNode( node, @@ -738,8 +737,7 @@ public Void visitSemiJoin(SemiJoinNode node, Context context) NodeRepresentation nodeOutput = addNode(node, "SemiJoin", ImmutableMap.of( - "criteria", anonymizer.anonymize(node.getSourceJoinSymbol()) + " = " + anonymizer.anonymize(node.getFilteringSourceJoinSymbol()), - "hash", formatHash(node.getSourceHashSymbol(), node.getFilteringSourceHashSymbol())), + "criteria", anonymizer.anonymize(node.getSourceJoinSymbol()) + " = " + anonymizer.anonymize(node.getFilteringSourceJoinSymbol())), context); node.getDistributionType().ifPresent(distributionType -> nodeOutput.appendDetails("Distribution: %s", distributionType)); node.getDynamicFilterId().ifPresent(dynamicFilterId -> nodeOutput.appendDetails("dynamicFilterId: %s", dynamicFilterId)); @@ -792,8 +790,7 @@ public Void visitIndexJoin(IndexJoinNode node, Context context) addNode(node, format("%sIndexJoin", node.getType().getJoinLabel()), ImmutableMap.of( - "criteria", Joiner.on(" AND ").join(anonymizeExpressions(joinExpressions)), - "hash", formatHash(node.getProbeHashSymbol(), node.getIndexHashSymbol())), + "criteria", Joiner.on(" AND ").join(anonymizeExpressions(joinExpressions))), context); node.getProbeSource().accept(this, new Context(context.isInitialPlan())); node.getIndexSource().accept(this, new Context(context.isInitialPlan())); diff --git a/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java b/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java index 237e26c16ae7..b71550a8a0c9 100644 --- a/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java +++ b/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java @@ -804,8 +804,6 @@ private JoinNode join(String planNodeId, PlanNode left, PlanNode right, JoinNode right.getOutputSymbols(), false, Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(distributionType), Optional.empty(), ImmutableMap.of(), diff --git a/core/trino-main/src/test/java/io/trino/cost/TestSemiJoinStatsRule.java b/core/trino-main/src/test/java/io/trino/cost/TestSemiJoinStatsRule.java index ace97c129a0c..a205fc803182 100644 --- a/core/trino-main/src/test/java/io/trino/cost/TestSemiJoinStatsRule.java +++ b/core/trino-main/src/test/java/io/trino/cost/TestSemiJoinStatsRule.java @@ -45,8 +45,6 @@ public void testSemiJoinPropagatesSourceStats() a, c, semiJoinOutput, - Optional.empty(), - Optional.empty(), Optional.empty()); }) .withSourceStats(0, PlanNodeStatsEstimate.builder() diff --git a/core/trino-main/src/test/java/io/trino/cost/TestSimpleFilterProjectSemiJoinStatsRule.java b/core/trino-main/src/test/java/io/trino/cost/TestSimpleFilterProjectSemiJoinStatsRule.java index ebbbb73ffb82..6efbb3b5668f 100644 --- a/core/trino-main/src/test/java/io/trino/cost/TestSimpleFilterProjectSemiJoinStatsRule.java +++ b/core/trino-main/src/test/java/io/trino/cost/TestSimpleFilterProjectSemiJoinStatsRule.java @@ -96,8 +96,6 @@ public void testFilterPositiveSemiJoin() a, c, semiJoinOutput, - Optional.empty(), - Optional.empty(), Optional.empty())); }) .withSourceStats(LEFT_SOURCE_ID, PlanNodeStatsEstimate.builder() @@ -135,8 +133,6 @@ public void testFilterPositiveNarrowingProjectSemiJoin() a, c, semiJoinOutput, - Optional.empty(), - Optional.empty(), Optional.empty()))); }) .withSourceStats(LEFT_SOURCE_ID, PlanNodeStatsEstimate.builder() @@ -173,8 +169,6 @@ public void testFilterPositivePlusExtraConjunctSemiJoin() a, c, semiJoinOutput, - Optional.empty(), - Optional.empty(), Optional.empty())); }) .withSourceStats(LEFT_SOURCE_ID, PlanNodeStatsEstimate.builder() @@ -211,8 +205,6 @@ public void testFilterNegativeSemiJoin() a, c, semiJoinOutput, - Optional.empty(), - Optional.empty(), Optional.empty())); }) .withSourceStats(LEFT_SOURCE_ID, PlanNodeStatsEstimate.builder() diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestMultiSourcePartitionedScheduler.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestMultiSourcePartitionedScheduler.java index e651317bab6c..603da03857c3 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestMultiSourcePartitionedScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestMultiSourcePartitionedScheduler.java @@ -577,8 +577,6 @@ private PlanFragment createFragment(TableHandle firstTableHandle, TableHandle se Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(DYNAMIC_FILTER_ID, buildSymbol), Optional.empty()), ImmutableSet.of(symbol), diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSchedulingUtils.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSchedulingUtils.java index 6528e04afb58..7458a34a84a9 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSchedulingUtils.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSchedulingUtils.java @@ -293,8 +293,6 @@ private static JoinNode join(String id, PlanNode left, PlanNode right) Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); } @@ -309,8 +307,6 @@ private static SemiJoinNode semiJoin(String id, PlanNode left, PlanNode right) right.getOutputSymbols().get(0), new Symbol(UNKNOWN, id), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.empty()); } @@ -321,9 +317,7 @@ private static IndexJoinNode indexJoin(String id, PlanNode left, PlanNode right) IndexJoinNode.Type.INNER, left, right, - ImmutableList.of(), - Optional.empty(), - Optional.empty()); + ImmutableList.of()); } private static SpatialJoinNode spatialJoin(String id, PlanNode left, PlanNode right) diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java index df69d4813ed1..3ce3c7e111f9 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java @@ -706,8 +706,6 @@ private static PlanFragment createFragment() Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(DYNAMIC_FILTER_ID, buildSymbol), Optional.empty()), ImmutableSet.of(symbol), diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/PlanUtils.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/PlanUtils.java index af6ac85a8a52..c4703c992405 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/PlanUtils.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/PlanUtils.java @@ -91,8 +91,6 @@ static PlanFragment createBroadcastJoinPlanFragment(String name, PlanFragment bu remote.getOutputSymbols(), false, Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(REPLICATED), Optional.empty(), ImmutableMap.of(), @@ -120,8 +118,6 @@ static PlanFragment createJoinPlanFragment(JoinType joinType, JoinNode.Distribut build.getOutputSymbols(), false, Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(distributionType), Optional.empty(), ImmutableMap.of(), @@ -148,8 +144,6 @@ static PlanFragment createBroadcastAndPartitionedJoinPlanFragment( broadcastBuild.getOutputSymbols(), false, Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(REPLICATED), Optional.empty(), ImmutableMap.of(), @@ -164,8 +158,6 @@ static PlanFragment createBroadcastAndPartitionedJoinPlanFragment( partitionedBuild.getOutputSymbols(), false, Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(PARTITIONED), Optional.empty(), ImmutableMap.of(), diff --git a/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java index 3171e6c59a7b..1ec8b08141ee 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestHashSemiJoinOperator.java @@ -32,7 +32,6 @@ import org.junit.jupiter.api.parallel.Execution; import java.util.List; -import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; @@ -78,18 +77,12 @@ public void tearDown() @Test public void testSemiJoin() - { - testSemiJoin(true); - testSemiJoin(false); - } - - private void testSemiJoin(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); // build OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); - RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), BIGINT); + RowPagesBuilder rowPagesBuilder = rowPagesBuilder(false, Ints.asList(0), BIGINT); Operator buildOperator = new ValuesOperator(operatorContext, rowPagesBuilder .row(10L) .row(30L) @@ -104,7 +97,6 @@ private void testSemiJoin(boolean hashEnabled) new PlanNodeId("test"), rowPagesBuilder.getTypes().get(0), 0, - rowPagesBuilder.getHashChannel(), 10, new JoinCompiler(typeOperators), typeOperators); @@ -117,18 +109,16 @@ private void testSemiJoin(boolean hashEnabled) // probe List probeTypes = ImmutableList.of(BIGINT, BIGINT); - RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(hashEnabled, Ints.asList(0), BIGINT, BIGINT); + RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(false, Ints.asList(0), BIGINT, BIGINT); List probeInput = rowPagesBuilderProbe .addSequencePage(10, 30, 0) .build(); - Optional probeHashChannel = hashEnabled ? Optional.of(probeTypes.size()) : Optional.empty(); OperatorFactory joinOperatorFactory = HashSemiJoinOperator.createOperatorFactory( 2, new PlanNodeId("test"), setBuilderOperatorFactory.getSetProvider(), rowPagesBuilderProbe.getTypes(), - 0, - probeHashChannel); + 0); // expected MaterializedResult expected = resultBuilder(driverContext.getSession(), concat(probeTypes, ImmutableList.of(BOOLEAN))) @@ -144,23 +134,17 @@ private void testSemiJoin(boolean hashEnabled) .row(39L, 9L, false) .build(); - OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); + OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, false, ImmutableList.of(probeTypes.size())); } @Test public void testSemiJoinOnVarcharType() - { - testSemiJoinOnVarcharType(true); - testSemiJoinOnVarcharType(false); - } - - private void testSemiJoinOnVarcharType(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); // build OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); - RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), VARCHAR); + RowPagesBuilder rowPagesBuilder = rowPagesBuilder(false, Ints.asList(0), VARCHAR); Operator buildOperator = new ValuesOperator(operatorContext, rowPagesBuilder .row("10") .row("30") @@ -175,7 +159,6 @@ private void testSemiJoinOnVarcharType(boolean hashEnabled) new PlanNodeId("test"), rowPagesBuilder.getTypes().get(0), 0, - rowPagesBuilder.getHashChannel(), 10, new JoinCompiler(typeOperators), typeOperators); @@ -188,18 +171,16 @@ private void testSemiJoinOnVarcharType(boolean hashEnabled) // probe List probeTypes = ImmutableList.of(VARCHAR, BIGINT); - RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(hashEnabled, Ints.asList(0), VARCHAR, BIGINT); + RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(false, Ints.asList(0), VARCHAR, BIGINT); List probeInput = rowPagesBuilderProbe .addSequencePage(10, 30, 0) .build(); - Optional probeHashChannel = hashEnabled ? Optional.of(probeTypes.size()) : Optional.empty(); OperatorFactory joinOperatorFactory = HashSemiJoinOperator.createOperatorFactory( 2, new PlanNodeId("test"), setBuilderOperatorFactory.getSetProvider(), rowPagesBuilderProbe.getTypes(), - 0, - probeHashChannel); + 0); // expected MaterializedResult expected = resultBuilder(driverContext.getSession(), concat(probeTypes, ImmutableList.of(BOOLEAN))) @@ -215,24 +196,18 @@ private void testSemiJoinOnVarcharType(boolean hashEnabled) .row("39", 9L, false) .build(); - OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); + OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, false, ImmutableList.of(probeTypes.size())); } @Test public void testBuildSideNulls() - { - testBuildSideNulls(true); - testBuildSideNulls(false); - } - - private void testBuildSideNulls(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); // build OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder rowPagesBuilder = rowPagesBuilder(false, Ints.asList(0), buildTypes); Operator buildOperator = new ValuesOperator(operatorContext, rowPagesBuilder .row(0L) .row(1L) @@ -246,7 +221,6 @@ private void testBuildSideNulls(boolean hashEnabled) new PlanNodeId("test"), buildTypes.get(0), 0, - rowPagesBuilder.getHashChannel(), 10, new JoinCompiler(typeOperators), typeOperators); @@ -259,18 +233,16 @@ private void testBuildSideNulls(boolean hashEnabled) // probe List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(hashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = rowPagesBuilderProbe .addSequencePage(4, 1) .build(); - Optional probeHashChannel = hashEnabled ? Optional.of(probeTypes.size()) : Optional.empty(); OperatorFactory joinOperatorFactory = HashSemiJoinOperator.createOperatorFactory( 2, new PlanNodeId("test"), setBuilderOperatorFactory.getSetProvider(), rowPagesBuilderProbe.getTypes(), - 0, - probeHashChannel); + 0); // expected MaterializedResult expected = resultBuilder(driverContext.getSession(), concat(probeTypes, ImmutableList.of(BOOLEAN))) @@ -280,24 +252,18 @@ private void testBuildSideNulls(boolean hashEnabled) .row(4L, null) .build(); - OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); + OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, false, ImmutableList.of(probeTypes.size())); } @Test public void testProbeSideNulls() - { - testProbeSideNulls(true); - testProbeSideNulls(false); - } - - private void testProbeSideNulls(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); // build OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder rowPagesBuilder = rowPagesBuilder(false, Ints.asList(0), buildTypes); Operator buildOperator = new ValuesOperator(operatorContext, rowPagesBuilder .row(0L) .row(1L) @@ -308,7 +274,6 @@ private void testProbeSideNulls(boolean hashEnabled) new PlanNodeId("test"), buildTypes.get(0), 0, - rowPagesBuilder.getHashChannel(), 10, new JoinCompiler(typeOperators), typeOperators); @@ -321,21 +286,19 @@ private void testProbeSideNulls(boolean hashEnabled) // probe List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(hashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = rowPagesBuilderProbe .row(0L) .row((Object) null) .row(1L) .row(2L) .build(); - Optional probeHashChannel = hashEnabled ? Optional.of(probeTypes.size()) : Optional.empty(); OperatorFactory joinOperatorFactory = HashSemiJoinOperator.createOperatorFactory( 2, new PlanNodeId("test"), setBuilderOperatorFactory.getSetProvider(), rowPagesBuilderProbe.getTypes(), - 0, - probeHashChannel); + 0); // expected MaterializedResult expected = resultBuilder(driverContext.getSession(), concat(probeTypes, ImmutableList.of(BOOLEAN))) @@ -345,24 +308,18 @@ private void testProbeSideNulls(boolean hashEnabled) .row(2L, false) .build(); - OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); + OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, false, ImmutableList.of(probeTypes.size())); } @Test public void testProbeAndBuildNulls() - { - testProbeAndBuildNulls(true); - testProbeAndBuildNulls(false); - } - - private void testProbeAndBuildNulls(boolean hashEnabled) { DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); // build OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder rowPagesBuilder = rowPagesBuilder(false, Ints.asList(0), buildTypes); Operator buildOperator = new ValuesOperator(operatorContext, rowPagesBuilder .row(0L) .row(1L) @@ -374,7 +331,6 @@ private void testProbeAndBuildNulls(boolean hashEnabled) new PlanNodeId("test"), buildTypes.get(0), 0, - rowPagesBuilder.getHashChannel(), 10, new JoinCompiler(typeOperators), typeOperators); @@ -387,21 +343,19 @@ private void testProbeAndBuildNulls(boolean hashEnabled) // probe List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(hashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = rowPagesBuilderProbe .row(0L) .row((Object) null) .row(1L) .row(2L) .build(); - Optional probeHashChannel = hashEnabled ? Optional.of(probeTypes.size()) : Optional.empty(); OperatorFactory joinOperatorFactory = HashSemiJoinOperator.createOperatorFactory( 2, new PlanNodeId("test"), setBuilderOperatorFactory.getSetProvider(), rowPagesBuilderProbe.getTypes(), - 0, - probeHashChannel); + 0); // expected MaterializedResult expected = resultBuilder(driverContext.getSession(), concat(probeTypes, ImmutableList.of(BOOLEAN))) @@ -411,47 +365,39 @@ private void testProbeAndBuildNulls(boolean hashEnabled) .row(2L, null) .build(); - OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, hashEnabled, ImmutableList.of(probeTypes.size())); + OperatorAssertion.assertOperatorEquals(joinOperatorFactory, driverContext, probeInput, expected, false, ImmutableList.of(probeTypes.size())); } @Test public void testMemoryLimit() { - assertThatThrownBy(() -> testMemoryLimit(true)) - .isInstanceOf(ExceededMemoryLimitException.class) - .hasMessageMatching("Query exceeded per-node memory limit of.*"); - - assertThatThrownBy(() -> testMemoryLimit(false)) + assertThatThrownBy(() -> { + DriverContext driverContext = createTaskContext(executor, scheduledExecutor, TEST_SESSION, DataSize.ofBytes(100)) + .addPipelineContext(0, true, true, false) + .addDriverContext(); + + OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); + List buildTypes = ImmutableList.of(BIGINT); + RowPagesBuilder rowPagesBuilder = rowPagesBuilder(false, Ints.asList(0), buildTypes); + Operator buildOperator = new ValuesOperator(operatorContext, rowPagesBuilder + .addSequencePage(10000, 20) + .build()); + SetBuilderOperatorFactory setBuilderOperatorFactory = new SetBuilderOperatorFactory( + 1, + new PlanNodeId("test"), + buildTypes.get(0), + 0, + 10, + new JoinCompiler(typeOperators), + typeOperators); + Operator setBuilderOperator = setBuilderOperatorFactory.createOperator(driverContext); + + Driver driver = Driver.createDriver(driverContext, buildOperator, setBuilderOperator); + while (!driver.isFinished()) { + driver.processUntilBlocked(); + } + }) .isInstanceOf(ExceededMemoryLimitException.class) .hasMessageMatching("Query exceeded per-node memory limit of.*"); } - - private void testMemoryLimit(boolean hashEnabled) - { - DriverContext driverContext = createTaskContext(executor, scheduledExecutor, TEST_SESSION, DataSize.ofBytes(100)) - .addPipelineContext(0, true, true, false) - .addDriverContext(); - - OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); - List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), buildTypes); - Operator buildOperator = new ValuesOperator(operatorContext, rowPagesBuilder - .addSequencePage(10000, 20) - .build()); - SetBuilderOperatorFactory setBuilderOperatorFactory = new SetBuilderOperatorFactory( - 1, - new PlanNodeId("test"), - buildTypes.get(0), - 0, - rowPagesBuilder.getHashChannel(), - 10, - new JoinCompiler(typeOperators), - typeOperators); - Operator setBuilderOperator = setBuilderOperatorFactory.createOperator(driverContext); - - Driver driver = Driver.createDriver(driverContext, buildOperator, setBuilderOperator); - while (!driver.isFinished()) { - driver.processUntilBlocked(); - } - } } diff --git a/core/trino-main/src/test/java/io/trino/operator/TestPagesIndex.java b/core/trino-main/src/test/java/io/trino/operator/TestPagesIndex.java index 089cd72a38b5..ca1076fa0165 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestPagesIndex.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestPagesIndex.java @@ -26,7 +26,6 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import static io.airlift.slice.SizeOf.sizeOf; import static io.airlift.slice.SizeOf.sizeOfIntArray; @@ -149,7 +148,6 @@ public void testGetEstimatedLookupSourceSizeInBytes() LookupSource lookupSource = pagesIndex.createLookupSourceSupplier( TEST_SESSION, ImmutableList.of(joinChannel), - OptionalInt.empty(), sortChannel.map(channel -> filterFunctionFactory), sortChannel, ImmutableList.of(filterFunctionFactory), diff --git a/core/trino-main/src/test/java/io/trino/operator/TestSimplePagesHashStrategy.java b/core/trino-main/src/test/java/io/trino/operator/TestSimplePagesHashStrategy.java index bb6957e05a7e..5acabd736fe8 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestSimplePagesHashStrategy.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestSimplePagesHashStrategy.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import static io.trino.spi.type.IntegerType.INTEGER; import static org.assertj.core.api.Assertions.assertThat; @@ -142,7 +141,6 @@ private static SimplePagesHashStrategy createSimplePagesHashStrategy(Type type, ImmutableList.of(), ImmutableList.of(new ObjectArrayList<>(channelBlocks)), ImmutableList.of(0), - OptionalInt.empty(), Optional.of(0), new BlockTypeOperators()); } diff --git a/core/trino-main/src/test/java/io/trino/operator/join/BenchmarkHashBuildAndJoinOperators.java b/core/trino-main/src/test/java/io/trino/operator/join/BenchmarkHashBuildAndJoinOperators.java index 2528af20d536..e14846065a4e 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/BenchmarkHashBuildAndJoinOperators.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/BenchmarkHashBuildAndJoinOperators.java @@ -104,9 +104,6 @@ public static class BuildContext @Param({"varchar", "bigint", "all"}) protected String hashColumns = "bigint"; - @Param({"false", "true"}) - protected boolean buildHashEnabled; - @Param({"1", "5"}) protected int buildRowsRepetition = 1; @@ -116,7 +113,6 @@ public static class BuildContext protected ExecutorService executor; protected ScheduledExecutorService scheduledExecutor; protected List buildPages; - protected OptionalInt hashChannel; protected List types; protected List hashChannels; @@ -152,11 +148,6 @@ public TaskContext createTaskContext() return TestingTaskContext.createTaskContext(executor, scheduledExecutor, getSession(), DataSize.of(2, GIGABYTE)); } - public OptionalInt getHashChannel() - { - return hashChannel; - } - public List getHashChannels() { return hashChannels; @@ -174,7 +165,7 @@ public List getBuildPages() protected void initializeBuildPages() { - RowPagesBuilder buildPagesBuilder = rowPagesBuilder(buildHashEnabled, hashChannels, ImmutableList.of(VARCHAR, BIGINT, BIGINT)); + RowPagesBuilder buildPagesBuilder = rowPagesBuilder(false, hashChannels, ImmutableList.of(VARCHAR, BIGINT, BIGINT)); int maxValue = buildRowsNumber / buildRowsRepetition + 40; int rows = 0; @@ -187,8 +178,6 @@ protected void initializeBuildPages() types = buildPagesBuilder.getTypes(); buildPages = buildPagesBuilder.build(); - hashChannel = buildPagesBuilder.getHashChannel() - .map(OptionalInt::of).orElse(OptionalInt.empty()); } } @@ -240,7 +229,6 @@ public void setup() lookupSourceFactory, types, hashChannels, - hashChannel, Optional.of(outputChannels), OptionalInt.empty(), unsupportedPartitioningSpillerFactory(), @@ -261,7 +249,7 @@ public List getProbePages() protected void initializeProbePages() { - RowPagesBuilder probePagesBuilder = rowPagesBuilder(buildHashEnabled, hashChannels, ImmutableList.of(VARCHAR, BIGINT, BIGINT)); + RowPagesBuilder probePagesBuilder = rowPagesBuilder(false, hashChannels, ImmutableList.of(VARCHAR, BIGINT, BIGINT)); Random random = new Random(42); int remainingRows = PROBE_ROWS_NUMBER; @@ -339,7 +327,6 @@ private static void buildHash(BuildContext buildContext, JoinBridgeManager rangeList(int endExclusive) { return IntStream.range(0, endExclusive) diff --git a/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java index 01546017f55a..6b2b442728fa 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java @@ -93,12 +93,9 @@ import static io.trino.operator.JoinOperatorType.lookupOuterJoin; import static io.trino.operator.JoinOperatorType.probeOuterJoin; import static io.trino.operator.OperatorAssertion.assertOperatorEquals; -import static io.trino.operator.OperatorAssertion.dropChannel; -import static io.trino.operator.OperatorAssertion.without; import static io.trino.operator.OperatorFactories.spillingJoin; import static io.trino.operator.SpillMetrics.SPILL_COUNT_METRIC_NAME; import static io.trino.operator.join.JoinTestUtils.buildLookupSource; -import static io.trino.operator.join.JoinTestUtils.getHashChannelAsInt; import static io.trino.operator.join.JoinTestUtils.innerJoinOperatorFactory; import static io.trino.operator.join.JoinTestUtils.instantiateBuildDrivers; import static io.trino.operator.join.JoinTestUtils.runDriverInThread; @@ -148,28 +145,22 @@ public void tearDown() @Test public void testInnerJoin() { - testInnerJoin(true, true, true); - testInnerJoin(true, true, false); - testInnerJoin(true, false, true); - testInnerJoin(true, false, false); - testInnerJoin(false, true, true); - testInnerJoin(false, true, false); - testInnerJoin(false, false, true); - testInnerJoin(false, false, false); + testInnerJoin(true); + testInnerJoin(false); } - private void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoin(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), false, SINGLE_STREAM_SPILLER_FACTORY); JoinBridgeManager lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager(); // probe factory - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)); List probeInput = probePages .addSequencePage(1000, 0, 1000, 2000) .build(); @@ -193,7 +184,7 @@ private void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, bool .row("29", 1029L, 2029L, "29", 39L, 49L) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test @@ -230,7 +221,7 @@ public void testInnerJoinWithRunLengthEncodedProbe() .row("21", "21") .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test @@ -253,7 +244,7 @@ public void testYield() // build with 40 entries int entries = 40; - RowPagesBuilder buildPages = rowPagesBuilder(true, Ints.asList(0), ImmutableList.of(BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(BIGINT)) .addSequencePage(entries, 42); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, true, taskContext, buildPages, Optional.of(filterFunction), false, SINGLE_STREAM_SPILLER_FACTORY); JoinBridgeManager lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager(); @@ -268,7 +259,6 @@ public void testYield() lookupSourceFactory, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty(), OptionalInt.of(1), PARTITIONING_SPILLER_FACTORY, @@ -314,67 +304,62 @@ private enum WhenSpill public void testInnerJoinWithSpill() throws Exception { - for (boolean probeHashEnabled : ImmutableList.of(false, true)) { - // spill all - innerJoinWithSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.NEVER), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - innerJoinWithSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.DURING_BUILD), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - innerJoinWithSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.AFTER_BUILD), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - innerJoinWithSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.DURING_USAGE), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + // spill all + innerJoinWithSpill(nCopies(PARTITION_COUNT, WhenSpill.NEVER), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(nCopies(PARTITION_COUNT, WhenSpill.DURING_BUILD), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(nCopies(PARTITION_COUNT, WhenSpill.AFTER_BUILD), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(nCopies(PARTITION_COUNT, WhenSpill.DURING_USAGE), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - // spill one - innerJoinWithSpill(probeHashEnabled, concat(singletonList(WhenSpill.DURING_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - innerJoinWithSpill(probeHashEnabled, concat(singletonList(WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - innerJoinWithSpill(probeHashEnabled, concat(singletonList(WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + // spill one + innerJoinWithSpill(concat(singletonList(WhenSpill.DURING_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(concat(singletonList(WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(concat(singletonList(WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - innerJoinWithSpill(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - innerJoinWithSpill(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); - } + innerJoinWithSpill(concat(asList(WhenSpill.DURING_BUILD, WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); + innerJoinWithSpill(concat(asList(WhenSpill.DURING_BUILD, WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER)), SINGLE_STREAM_SPILLER_FACTORY, PARTITIONING_SPILLER_FACTORY); } @Test public void testInnerJoinWithFailingSpill() { - for (boolean probeHashEnabled : ImmutableList.of(false, true)) { - // spill all - testInnerJoinWithFailingSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.DURING_USAGE)); - testInnerJoinWithFailingSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.DURING_BUILD)); - testInnerJoinWithFailingSpill(probeHashEnabled, nCopies(PARTITION_COUNT, WhenSpill.AFTER_BUILD)); + // spill all + testInnerJoinWithFailingSpill(nCopies(PARTITION_COUNT, WhenSpill.DURING_USAGE)); + testInnerJoinWithFailingSpill(nCopies(PARTITION_COUNT, WhenSpill.DURING_BUILD)); + testInnerJoinWithFailingSpill(nCopies(PARTITION_COUNT, WhenSpill.AFTER_BUILD)); - // spill one - testInnerJoinWithFailingSpill(probeHashEnabled, concat(singletonList(WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER))); - testInnerJoinWithFailingSpill(probeHashEnabled, concat(singletonList(WhenSpill.DURING_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER))); - testInnerJoinWithFailingSpill(probeHashEnabled, concat(singletonList(WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER))); + // spill one + testInnerJoinWithFailingSpill(concat(singletonList(WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER))); + testInnerJoinWithFailingSpill(concat(singletonList(WhenSpill.DURING_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER))); + testInnerJoinWithFailingSpill(concat(singletonList(WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 1, WhenSpill.NEVER))); - testInnerJoinWithFailingSpill(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER))); - testInnerJoinWithFailingSpill(probeHashEnabled, concat(asList(WhenSpill.DURING_BUILD, WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER))); - } + testInnerJoinWithFailingSpill(concat(asList(WhenSpill.DURING_BUILD, WhenSpill.AFTER_BUILD), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER))); + testInnerJoinWithFailingSpill(concat(asList(WhenSpill.DURING_BUILD, WhenSpill.DURING_USAGE), nCopies(PARTITION_COUNT - 2, WhenSpill.NEVER))); } - private void testInnerJoinWithFailingSpill(boolean probeHashEnabled, List whenSpill) + private void testInnerJoinWithFailingSpill(List whenSpill) { assertThatThrownBy(() -> innerJoinWithSpill( - probeHashEnabled, whenSpill, new DummySpillerFactory().failSpill(), new GenericPartitioningSpillerFactory(new DummySpillerFactory()))) .isInstanceOf(RuntimeException.class) .hasMessage("Spill failed"); - assertThatThrownBy(() -> innerJoinWithSpill(probeHashEnabled, + assertThatThrownBy(() -> innerJoinWithSpill( whenSpill, new DummySpillerFactory(), new GenericPartitioningSpillerFactory(new DummySpillerFactory().failSpill()))) .isInstanceOf(RuntimeException.class) .hasMessage("Spill failed"); - assertThatThrownBy(() -> innerJoinWithSpill(probeHashEnabled, + assertThatThrownBy(() -> innerJoinWithSpill( whenSpill, new DummySpillerFactory().failUnspill(), new GenericPartitioningSpillerFactory(new DummySpillerFactory()))) .isInstanceOf(RuntimeException.class) .hasMessage("Unspill failed"); - assertThatThrownBy(() -> innerJoinWithSpill(probeHashEnabled, + assertThatThrownBy(() -> innerJoinWithSpill( whenSpill, new DummySpillerFactory(), new GenericPartitioningSpillerFactory(new DummySpillerFactory().failUnspill()))) @@ -382,7 +367,7 @@ private void testInnerJoinWithFailingSpill(boolean probeHashEnabled, List whenSpill, SingleStreamSpillerFactory buildSpillerFactory, PartitioningSpillerFactory joinSpillerFactory) + private void innerJoinWithSpill(List whenSpill, SingleStreamSpillerFactory buildSpillerFactory, PartitioningSpillerFactory joinSpillerFactory) throws Exception { TaskStateMachine taskStateMachine = new TaskStateMachine(new TaskId(new StageId("query", 0), 0, 0), executor); @@ -410,7 +395,7 @@ private void innerJoinWithSpill(boolean probeHashEnabled, List whenSp JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT)) + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT)) .row("20", 123_000L) .row("20", 123_000L) .pageBreak() @@ -495,7 +480,7 @@ private void innerJoinWithSpill(boolean probeHashEnabled, List whenSp .row("33", 123_003L, "33", 303L) .build(); - assertThat(getProperColumns(joinOperator, concat(probePages.getTypes(), buildPages.getTypes()), probePages, actualPages).getMaterializedRows()).containsExactlyInAnyOrderElementsOf(expected.getMaterializedRows()); + assertThat(getProperColumns(joinOperator, concat(probePages.getTypes(), buildPages.getTypes()), actualPages).getMaterializedRows()).containsExactlyInAnyOrderElementsOf(expected.getMaterializedRows()); Metrics probeMetrics = joinOperator.getOperatorContext() .getOperatorStats() @@ -558,13 +543,8 @@ private static List getPages(PageBuffer pageBuffer) return result; } - private static MaterializedResult getProperColumns(Operator joinOperator, List types, RowPagesBuilder probePages, List actualPages) + private static MaterializedResult getProperColumns(Operator joinOperator, List types, List actualPages) { - if (probePages.getHashChannel().isPresent()) { - List hashChannels = ImmutableList.of(probePages.getHashChannel().get()); - actualPages = dropChannel(actualPages, hashChannels); - types = without(types, hashChannels); - } return OperatorAssertion.toMaterializedResult(joinOperator.getOperatorContext().getSession(), types, actualPages); } @@ -617,23 +597,17 @@ public void testBuildGracefulSpill() @Test public void testInnerJoinWithNullProbe() { - testInnerJoinWithNullProbe(true, true, true); - testInnerJoinWithNullProbe(true, true, false); - testInnerJoinWithNullProbe(true, false, true); - testInnerJoinWithNullProbe(true, false, false); - testInnerJoinWithNullProbe(false, true, true); - testInnerJoinWithNullProbe(false, true, false); - testInnerJoinWithNullProbe(false, false, true); - testInnerJoinWithNullProbe(false, false, false); + testInnerJoinWithNullProbe(true); + testInnerJoinWithNullProbe(false); } - private void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithNullProbe(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row("a") .row("b") .row("c"); @@ -642,7 +616,7 @@ private void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHash // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row((String) null) @@ -663,28 +637,22 @@ private void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHash .row("b", "b") .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testInnerJoinWithOutputSingleMatch() { - testInnerJoinWithOutputSingleMatch(true, true, true); - testInnerJoinWithOutputSingleMatch(true, true, false); - testInnerJoinWithOutputSingleMatch(true, false, true); - testInnerJoinWithOutputSingleMatch(true, false, false); - testInnerJoinWithOutputSingleMatch(false, true, true); - testInnerJoinWithOutputSingleMatch(false, true, false); - testInnerJoinWithOutputSingleMatch(false, false, true); - testInnerJoinWithOutputSingleMatch(false, false, false); + testInnerJoinWithOutputSingleMatch(true); + testInnerJoinWithOutputSingleMatch(false); } - private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row("a") .row("a") .row("b"); @@ -693,7 +661,7 @@ private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean p // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row("b") @@ -711,29 +679,23 @@ private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean p .row("b", "b") .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testInnerJoinWithNullBuild() { - testInnerJoinWithNullBuild(true, true, true); - testInnerJoinWithNullBuild(true, true, false); - testInnerJoinWithNullBuild(true, false, true); - testInnerJoinWithNullBuild(true, false, false); - testInnerJoinWithNullBuild(false, true, true); - testInnerJoinWithNullBuild(false, true, false); - testInnerJoinWithNullBuild(false, false, true); - testInnerJoinWithNullBuild(false, false, false); + testInnerJoinWithNullBuild(true); + testInnerJoinWithNullBuild(false); } - private void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithNullBuild(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row("a") .row((String) null) .row((String) null) @@ -744,7 +706,7 @@ private void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHash // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row("b") @@ -763,29 +725,23 @@ private void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHash .row("b", "b") .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testInnerJoinWithNullOnBothSides() { - testInnerJoinWithNullOnBothSides(true, true, true); - testInnerJoinWithNullOnBothSides(true, true, false); - testInnerJoinWithNullOnBothSides(true, false, true); - testInnerJoinWithNullOnBothSides(true, false, false); - testInnerJoinWithNullOnBothSides(false, true, true); - testInnerJoinWithNullOnBothSides(false, true, false); - testInnerJoinWithNullOnBothSides(false, false, true); - testInnerJoinWithNullOnBothSides(false, false, false); + testInnerJoinWithNullOnBothSides(true); + testInnerJoinWithNullOnBothSides(false); } - private void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithNullOnBothSides(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row("a") .row((String) null) .row((String) null) @@ -796,7 +752,7 @@ private void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean pro // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row("b") @@ -816,36 +772,30 @@ private void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean pro .row("b", "b") .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testProbeOuterJoin() { - testProbeOuterJoin(true, true, true); - testProbeOuterJoin(true, true, false); - testProbeOuterJoin(true, false, true); - testProbeOuterJoin(true, false, false); - testProbeOuterJoin(false, true, true); - testProbeOuterJoin(false, true, false); - testProbeOuterJoin(false, false, true); - testProbeOuterJoin(false, false, false); + testProbeOuterJoin(true); + testProbeOuterJoin(false); } - private void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testProbeOuterJoin(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), false, SINGLE_STREAM_SPILLER_FACTORY); JoinBridgeManager lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .addSequencePage(15, 20, 1020, 2020) .build(); @@ -874,23 +824,17 @@ private void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, .row("34", 1034L, 2034L, null, null, null) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testProbeOuterJoinWithFilterFunction() { - testProbeOuterJoinWithFilterFunction(true, true, true); - testProbeOuterJoinWithFilterFunction(true, true, false); - testProbeOuterJoinWithFilterFunction(true, false, true); - testProbeOuterJoinWithFilterFunction(true, false, false); - testProbeOuterJoinWithFilterFunction(false, true, true); - testProbeOuterJoinWithFilterFunction(false, true, false); - testProbeOuterJoinWithFilterFunction(false, false, true); - testProbeOuterJoinWithFilterFunction(false, false, false); + testProbeOuterJoinWithFilterFunction(true); + testProbeOuterJoinWithFilterFunction(false); } - private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); @@ -899,14 +843,14 @@ private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean // build factory List buildTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.of(filterFunction), false, SINGLE_STREAM_SPILLER_FACTORY); JoinBridgeManager lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .addSequencePage(15, 20, 1020, 2020) .build(); @@ -935,29 +879,23 @@ private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean .row("34", 1034L, 2034L, null, null, null) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testOuterJoinWithNullProbe() { - testOuterJoinWithNullProbe(true, true, true); - testOuterJoinWithNullProbe(true, true, false); - testOuterJoinWithNullProbe(true, false, true); - testOuterJoinWithNullProbe(true, false, false); - testOuterJoinWithNullProbe(false, true, true); - testOuterJoinWithNullProbe(false, true, false); - testOuterJoinWithNullProbe(false, false, true); - testOuterJoinWithNullProbe(false, false, false); + testOuterJoinWithNullProbe(true); + testOuterJoinWithNullProbe(false); } - private void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testOuterJoinWithNullProbe(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row("a") .row("b") .row("c"); @@ -966,7 +904,7 @@ private void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHash // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row((String) null) @@ -989,23 +927,17 @@ private void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHash .row("b", "b") .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testOuterJoinWithNullProbeAndFilterFunction() { - testOuterJoinWithNullProbeAndFilterFunction(true, true, true); - testOuterJoinWithNullProbeAndFilterFunction(true, true, false); - testOuterJoinWithNullProbeAndFilterFunction(true, false, true); - testOuterJoinWithNullProbeAndFilterFunction(true, false, false); - testOuterJoinWithNullProbeAndFilterFunction(false, true, true); - testOuterJoinWithNullProbeAndFilterFunction(false, true, false); - testOuterJoinWithNullProbeAndFilterFunction(false, false, true); - testOuterJoinWithNullProbeAndFilterFunction(false, false, false); + testOuterJoinWithNullProbeAndFilterFunction(true); + testOuterJoinWithNullProbeAndFilterFunction(false); } - private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); @@ -1014,7 +946,7 @@ private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row("a") .row("b") .row("c"); @@ -1023,7 +955,7 @@ private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row((String) null) @@ -1046,29 +978,23 @@ private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, .row("b", null) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testOuterJoinWithNullBuild() { - testOuterJoinWithNullBuild(true, true, true); - testOuterJoinWithNullBuild(true, true, false); - testOuterJoinWithNullBuild(true, false, true); - testOuterJoinWithNullBuild(true, false, false); - testOuterJoinWithNullBuild(false, true, true); - testOuterJoinWithNullBuild(false, true, false); - testOuterJoinWithNullBuild(false, false, true); - testOuterJoinWithNullBuild(false, false, false); + testOuterJoinWithNullBuild(true); + testOuterJoinWithNullBuild(false); } - private void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testOuterJoinWithNullBuild(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR)) .row("a") .row((String) null) .row((String) null) @@ -1079,7 +1005,7 @@ private void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHash // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row("b") @@ -1099,23 +1025,17 @@ private void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHash .row("c", null) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testOuterJoinWithNullBuildAndFilterFunction() { - testOuterJoinWithNullBuildAndFilterFunction(true, true, true); - testOuterJoinWithNullBuildAndFilterFunction(true, true, false); - testOuterJoinWithNullBuildAndFilterFunction(true, false, true); - testOuterJoinWithNullBuildAndFilterFunction(true, false, false); - testOuterJoinWithNullBuildAndFilterFunction(false, true, true); - testOuterJoinWithNullBuildAndFilterFunction(false, true, false); - testOuterJoinWithNullBuildAndFilterFunction(false, false, true); - testOuterJoinWithNullBuildAndFilterFunction(false, false, false); + testOuterJoinWithNullBuildAndFilterFunction(true); + testOuterJoinWithNullBuildAndFilterFunction(false); } - private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); @@ -1125,7 +1045,7 @@ private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR)) .row("a") .row((String) null) .row((String) null) @@ -1136,7 +1056,7 @@ private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row("b") @@ -1156,28 +1076,22 @@ private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, .row("c", null) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testOuterJoinWithNullOnBothSides() { - testOuterJoinWithNullOnBothSides(true, true, true); - testOuterJoinWithNullOnBothSides(true, true, false); - testOuterJoinWithNullOnBothSides(true, false, true); - testOuterJoinWithNullOnBothSides(true, false, false); - testOuterJoinWithNullOnBothSides(false, true, true); - testOuterJoinWithNullOnBothSides(false, true, false); - testOuterJoinWithNullOnBothSides(false, false, true); - testOuterJoinWithNullOnBothSides(false, false, false); + testOuterJoinWithNullOnBothSides(true); + testOuterJoinWithNullOnBothSides(false); } - private void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testOuterJoinWithNullOnBothSides(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR)) .row("a") .row((String) null) .row((String) null) @@ -1188,7 +1102,7 @@ private void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean pro // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row("b") @@ -1210,23 +1124,17 @@ private void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean pro .row("c", null) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testOuterJoinWithNullOnBothSidesAndFilterFunction() { - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false); } - private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); @@ -1235,7 +1143,7 @@ private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelB ImmutableSet.of("a", "c").contains(VARCHAR.getSlice(rightPage.getBlock(0), rightPosition).toStringAscii())); // build factory - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR)) .row("a") .row((String) null) .row((String) null) @@ -1246,7 +1154,7 @@ private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelB // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row("b") @@ -1268,23 +1176,21 @@ private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelB .row("c", null) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testMemoryLimit() { - testMemoryLimit(true, true); - testMemoryLimit(true, false); - testMemoryLimit(false, true); - testMemoryLimit(false, false); + testMemoryLimit(true); + testMemoryLimit(false); } - private void testMemoryLimit(boolean parallelBuild, boolean buildHashEnabled) + private void testMemoryLimit(boolean parallelBuild) { TaskContext taskContext = TestingTaskContext.createTaskContext(executor, scheduledExecutor, TEST_SESSION, DataSize.ofBytes(100)); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), false, SINGLE_STREAM_SPILLER_FACTORY); instantiateBuildDrivers(buildSideSetup, taskContext); @@ -1320,7 +1226,6 @@ private void testHashBuilderFinishInputWaitsForMemory(boolean spillEnabled) 0, ImmutableList.of(0), ImmutableList.of(1), - OptionalInt.empty(), Optional.empty(), Optional.empty(), ImmutableList.of(), @@ -1386,7 +1291,6 @@ public void testHashBuilderUnspillWaitsForMemory() 0, ImmutableList.of(0), ImmutableList.of(0), - OptionalInt.empty(), Optional.empty(), Optional.empty(), ImmutableList.of(), @@ -1458,7 +1362,6 @@ public void testMemoryRevokeCompactionUpdatesRevocableMemory() 0, ImmutableList.of(0), ImmutableList.of(0), - OptionalInt.empty(), Optional.empty(), Optional.empty(), ImmutableList.of(), @@ -1496,29 +1399,23 @@ public void testMemoryRevokeCompactionUpdatesRevocableMemory() @Test public void testInnerJoinWithEmptyLookupSource() { - testInnerJoinWithEmptyLookupSource(true, true, true); - testInnerJoinWithEmptyLookupSource(true, true, false); - testInnerJoinWithEmptyLookupSource(true, false, true); - testInnerJoinWithEmptyLookupSource(true, false, false); - testInnerJoinWithEmptyLookupSource(false, true, true); - testInnerJoinWithEmptyLookupSource(false, true, false); - testInnerJoinWithEmptyLookupSource(false, false, true); - testInnerJoinWithEmptyLookupSource(false, false, false); + testInnerJoinWithEmptyLookupSource(true); + testInnerJoinWithEmptyLookupSource(false); } - private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), false, SINGLE_STREAM_SPILLER_FACTORY); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); OperatorFactory joinOperatorFactory = spillingJoin( innerJoin(false, false), 0, @@ -1526,7 +1423,6 @@ private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean p lookupSourceFactoryManager, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty(), OptionalInt.of(1), PARTITIONING_SPILLER_FACTORY, @@ -1546,29 +1442,23 @@ private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean p @Test public void testLookupOuterJoinWithEmptyLookupSource() { - testLookupOuterJoinWithEmptyLookupSource(true, true, true); - testLookupOuterJoinWithEmptyLookupSource(true, true, false); - testLookupOuterJoinWithEmptyLookupSource(true, false, true); - testLookupOuterJoinWithEmptyLookupSource(true, false, false); - testLookupOuterJoinWithEmptyLookupSource(false, true, true); - testLookupOuterJoinWithEmptyLookupSource(false, true, false); - testLookupOuterJoinWithEmptyLookupSource(false, false, true); - testLookupOuterJoinWithEmptyLookupSource(false, false, false); + testLookupOuterJoinWithEmptyLookupSource(true); + testLookupOuterJoinWithEmptyLookupSource(false); } - private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), false, SINGLE_STREAM_SPILLER_FACTORY); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); OperatorFactory joinOperatorFactory = spillingJoin( lookupOuterJoin(false), 0, @@ -1576,7 +1466,6 @@ private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boo lookupSourceFactoryManager, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty(), OptionalInt.of(1), PARTITIONING_SPILLER_FACTORY, @@ -1596,29 +1485,23 @@ private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boo @Test public void testProbeOuterJoinWithEmptyLookupSource() { - testProbeOuterJoinWithEmptyLookupSource(true, true, true); - testProbeOuterJoinWithEmptyLookupSource(true, true, false); - testProbeOuterJoinWithEmptyLookupSource(true, false, true); - testProbeOuterJoinWithEmptyLookupSource(true, false, false); - testProbeOuterJoinWithEmptyLookupSource(false, true, true); - testProbeOuterJoinWithEmptyLookupSource(false, true, false); - testProbeOuterJoinWithEmptyLookupSource(false, false, true); - testProbeOuterJoinWithEmptyLookupSource(false, false, false); + testProbeOuterJoinWithEmptyLookupSource(true); + testProbeOuterJoinWithEmptyLookupSource(false); } - private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), false, SINGLE_STREAM_SPILLER_FACTORY); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row("b") @@ -1632,7 +1515,6 @@ private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, bool lookupSourceFactoryManager, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty(), OptionalInt.of(1), PARTITIONING_SPILLER_FACTORY, @@ -1649,35 +1531,29 @@ private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, bool .row(null, null) .row("c", null) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testFullOuterJoinWithEmptyLookupSource() { - testFullOuterJoinWithEmptyLookupSource(true, true, true); - testFullOuterJoinWithEmptyLookupSource(true, true, false); - testFullOuterJoinWithEmptyLookupSource(true, false, true); - testFullOuterJoinWithEmptyLookupSource(true, false, false); - testFullOuterJoinWithEmptyLookupSource(false, true, true); - testFullOuterJoinWithEmptyLookupSource(false, true, false); - testFullOuterJoinWithEmptyLookupSource(false, false, true); - testFullOuterJoinWithEmptyLookupSource(false, false, false); + testFullOuterJoinWithEmptyLookupSource(true); + testFullOuterJoinWithEmptyLookupSource(false); } - private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), false, SINGLE_STREAM_SPILLER_FACTORY); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") .row("b") @@ -1691,7 +1567,6 @@ private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boole lookupSourceFactoryManager, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty(), OptionalInt.of(1), PARTITIONING_SPILLER_FACTORY, @@ -1708,29 +1583,23 @@ private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boole .row(null, null) .row("c", null) .build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe() { - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false); } - private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row("a") .row("b") .row((String) null) @@ -1740,7 +1609,7 @@ private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallel // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages.build(); OperatorFactory joinOperatorFactory = spillingJoin( innerJoin(false, false), @@ -1749,7 +1618,6 @@ private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallel lookupSourceFactoryManager, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty(), OptionalInt.of(1), PARTITIONING_SPILLER_FACTORY, @@ -1761,29 +1629,23 @@ private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallel // expected MaterializedResult expected = MaterializedResult.resultBuilder(taskContext.getSession(), concat(probeTypes, buildTypes)).build(); - assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected, true, getHashChannels(probePages, buildPages)); + assertOperatorEquals(joinOperatorFactory, taskContext.addPipelineContext(0, true, true, false).addDriverContext(), probeInput, expected); } @Test public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe() throws Exception { - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, true, true); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, true, false); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, false, true); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, false, false); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, true, true); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, true, false); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, false, true); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, false, false); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false); } - private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelBuild) throws Exception { // join that waits for build side to be collected TaskContext taskContext = createTaskContext(); - OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, true); + OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, true); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); @@ -1798,7 +1660,7 @@ private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallel // join that doesn't wait for build side to be collected taskContext = createTaskContext(); - joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, false); + joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, false); driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); @@ -1817,25 +1679,19 @@ private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallel public void testInnerJoinWithBlockingLookupSource() throws Exception { - testInnerJoinWithBlockingLookupSource(true, true, true); - testInnerJoinWithBlockingLookupSource(true, true, false); - testInnerJoinWithBlockingLookupSource(true, false, true); - testInnerJoinWithBlockingLookupSource(true, false, false); - testInnerJoinWithBlockingLookupSource(false, true, true); - testInnerJoinWithBlockingLookupSource(false, true, false); - testInnerJoinWithBlockingLookupSource(false, false, true); - testInnerJoinWithBlockingLookupSource(false, false, false); + testInnerJoinWithBlockingLookupSource(true); + testInnerJoinWithBlockingLookupSource(false); } - private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild) throws Exception { - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR)); Page probePage = getOnlyElement(probePages.addSequencePage(1, 0).build()); // join that waits for build side to be collected TaskContext taskContext = createTaskContext(); - OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, true); + OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, true); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); @@ -1849,7 +1705,7 @@ private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolea // join that doesn't wait for build side to be collected taskContext = createTaskContext(); - joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, false); + joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, false); driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); @@ -1868,17 +1724,17 @@ private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolea } } - private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskContext taskContext, boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean waitForBuild) + private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskContext taskContext, boolean parallelBuild, boolean waitForBuild) { // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), false, SINGLE_STREAM_SPILLER_FACTORY); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); OperatorFactory joinOperatorFactory = spillingJoin( innerJoin(false, waitForBuild), 0, @@ -1886,7 +1742,6 @@ private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskCo lookupSourceFactoryManager, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty(), OptionalInt.of(1), PARTITIONING_SPILLER_FACTORY, @@ -1903,18 +1758,6 @@ private TaskContext createTaskContext() return TestingTaskContext.createTaskContext(executor, scheduledExecutor, TEST_SESSION); } - private static List getHashChannels(RowPagesBuilder probe, RowPagesBuilder build) - { - ImmutableList.Builder hashChannels = ImmutableList.builder(); - if (probe.getHashChannel().isPresent()) { - hashChannels.add(probe.getHashChannel().get()); - } - if (build.getHashChannel().isPresent()) { - hashChannels.add(probe.getTypes().size() + build.getHashChannel().get()); - } - return hashChannels.build(); - } - private OperatorFactory probeOuterJoinOperatorFactory( JoinBridgeManager lookupSourceFactoryManager, RowPagesBuilder probePages) @@ -1926,7 +1769,6 @@ private OperatorFactory probeOuterJoinOperatorFactory( lookupSourceFactoryManager, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty(), OptionalInt.of(1), PARTITIONING_SPILLER_FACTORY, diff --git a/core/trino-main/src/test/java/io/trino/operator/join/TestLookupJoinPageBuilder.java b/core/trino-main/src/test/java/io/trino/operator/join/TestLookupJoinPageBuilder.java index 00ef3c21858e..6616b7aacaa1 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/TestLookupJoinPageBuilder.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/TestLookupJoinPageBuilder.java @@ -25,7 +25,6 @@ import org.junit.jupiter.api.Test; import java.util.List; -import java.util.OptionalInt; import static io.trino.spi.type.BigintType.BIGINT; import static org.assertj.core.api.Assertions.assertThat; @@ -43,7 +42,7 @@ public void testPageBuilder() Block block = blockBuilder.build(); Page page = new Page(block, block); - JoinProbeFactory joinProbeFactory = new JoinProbeFactory(new int[] {0, 1}, ImmutableList.of(0, 1), OptionalInt.empty()); + JoinProbeFactory joinProbeFactory = new JoinProbeFactory(new int[] {0, 1}, ImmutableList.of(0, 1)); JoinProbe probe = joinProbeFactory.createJoinProbe(page); LookupSource lookupSource = new TestLookupSource(ImmutableList.of(BIGINT, BIGINT), page); LookupJoinPageBuilder lookupJoinPageBuilder = new LookupJoinPageBuilder(ImmutableList.of(BIGINT, BIGINT)); @@ -91,7 +90,7 @@ public void testDifferentPositions() } Block block = blockBuilder.build(); Page page = new Page(block); - JoinProbeFactory joinProbeFactory = new JoinProbeFactory(new int[] {0}, ImmutableList.of(0), OptionalInt.empty()); + JoinProbeFactory joinProbeFactory = new JoinProbeFactory(new int[] {0}, ImmutableList.of(0)); LookupSource lookupSource = new TestLookupSource(ImmutableList.of(BIGINT), page); LookupJoinPageBuilder lookupJoinPageBuilder = new LookupJoinPageBuilder(ImmutableList.of(BIGINT)); @@ -163,7 +162,7 @@ public void testCrossJoinWithEmptyBuild() // nothing on the build side so we don't append anything LookupSource lookupSource = new TestLookupSource(ImmutableList.of(), page); - JoinProbe probe = new JoinProbeFactory(new int[] {0}, ImmutableList.of(0), OptionalInt.empty()).createJoinProbe(page); + JoinProbe probe = new JoinProbeFactory(new int[] {0}, ImmutableList.of(0)).createJoinProbe(page); LookupJoinPageBuilder lookupJoinPageBuilder = new LookupJoinPageBuilder(ImmutableList.of(BIGINT)); // append the same row many times should also flush in the end diff --git a/core/trino-main/src/test/java/io/trino/operator/join/TestPositionLinks.java b/core/trino-main/src/test/java/io/trino/operator/join/TestPositionLinks.java index 3c95d22a8ee2..8c896ce61f6b 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/TestPositionLinks.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/TestPositionLinks.java @@ -25,7 +25,6 @@ import org.junit.jupiter.api.Test; import java.util.Optional; -import java.util.OptionalInt; import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.operator.SyntheticAddress.encodeSyntheticAddress; @@ -316,7 +315,6 @@ private static PagesHashStrategy pagesHashStrategy() ImmutableList.of(), ImmutableList.of(new ObjectArrayList<>(ImmutableList.of(TEST_PAGE.getBlock(0)))), ImmutableList.of(), - OptionalInt.empty(), Optional.of(0), new BlockTypeOperators(new TypeOperators())); } diff --git a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/BenchmarkHashBuildAndJoinOperators.java b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/BenchmarkHashBuildAndJoinOperators.java index 65c5c5e86b20..2a8470071b9f 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/BenchmarkHashBuildAndJoinOperators.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/BenchmarkHashBuildAndJoinOperators.java @@ -53,7 +53,6 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; @@ -115,7 +114,6 @@ public static class BuildContext protected ExecutorService executor; protected ScheduledExecutorService scheduledExecutor; protected List buildPages; - protected OptionalInt hashChannel; protected List types; protected List hashChannels; @@ -151,11 +149,6 @@ public TaskContext createTaskContext() return TestingTaskContext.createTaskContext(executor, scheduledExecutor, getSession(), DataSize.of(2, GIGABYTE)); } - public OptionalInt getHashChannel() - { - return hashChannel; - } - public List getHashChannels() { return hashChannels; @@ -186,8 +179,6 @@ protected void initializeBuildPages() types = buildPagesBuilder.getTypes(); buildPages = buildPagesBuilder.build(); - hashChannel = buildPagesBuilder.getHashChannel() - .map(OptionalInt::of).orElse(OptionalInt.empty()); } } @@ -240,7 +231,6 @@ public void setup() false, types, hashChannels, - hashChannel, Optional.of(outputChannels)); buildHash(this, lookupSourceFactory, outputChannels, partitionCount); initializeProbePages(); @@ -340,7 +330,6 @@ private static void buildHash(BuildContext buildContext, JoinBridgeManager rangeList(int endExclusive) { return IntStream.range(0, endExclusive) diff --git a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashBuilderOperator.java b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashBuilderOperator.java index 78106124ca68..265b4186037c 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashBuilderOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashBuilderOperator.java @@ -34,7 +34,6 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; @@ -108,7 +107,6 @@ public void test() 0, ImmutableList.of(0), ImmutableList.of(1), - OptionalInt.empty(), Optional.empty(), Optional.empty(), ImmutableList.of(), diff --git a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashJoinOperator.java index 5db97138ecb5..72897d6fa01e 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestHashJoinOperator.java @@ -68,12 +68,10 @@ import static io.trino.operator.JoinOperatorType.innerJoin; import static io.trino.operator.JoinOperatorType.probeOuterJoin; import static io.trino.operator.OperatorAssertion.assertOperatorEquals; -import static io.trino.operator.OperatorAssertion.dropChannel; import static io.trino.operator.OperatorAssertion.toMaterializedResult; import static io.trino.operator.OperatorAssertion.toPages; import static io.trino.operator.OperatorFactories.join; import static io.trino.operator.join.unspilled.JoinTestUtils.buildLookupSource; -import static io.trino.operator.join.unspilled.JoinTestUtils.getHashChannelAsInt; import static io.trino.operator.join.unspilled.JoinTestUtils.innerJoinOperatorFactory; import static io.trino.operator.join.unspilled.JoinTestUtils.instantiateBuildDrivers; import static io.trino.operator.join.unspilled.JoinTestUtils.setupBuildSide; @@ -113,28 +111,22 @@ public void tearDown() @Test public void testInnerJoin() { - testInnerJoin(false, false, false); - testInnerJoin(false, false, true); - testInnerJoin(false, true, false); - testInnerJoin(false, true, true); - testInnerJoin(true, false, false); - testInnerJoin(true, false, true); - testInnerJoin(true, true, false); - testInnerJoin(true, true, true); + testInnerJoin(false); + testInnerJoin(true); } - private void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoin(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty()); JoinBridgeManager lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager(); // probe factory - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)); List probeInput = probePages .addSequencePage(1000, 0, 1000, 2000) .build(); @@ -164,17 +156,13 @@ private void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, bool @Test public void testInnerJoinWithRunLengthEncodedProbe() { - testInnerJoinWithRunLengthEncodedProbe(false, false, false); - testInnerJoinWithRunLengthEncodedProbe(false, false, true); - testInnerJoinWithRunLengthEncodedProbe(false, true, false); - testInnerJoinWithRunLengthEncodedProbe(false, true, true); - testInnerJoinWithRunLengthEncodedProbe(true, false, false); - testInnerJoinWithRunLengthEncodedProbe(true, false, true); - testInnerJoinWithRunLengthEncodedProbe(true, true, false); - testInnerJoinWithRunLengthEncodedProbe(true, true, true); + testInnerJoinWithRunLengthEncodedProbe(false, false); + testInnerJoinWithRunLengthEncodedProbe(false, true); + testInnerJoinWithRunLengthEncodedProbe(true, false); + testInnerJoinWithRunLengthEncodedProbe(true, true); } - private void testInnerJoinWithRunLengthEncodedProbe(boolean withFilter, boolean probeHashEnabled, boolean singleBigintLookupSource) + private void testInnerJoinWithRunLengthEncodedProbe(boolean withFilter, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -187,7 +175,7 @@ private void testInnerJoinWithRunLengthEncodedProbe(boolean withFilter, boolean JoinBridgeManager lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager(); // probe factory - RowPagesBuilder probePagesBuilder = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT)) + RowPagesBuilder probePagesBuilder = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT)) .addBlocksPage( RunLengthEncodedBlock.create(VARCHAR, Slices.utf8Slice("20"), 2), createLongsBlock(42, 43)) @@ -205,10 +193,6 @@ private void testInnerJoinWithRunLengthEncodedProbe(boolean withFilter, boolean DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); List pages = toPages(joinOperatorFactory, driverContext, probePagesBuilder.build(), true, true); - if (probeHashEnabled) { - // Drop the hashChannel for all pages - pages = dropChannel(pages, getHashChannels(probePagesBuilder, buildPages)); - } assertThat(pages).hasSize(2); if (withFilter) { @@ -268,7 +252,7 @@ private void testYield(boolean singleBigintLookupSource) // build with 40 entries int entries = 40; - RowPagesBuilder buildPages = rowPagesBuilder(true, Ints.asList(0), ImmutableList.of(BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(BIGINT)) .addSequencePage(entries, 42); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, true, taskContext, buildPages, Optional.of(filterFunction), singleBigintLookupSource); JoinBridgeManager lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager(); @@ -284,7 +268,6 @@ private void testYield(boolean singleBigintLookupSource) true, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty()); instantiateBuildDrivers(buildSideSetup, taskContext); @@ -321,31 +304,19 @@ private void testYield(boolean singleBigintLookupSource) @Test public void testInnerJoinWithNullProbe() { - testInnerJoinWithNullProbe(false, false, false, false); - testInnerJoinWithNullProbe(false, false, false, true); - testInnerJoinWithNullProbe(false, false, true, false); - testInnerJoinWithNullProbe(false, false, true, true); - testInnerJoinWithNullProbe(false, true, false, false); - testInnerJoinWithNullProbe(false, true, false, true); - testInnerJoinWithNullProbe(false, true, true, false); - testInnerJoinWithNullProbe(false, true, true, true); - testInnerJoinWithNullProbe(true, false, false, false); - testInnerJoinWithNullProbe(true, false, false, true); - testInnerJoinWithNullProbe(true, false, true, false); - testInnerJoinWithNullProbe(true, false, true, true); - testInnerJoinWithNullProbe(true, true, false, false); - testInnerJoinWithNullProbe(true, true, false, true); - testInnerJoinWithNullProbe(true, true, true, false); - testInnerJoinWithNullProbe(true, true, true, true); + testInnerJoinWithNullProbe(false, false); + testInnerJoinWithNullProbe(false, true); + testInnerJoinWithNullProbe(true, false); + testInnerJoinWithNullProbe(true, true); } - private void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testInnerJoinWithNullProbe(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row(1L) .row(2L) .row(3L); @@ -354,7 +325,7 @@ private void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHash // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row((String) null) @@ -381,30 +352,18 @@ private void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHash @Test public void testInnerJoinWithOutputSingleMatch() { - testInnerJoinWithOutputSingleMatch(false, false, false, false); - testInnerJoinWithOutputSingleMatch(false, false, false, true); - testInnerJoinWithOutputSingleMatch(false, false, true, false); - testInnerJoinWithOutputSingleMatch(false, false, true, true); - testInnerJoinWithOutputSingleMatch(false, true, false, false); - testInnerJoinWithOutputSingleMatch(false, true, false, true); - testInnerJoinWithOutputSingleMatch(false, true, true, false); - testInnerJoinWithOutputSingleMatch(false, true, true, true); - testInnerJoinWithOutputSingleMatch(true, false, false, false); - testInnerJoinWithOutputSingleMatch(true, false, false, true); - testInnerJoinWithOutputSingleMatch(true, false, true, false); - testInnerJoinWithOutputSingleMatch(true, false, true, true); - testInnerJoinWithOutputSingleMatch(true, true, false, false); - testInnerJoinWithOutputSingleMatch(true, true, false, true); - testInnerJoinWithOutputSingleMatch(true, true, true, false); - testInnerJoinWithOutputSingleMatch(true, true, true, true); + testInnerJoinWithOutputSingleMatch(false, false); + testInnerJoinWithOutputSingleMatch(false, true); + testInnerJoinWithOutputSingleMatch(true, false); + testInnerJoinWithOutputSingleMatch(true, true); } - private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row(1L) .row(1L) .row(2L); @@ -413,7 +372,7 @@ private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean p // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row(2L) @@ -437,23 +396,17 @@ private void testInnerJoinWithOutputSingleMatch(boolean parallelBuild, boolean p @Test public void testInnerJoinWithNullBuild() { - testInnerJoinWithNullBuild(false, false, false); - testInnerJoinWithNullBuild(false, false, true); - testInnerJoinWithNullBuild(false, true, false); - testInnerJoinWithNullBuild(false, true, true); - testInnerJoinWithNullBuild(true, false, false); - testInnerJoinWithNullBuild(true, false, true); - testInnerJoinWithNullBuild(true, true, false); - testInnerJoinWithNullBuild(true, true, true); + testInnerJoinWithNullBuild(false); + testInnerJoinWithNullBuild(true); } - private void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithNullBuild(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row(1L) .row((String) null) .row((String) null) @@ -464,7 +417,7 @@ private void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHash // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row(2L) @@ -489,23 +442,17 @@ private void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHash @Test public void testInnerJoinWithNullOnBothSides() { - testInnerJoinWithNullOnBothSides(false, false, false); - testInnerJoinWithNullOnBothSides(false, false, true); - testInnerJoinWithNullOnBothSides(false, true, false); - testInnerJoinWithNullOnBothSides(false, true, true); - testInnerJoinWithNullOnBothSides(true, false, false); - testInnerJoinWithNullOnBothSides(true, false, true); - testInnerJoinWithNullOnBothSides(true, true, false); - testInnerJoinWithNullOnBothSides(true, true, true); + testInnerJoinWithNullOnBothSides(false); + testInnerJoinWithNullOnBothSides(true); } - private void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithNullOnBothSides(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row(1L) .row((String) null) .row((String) null) @@ -516,7 +463,7 @@ private void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean pro // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row(2L) @@ -542,30 +489,24 @@ private void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean pro @Test public void testProbeOuterJoin() { - testProbeOuterJoin(false, false, false); - testProbeOuterJoin(false, false, true); - testProbeOuterJoin(false, true, false); - testProbeOuterJoin(false, true, true); - testProbeOuterJoin(true, false, false); - testProbeOuterJoin(true, false, true); - testProbeOuterJoin(true, true, false); - testProbeOuterJoin(true, true, true); + testProbeOuterJoin(false); + testProbeOuterJoin(true); } - private void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testProbeOuterJoin(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty()); JoinBridgeManager lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .addSequencePage(15, 20, 1020, 2020) .build(); @@ -600,17 +541,11 @@ private void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, @Test public void testProbeOuterJoinWithFilterFunction() { - testProbeOuterJoinWithFilterFunction(false, false, false); - testProbeOuterJoinWithFilterFunction(false, false, true); - testProbeOuterJoinWithFilterFunction(false, true, false); - testProbeOuterJoinWithFilterFunction(false, true, true); - testProbeOuterJoinWithFilterFunction(true, false, false); - testProbeOuterJoinWithFilterFunction(true, false, true); - testProbeOuterJoinWithFilterFunction(true, true, false); - testProbeOuterJoinWithFilterFunction(true, true, true); + testProbeOuterJoinWithFilterFunction(false); + testProbeOuterJoinWithFilterFunction(true); } - private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild) { TaskContext taskContext = createTaskContext(); @@ -619,14 +554,14 @@ private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean // build factory List buildTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.of(filterFunction)); JoinBridgeManager lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .addSequencePage(15, 20, 1020, 2020) .build(); @@ -661,31 +596,19 @@ private void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean @Test public void testOuterJoinWithNullProbe() { - testOuterJoinWithNullProbe(false, false, false, false); - testOuterJoinWithNullProbe(false, false, false, true); - testOuterJoinWithNullProbe(false, false, true, false); - testOuterJoinWithNullProbe(false, false, true, true); - testOuterJoinWithNullProbe(false, true, false, false); - testOuterJoinWithNullProbe(false, true, false, true); - testOuterJoinWithNullProbe(false, true, true, false); - testOuterJoinWithNullProbe(false, true, true, true); - testOuterJoinWithNullProbe(true, false, false, false); - testOuterJoinWithNullProbe(true, false, false, true); - testOuterJoinWithNullProbe(true, false, true, false); - testOuterJoinWithNullProbe(true, false, true, true); - testOuterJoinWithNullProbe(true, true, false, false); - testOuterJoinWithNullProbe(true, true, false, true); - testOuterJoinWithNullProbe(true, true, true, false); - testOuterJoinWithNullProbe(true, true, true, true); + testOuterJoinWithNullProbe(false, false); + testOuterJoinWithNullProbe(false, true); + testOuterJoinWithNullProbe(true, false); + testOuterJoinWithNullProbe(true, true); } - private void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testOuterJoinWithNullProbe(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row(1L) .row(2L) .row(3L); @@ -694,7 +617,7 @@ private void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHash // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row((String) null) @@ -723,25 +646,13 @@ private void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHash @Test public void testOuterJoinWithNullProbeAndFilterFunction() { - testOuterJoinWithNullProbeAndFilterFunction(false, false, false, false); - testOuterJoinWithNullProbeAndFilterFunction(false, false, false, true); - testOuterJoinWithNullProbeAndFilterFunction(false, false, true, false); - testOuterJoinWithNullProbeAndFilterFunction(false, false, true, true); - testOuterJoinWithNullProbeAndFilterFunction(false, true, false, false); - testOuterJoinWithNullProbeAndFilterFunction(false, true, false, true); - testOuterJoinWithNullProbeAndFilterFunction(false, true, true, false); - testOuterJoinWithNullProbeAndFilterFunction(false, true, true, true); - testOuterJoinWithNullProbeAndFilterFunction(true, false, false, false); - testOuterJoinWithNullProbeAndFilterFunction(true, false, false, true); - testOuterJoinWithNullProbeAndFilterFunction(true, false, true, false); - testOuterJoinWithNullProbeAndFilterFunction(true, false, true, true); - testOuterJoinWithNullProbeAndFilterFunction(true, true, false, false); - testOuterJoinWithNullProbeAndFilterFunction(true, true, false, true); - testOuterJoinWithNullProbeAndFilterFunction(true, true, true, false); - testOuterJoinWithNullProbeAndFilterFunction(true, true, true, true); + testOuterJoinWithNullProbeAndFilterFunction(false, false); + testOuterJoinWithNullProbeAndFilterFunction(false, true); + testOuterJoinWithNullProbeAndFilterFunction(true, false); + testOuterJoinWithNullProbeAndFilterFunction(true, true); } - private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -750,7 +661,7 @@ private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row(1L) .row(2L) .row(3L); @@ -759,7 +670,7 @@ private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row((String) null) @@ -788,31 +699,19 @@ private void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, @Test public void testOuterJoinWithNullBuild() { - testOuterJoinWithNullBuild(false, false, false, false); - testOuterJoinWithNullBuild(false, false, false, true); - testOuterJoinWithNullBuild(false, false, true, false); - testOuterJoinWithNullBuild(false, false, true, true); - testOuterJoinWithNullBuild(false, true, false, false); - testOuterJoinWithNullBuild(false, true, false, true); - testOuterJoinWithNullBuild(false, true, true, false); - testOuterJoinWithNullBuild(false, true, true, true); - testOuterJoinWithNullBuild(true, false, false, false); - testOuterJoinWithNullBuild(true, false, false, true); - testOuterJoinWithNullBuild(true, false, true, false); - testOuterJoinWithNullBuild(true, false, true, true); - testOuterJoinWithNullBuild(true, true, false, false); - testOuterJoinWithNullBuild(true, true, false, true); - testOuterJoinWithNullBuild(true, true, true, false); - testOuterJoinWithNullBuild(true, true, true, true); + testOuterJoinWithNullBuild(false, false); + testOuterJoinWithNullBuild(false, true); + testOuterJoinWithNullBuild(true, false); + testOuterJoinWithNullBuild(true, true); } - private void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testOuterJoinWithNullBuild(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(BIGINT)) .row(1L) .row((String) null) .row((String) null) @@ -823,7 +722,7 @@ private void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHash // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row(2L) @@ -849,25 +748,13 @@ private void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHash @Test public void testOuterJoinWithNullBuildAndFilterFunction() { - testOuterJoinWithNullBuildAndFilterFunction(false, false, false, false); - testOuterJoinWithNullBuildAndFilterFunction(false, false, false, true); - testOuterJoinWithNullBuildAndFilterFunction(false, false, true, false); - testOuterJoinWithNullBuildAndFilterFunction(false, false, true, true); - testOuterJoinWithNullBuildAndFilterFunction(false, true, false, false); - testOuterJoinWithNullBuildAndFilterFunction(false, true, false, true); - testOuterJoinWithNullBuildAndFilterFunction(false, true, true, false); - testOuterJoinWithNullBuildAndFilterFunction(false, true, true, true); - testOuterJoinWithNullBuildAndFilterFunction(true, false, false, false); - testOuterJoinWithNullBuildAndFilterFunction(true, false, false, true); - testOuterJoinWithNullBuildAndFilterFunction(true, false, true, false); - testOuterJoinWithNullBuildAndFilterFunction(true, false, true, true); - testOuterJoinWithNullBuildAndFilterFunction(true, true, false, false); - testOuterJoinWithNullBuildAndFilterFunction(true, true, false, true); - testOuterJoinWithNullBuildAndFilterFunction(true, true, true, false); - testOuterJoinWithNullBuildAndFilterFunction(true, true, true, true); + testOuterJoinWithNullBuildAndFilterFunction(false, false); + testOuterJoinWithNullBuildAndFilterFunction(false, true); + testOuterJoinWithNullBuildAndFilterFunction(true, false); + testOuterJoinWithNullBuildAndFilterFunction(true, true); } - private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -877,7 +764,7 @@ private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(BIGINT)) .row(1L) .row((String) null) .row((String) null) @@ -888,7 +775,7 @@ private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row(2L) @@ -914,30 +801,18 @@ private void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, @Test public void testOuterJoinWithNullOnBothSides() { - testOuterJoinWithNullOnBothSides(false, false, false, false); - testOuterJoinWithNullOnBothSides(false, false, false, true); - testOuterJoinWithNullOnBothSides(false, false, true, false); - testOuterJoinWithNullOnBothSides(false, false, true, true); - testOuterJoinWithNullOnBothSides(false, true, false, false); - testOuterJoinWithNullOnBothSides(false, true, false, true); - testOuterJoinWithNullOnBothSides(false, true, true, false); - testOuterJoinWithNullOnBothSides(false, true, true, true); - testOuterJoinWithNullOnBothSides(true, false, false, false); - testOuterJoinWithNullOnBothSides(true, false, false, true); - testOuterJoinWithNullOnBothSides(true, false, true, false); - testOuterJoinWithNullOnBothSides(true, false, true, true); - testOuterJoinWithNullOnBothSides(true, true, false, false); - testOuterJoinWithNullOnBothSides(true, true, false, true); - testOuterJoinWithNullOnBothSides(true, true, true, false); - testOuterJoinWithNullOnBothSides(true, true, true, true); + testOuterJoinWithNullOnBothSides(false, false); + testOuterJoinWithNullOnBothSides(false, true); + testOuterJoinWithNullOnBothSides(true, false); + testOuterJoinWithNullOnBothSides(true, true); } - private void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(BIGINT)) .row(1L) .row((String) null) .row((String) null) @@ -948,7 +823,7 @@ private void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean pro // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row(2L) @@ -976,25 +851,13 @@ private void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean pro @Test public void testOuterJoinWithNullOnBothSidesAndFilterFunction() { - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, false, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, false, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, true, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false, true, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, false, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, false, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, true, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true, true, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, false, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, false, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, true, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false, true, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, false, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, false, true); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, true, false); - testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true, true, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(false, true); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, false); + testOuterJoinWithNullOnBothSidesAndFilterFunction(true, true); } - private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); @@ -1003,7 +866,7 @@ private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelB ImmutableSet.of(1L, 3L).contains(BIGINT.getLong(rightPage.getBlock(0), rightPosition))); // build factory - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(BIGINT)) .row(1L) .row((String) null) .row((String) null) @@ -1014,7 +877,7 @@ private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelB // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row(2L) @@ -1042,17 +905,15 @@ private void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelB @Test public void testMemoryLimit() { - testMemoryLimit(false, false); - testMemoryLimit(false, true); - testMemoryLimit(true, false); - testMemoryLimit(true, true); + testMemoryLimit(false); + testMemoryLimit(true); } - private void testMemoryLimit(boolean parallelBuild, boolean buildHashEnabled) + private void testMemoryLimit(boolean parallelBuild) { TaskContext taskContext = TestingTaskContext.createTaskContext(executor, scheduledExecutor, TEST_SESSION, DataSize.ofBytes(100)); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty()); instantiateBuildDrivers(buildSideSetup, taskContext); @@ -1065,37 +926,25 @@ private void testMemoryLimit(boolean parallelBuild, boolean buildHashEnabled) @Test public void testInnerJoinWithEmptyLookupSource() { - testInnerJoinWithEmptyLookupSource(false, false, false, false); - testInnerJoinWithEmptyLookupSource(false, false, false, true); - testInnerJoinWithEmptyLookupSource(false, false, true, false); - testInnerJoinWithEmptyLookupSource(false, false, true, true); - testInnerJoinWithEmptyLookupSource(false, true, false, false); - testInnerJoinWithEmptyLookupSource(false, true, false, true); - testInnerJoinWithEmptyLookupSource(false, true, true, false); - testInnerJoinWithEmptyLookupSource(false, true, true, true); - testInnerJoinWithEmptyLookupSource(true, false, false, false); - testInnerJoinWithEmptyLookupSource(true, false, false, true); - testInnerJoinWithEmptyLookupSource(true, false, true, false); - testInnerJoinWithEmptyLookupSource(true, false, true, true); - testInnerJoinWithEmptyLookupSource(true, true, false, false); - testInnerJoinWithEmptyLookupSource(true, true, false, true); - testInnerJoinWithEmptyLookupSource(true, true, true, false); - testInnerJoinWithEmptyLookupSource(true, true, true, true); + testInnerJoinWithEmptyLookupSource(false, false); + testInnerJoinWithEmptyLookupSource(false, true); + testInnerJoinWithEmptyLookupSource(true, false); + testInnerJoinWithEmptyLookupSource(true, true); } - private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), singleBigintLookupSource); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); OperatorFactory joinOperatorFactory = join( innerJoin(false, false), 0, @@ -1104,7 +953,6 @@ private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean p false, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty()); // drivers and operators @@ -1121,37 +969,25 @@ private void testInnerJoinWithEmptyLookupSource(boolean parallelBuild, boolean p @Test public void testLookupOuterJoinWithEmptyLookupSource() { - testLookupOuterJoinWithEmptyLookupSource(false, false, false, false); - testLookupOuterJoinWithEmptyLookupSource(false, false, false, true); - testLookupOuterJoinWithEmptyLookupSource(false, false, true, false); - testLookupOuterJoinWithEmptyLookupSource(false, false, true, true); - testLookupOuterJoinWithEmptyLookupSource(false, true, false, false); - testLookupOuterJoinWithEmptyLookupSource(false, true, false, true); - testLookupOuterJoinWithEmptyLookupSource(false, true, true, false); - testLookupOuterJoinWithEmptyLookupSource(false, true, true, true); - testLookupOuterJoinWithEmptyLookupSource(true, false, false, false); - testLookupOuterJoinWithEmptyLookupSource(true, false, false, true); - testLookupOuterJoinWithEmptyLookupSource(true, false, true, false); - testLookupOuterJoinWithEmptyLookupSource(true, false, true, true); - testLookupOuterJoinWithEmptyLookupSource(true, true, false, false); - testLookupOuterJoinWithEmptyLookupSource(true, true, false, true); - testLookupOuterJoinWithEmptyLookupSource(true, true, true, false); - testLookupOuterJoinWithEmptyLookupSource(true, true, true, true); + testLookupOuterJoinWithEmptyLookupSource(false, false); + testLookupOuterJoinWithEmptyLookupSource(false, true); + testLookupOuterJoinWithEmptyLookupSource(true, false); + testLookupOuterJoinWithEmptyLookupSource(true, true); } - private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), singleBigintLookupSource); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); OperatorFactory joinOperatorFactory = join( JoinOperatorType.lookupOuterJoin(false), 0, @@ -1160,7 +996,6 @@ private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boo false, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty()); // drivers and operators @@ -1177,37 +1012,25 @@ private void testLookupOuterJoinWithEmptyLookupSource(boolean parallelBuild, boo @Test public void testProbeOuterJoinWithEmptyLookupSource() { - testProbeOuterJoinWithEmptyLookupSource(false, false, false, false); - testProbeOuterJoinWithEmptyLookupSource(false, false, false, true); - testProbeOuterJoinWithEmptyLookupSource(false, false, true, false); - testProbeOuterJoinWithEmptyLookupSource(false, false, true, true); - testProbeOuterJoinWithEmptyLookupSource(false, true, false, false); - testProbeOuterJoinWithEmptyLookupSource(false, true, false, true); - testProbeOuterJoinWithEmptyLookupSource(false, true, true, false); - testProbeOuterJoinWithEmptyLookupSource(false, true, true, true); - testProbeOuterJoinWithEmptyLookupSource(true, false, false, false); - testProbeOuterJoinWithEmptyLookupSource(true, false, false, true); - testProbeOuterJoinWithEmptyLookupSource(true, false, true, false); - testProbeOuterJoinWithEmptyLookupSource(true, false, true, true); - testProbeOuterJoinWithEmptyLookupSource(true, true, false, false); - testProbeOuterJoinWithEmptyLookupSource(true, true, false, true); - testProbeOuterJoinWithEmptyLookupSource(true, true, true, false); - testProbeOuterJoinWithEmptyLookupSource(true, true, true, true); + testProbeOuterJoinWithEmptyLookupSource(false, false); + testProbeOuterJoinWithEmptyLookupSource(false, true); + testProbeOuterJoinWithEmptyLookupSource(true, false); + testProbeOuterJoinWithEmptyLookupSource(true, true); } - private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), singleBigintLookupSource); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row(2L) @@ -1222,7 +1045,6 @@ private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, bool false, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty()); // build drivers and operators @@ -1242,37 +1064,25 @@ private void testProbeOuterJoinWithEmptyLookupSource(boolean parallelBuild, bool @Test public void testFullOuterJoinWithEmptyLookupSource() { - testFullOuterJoinWithEmptyLookupSource(false, false, false, false); - testFullOuterJoinWithEmptyLookupSource(false, false, false, true); - testFullOuterJoinWithEmptyLookupSource(false, false, true, false); - testFullOuterJoinWithEmptyLookupSource(false, false, true, true); - testFullOuterJoinWithEmptyLookupSource(false, true, false, false); - testFullOuterJoinWithEmptyLookupSource(false, true, false, true); - testFullOuterJoinWithEmptyLookupSource(false, true, true, false); - testFullOuterJoinWithEmptyLookupSource(false, true, true, true); - testFullOuterJoinWithEmptyLookupSource(true, false, false, false); - testFullOuterJoinWithEmptyLookupSource(true, false, false, true); - testFullOuterJoinWithEmptyLookupSource(true, false, true, false); - testFullOuterJoinWithEmptyLookupSource(true, false, true, true); - testFullOuterJoinWithEmptyLookupSource(true, true, false, false); - testFullOuterJoinWithEmptyLookupSource(true, true, false, true); - testFullOuterJoinWithEmptyLookupSource(true, true, true, false); - testFullOuterJoinWithEmptyLookupSource(true, true, true, true); + testFullOuterJoinWithEmptyLookupSource(false, false); + testFullOuterJoinWithEmptyLookupSource(false, true); + testFullOuterJoinWithEmptyLookupSource(true, false); + testFullOuterJoinWithEmptyLookupSource(true, true); } - private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty(), singleBigintLookupSource); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages .row(1L) .row(2L) @@ -1287,7 +1097,6 @@ private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boole false, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty()); // build drivers and operators @@ -1307,31 +1116,19 @@ private void testFullOuterJoinWithEmptyLookupSource(boolean parallelBuild, boole @Test public void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe() { - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, false, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, false, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, true, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false, true, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, false, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, false, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, true, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true, true, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, false, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, false, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, true, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false, true, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, false, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, false, true); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, true, false); - testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true, true, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(false, true); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, false); + testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(true, true); } - private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean singleBigintLookupSource) + private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallelBuild, boolean singleBigintLookupSource) { TaskContext taskContext = createTaskContext(); // build factory List buildTypes = ImmutableList.of(BIGINT); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes) .row(1L) .row(2L) .row((String) null) @@ -1341,7 +1138,7 @@ private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallel // probe factory List probeTypes = ImmutableList.of(BIGINT); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); List probeInput = probePages.build(); OperatorFactory joinOperatorFactory = join( innerJoin(false, false), @@ -1351,7 +1148,6 @@ private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallel false, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty()); // build drivers and operators @@ -1367,22 +1163,16 @@ private void testInnerJoinWithNonEmptyLookupSourceAndEmptyProbe(boolean parallel public void testInnerJoinWithBlockingLookupSourceAndEmptyProbe() throws Exception { - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, false, false); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, false, true); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, true, false); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false, true, true); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, false, false); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, false, true); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, true, false); - testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true, true, true); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(false); + testInnerJoinWithBlockingLookupSourceAndEmptyProbe(true); } - private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallelBuild) throws Exception { // join that waits for build side to be collected TaskContext taskContext = createTaskContext(); - OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, true); + OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, true); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); @@ -1397,7 +1187,7 @@ private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallel // join that doesn't wait for build side to be collected taskContext = createTaskContext(); - joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, false); + joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, false); driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); @@ -1416,25 +1206,19 @@ private void testInnerJoinWithBlockingLookupSourceAndEmptyProbe(boolean parallel public void testInnerJoinWithBlockingLookupSource() throws Exception { - testInnerJoinWithBlockingLookupSource(false, false, false); - testInnerJoinWithBlockingLookupSource(false, false, true); - testInnerJoinWithBlockingLookupSource(false, true, false); - testInnerJoinWithBlockingLookupSource(false, true, true); - testInnerJoinWithBlockingLookupSource(true, false, false); - testInnerJoinWithBlockingLookupSource(true, false, true); - testInnerJoinWithBlockingLookupSource(true, true, false); - testInnerJoinWithBlockingLookupSource(true, true, true); + testInnerJoinWithBlockingLookupSource(false); + testInnerJoinWithBlockingLookupSource(true); } - private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled) + private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild) throws Exception { - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), ImmutableList.of(VARCHAR)); Page probePage = getOnlyElement(probePages.addSequencePage(1, 0).build()); // join that waits for build side to be collected TaskContext taskContext = createTaskContext(); - OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, true); + OperatorFactory joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, true); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); @@ -1448,7 +1232,7 @@ private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolea // join that doesn't wait for build side to be collected taskContext = createTaskContext(); - joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, probeHashEnabled, buildHashEnabled, false); + joinOperatorFactory = createJoinOperatorFactoryWithBlockingLookupSource(taskContext, parallelBuild, false); driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); try (Operator joinOperator = joinOperatorFactory.createOperator(driverContext)) { joinOperatorFactory.noMoreOperators(); @@ -1467,17 +1251,17 @@ private void testInnerJoinWithBlockingLookupSource(boolean parallelBuild, boolea } } - private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskContext taskContext, boolean parallelBuild, boolean probeHashEnabled, boolean buildHashEnabled, boolean waitForBuild) + private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskContext taskContext, boolean parallelBuild, boolean waitForBuild) { // build factory List buildTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes); + RowPagesBuilder buildPages = rowPagesBuilder(false, Ints.asList(0), buildTypes); BuildSideSetup buildSideSetup = setupBuildSide(nodePartitioningManager, parallelBuild, taskContext, buildPages, Optional.empty()); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); // probe factory List probeTypes = ImmutableList.of(VARCHAR); - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); + RowPagesBuilder probePages = rowPagesBuilder(false, Ints.asList(0), probeTypes); OperatorFactory joinOperatorFactory = join( innerJoin(false, waitForBuild), 0, @@ -1486,7 +1270,6 @@ private OperatorFactory createJoinOperatorFactoryWithBlockingLookupSource(TaskCo false, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty()); // build drivers and operators @@ -1525,7 +1308,6 @@ private OperatorFactory probeOuterJoinOperatorFactory( hasFilter, probePages.getTypes(), Ints.asList(0), - getHashChannelAsInt(probePages), Optional.empty()); } diff --git a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestLookupJoinPageBuilder.java b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestLookupJoinPageBuilder.java index 7795a0d011c5..ecd2518b1a8d 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestLookupJoinPageBuilder.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/unspilled/TestLookupJoinPageBuilder.java @@ -26,7 +26,6 @@ import org.junit.jupiter.api.Test; import java.util.List; -import java.util.OptionalInt; import static io.trino.spi.type.BigintType.BIGINT; import static org.assertj.core.api.Assertions.assertThat; @@ -44,7 +43,7 @@ public void testPageBuilder() Block block = blockBuilder.build(); Page page = new Page(block, block); - JoinProbeFactory joinProbeFactory = new JoinProbeFactory(ImmutableList.of(0, 1), ImmutableList.of(0, 1), OptionalInt.empty(), false); + JoinProbeFactory joinProbeFactory = new JoinProbeFactory(ImmutableList.of(0, 1), ImmutableList.of(0, 1), false); LookupSource lookupSource = new TestLookupSource(ImmutableList.of(BIGINT, BIGINT), page); JoinProbe probe = joinProbeFactory.createJoinProbe(page, lookupSource); LookupJoinPageBuilder lookupJoinPageBuilder = new LookupJoinPageBuilder(ImmutableList.of(BIGINT, BIGINT)); @@ -92,7 +91,7 @@ public void testDifferentPositions() } Block block = blockBuilder.build(); Page page = new Page(block); - JoinProbeFactory joinProbeFactory = new JoinProbeFactory(ImmutableList.of(0), ImmutableList.of(0), OptionalInt.empty(), false); + JoinProbeFactory joinProbeFactory = new JoinProbeFactory(ImmutableList.of(0), ImmutableList.of(0), false); LookupSource lookupSource = new TestLookupSource(ImmutableList.of(BIGINT), page); LookupJoinPageBuilder lookupJoinPageBuilder = new LookupJoinPageBuilder(ImmutableList.of(BIGINT)); @@ -164,7 +163,7 @@ public void testCrossJoinWithEmptyBuild() // nothing on the build side so we don't append anything LookupSource lookupSource = new TestLookupSource(ImmutableList.of(), page); - JoinProbe probe = new JoinProbeFactory(ImmutableList.of(0), ImmutableList.of(0), OptionalInt.empty(), false).createJoinProbe(page, lookupSource); + JoinProbe probe = new JoinProbeFactory(ImmutableList.of(0), ImmutableList.of(0), false).createJoinProbe(page, lookupSource); LookupJoinPageBuilder lookupJoinPageBuilder = new LookupJoinPageBuilder(ImmutableList.of(BIGINT)); // append the same row many times should also flush in the end diff --git a/core/trino-main/src/test/java/io/trino/server/TestDynamicFilterService.java b/core/trino-main/src/test/java/io/trino/server/TestDynamicFilterService.java index b17037445974..b394053400e5 100644 --- a/core/trino-main/src/test/java/io/trino/server/TestDynamicFilterService.java +++ b/core/trino-main/src/test/java/io/trino/server/TestDynamicFilterService.java @@ -1055,8 +1055,6 @@ private static PlanFragment createPlan( Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(producedDynamicFilterId, buildSymbol), Optional.empty()), ImmutableSet.of(symbol), diff --git a/core/trino-main/src/test/java/io/trino/sql/gen/TestJoinCompiler.java b/core/trino-main/src/test/java/io/trino/sql/gen/TestJoinCompiler.java index 579ed657d7f2..a9c0e57f1fa1 100644 --- a/core/trino-main/src/test/java/io/trino/sql/gen/TestJoinCompiler.java +++ b/core/trino-main/src/test/java/io/trino/sql/gen/TestJoinCompiler.java @@ -28,14 +28,11 @@ import io.trino.type.BlockTypeOperators.BlockPositionEqual; import io.trino.type.BlockTypeOperators.BlockPositionHashCode; import io.trino.type.BlockTypeOperators.BlockPositionIsIdentical; -import io.trino.type.TypeTestUtils; import it.unimi.dsi.fastutil.objects.ObjectArrayList; import org.junit.jupiter.api.Test; -import java.util.Arrays; import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import static io.airlift.slice.SizeOf.instanceSize; import static io.airlift.slice.SizeOf.sizeOf; @@ -56,241 +53,203 @@ public class TestJoinCompiler @Test public void testSingleChannel() { - for (boolean hashEnabled : Arrays.asList(true, false)) { - List joinTypes = ImmutableList.of(VARCHAR); - List joinChannels = Ints.asList(0); - - // compile a single channel hash strategy - PagesHashStrategyFactory pagesHashStrategyFactory = joinCompiler.compilePagesHashStrategyFactory(joinTypes, joinChannels); - - // create hash strategy with a single channel blocks -- make sure there is some overlap in values - ObjectArrayList channel = new ObjectArrayList<>(); - channel.add(BlockAssertions.createStringSequenceBlock(10, 20)); - channel.add(BlockAssertions.createStringSequenceBlock(20, 30)); - channel.add(BlockAssertions.createStringSequenceBlock(15, 25)); - - OptionalInt hashChannel = OptionalInt.empty(); - List> channels = ImmutableList.of(channel); - if (hashEnabled) { - ObjectArrayList hashChannelBuilder = new ObjectArrayList<>(); - for (Block block : channel) { - hashChannelBuilder.add(TypeTestUtils.getHashBlock(joinTypes, block)); - } - hashChannel = OptionalInt.of(1); - channels = ImmutableList.of(channel, hashChannelBuilder); - } - PagesHashStrategy hashStrategy = pagesHashStrategyFactory.createPagesHashStrategy(channels, hashChannel); - - // verify channel count - assertThat(hashStrategy.getChannelCount()).isEqualTo(1); - - BlockTypeOperators blockTypeOperators = new BlockTypeOperators(); - BlockPositionEqual equalOperator = blockTypeOperators.getEqualOperator(VARCHAR); - BlockPositionIsIdentical identicalOperator = blockTypeOperators.getIdenticalOperator(VARCHAR); - BlockPositionHashCode hashCodeOperator = blockTypeOperators.getHashCodeOperator(VARCHAR); - - // verify hashStrategy is consistent with equals and hash code from block - for (int leftBlockIndex = 0; leftBlockIndex < channel.size(); leftBlockIndex++) { - Block leftBlock = channel.get(leftBlockIndex); - - PageBuilder pageBuilder = new PageBuilder(ImmutableList.of(VARCHAR)); - - for (int leftBlockPosition = 0; leftBlockPosition < leftBlock.getPositionCount(); leftBlockPosition++) { - // hash code of position must match block hash - assertThat(hashStrategy.hashPosition(leftBlockIndex, leftBlockPosition)).isEqualTo(hashCodeOperator.hashCodeNullSafe(leftBlock, leftBlockPosition)); - - // position must be equal to itself - assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, leftBlockIndex, leftBlockPosition)).isTrue(); - - // check equality of every position against every other position in the block - for (int rightBlockIndex = 0; rightBlockIndex < channel.size(); rightBlockIndex++) { - Block rightBlock = channel.get(rightBlockIndex); - for (int rightBlockPosition = 0; rightBlockPosition < rightBlock.getPositionCount(); rightBlockPosition++) { - boolean expected = equalOperator.equalNullSafe(leftBlock, leftBlockPosition, rightBlock, rightBlockPosition); - boolean expectedIdentical = identicalOperator.isIdentical(leftBlock, leftBlockPosition, rightBlock, rightBlockPosition); - assertThat(hashStrategy.positionEqualsRow(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); - assertThat(hashStrategy.positionIdenticalToRow(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expectedIdentical); - assertThat(hashStrategy.rowEqualsRow(leftBlockPosition, new Page(leftBlock), rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); - assertThat(hashStrategy.rowIdenticalToRow(leftBlockPosition, new Page(leftBlock), rightBlockPosition, new Page(rightBlock))).isEqualTo(expectedIdentical); - assertThat(hashStrategy.positionEqualsRowIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); - assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expected); - assertThat(hashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expected); - assertThat(hashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedIdentical); - } - } + List joinTypes = ImmutableList.of(VARCHAR); + List joinChannels = Ints.asList(0); + + // compile a single channel hash strategy + PagesHashStrategyFactory pagesHashStrategyFactory = joinCompiler.compilePagesHashStrategyFactory(joinTypes, joinChannels); + + // create hash strategy with a single channel blocks -- make sure there is some overlap in values + ObjectArrayList channel = new ObjectArrayList<>(); + channel.add(BlockAssertions.createStringSequenceBlock(10, 20)); + channel.add(BlockAssertions.createStringSequenceBlock(20, 30)); + channel.add(BlockAssertions.createStringSequenceBlock(15, 25)); + + List> channels = ImmutableList.of(channel); + PagesHashStrategy hashStrategy = pagesHashStrategyFactory.createPagesHashStrategy(channels); + + // verify channel count + assertThat(hashStrategy.getChannelCount()).isEqualTo(1); - // check equality of every position against every other position in the block cursor - for (int rightBlockIndex = 0; rightBlockIndex < channel.size(); rightBlockIndex++) { - Block rightBlock = channel.get(rightBlockIndex); - for (int rightBlockPosition = 0; rightBlockPosition < rightBlock.getPositionCount(); rightBlockPosition++) { - boolean expected = equalOperator.equalNullSafe(leftBlock, leftBlockPosition, rightBlock, rightBlockPosition); - boolean expectedIdentical = identicalOperator.isIdentical(leftBlock, leftBlockPosition, rightBlock, rightBlockPosition); - assertThat(hashStrategy.positionEqualsRow(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); - assertThat(hashStrategy.positionIdenticalToRow(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expectedIdentical); - assertThat(hashStrategy.rowEqualsRow(leftBlockPosition, new Page(leftBlock), rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); - assertThat(hashStrategy.rowIdenticalToRow(leftBlockPosition, new Page(leftBlock), rightBlockPosition, new Page(rightBlock))).isEqualTo(expectedIdentical); - assertThat(hashStrategy.positionEqualsRowIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); - assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expected); - assertThat(hashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expected); - assertThat(hashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedIdentical); - } + BlockTypeOperators blockTypeOperators = new BlockTypeOperators(); + BlockPositionEqual equalOperator = blockTypeOperators.getEqualOperator(VARCHAR); + BlockPositionIsIdentical identicalOperator = blockTypeOperators.getIdenticalOperator(VARCHAR); + BlockPositionHashCode hashCodeOperator = blockTypeOperators.getHashCodeOperator(VARCHAR); + + // verify hashStrategy is consistent with equals and hash code from block + for (int leftBlockIndex = 0; leftBlockIndex < channel.size(); leftBlockIndex++) { + Block leftBlock = channel.get(leftBlockIndex); + + PageBuilder pageBuilder = new PageBuilder(ImmutableList.of(VARCHAR)); + + for (int leftBlockPosition = 0; leftBlockPosition < leftBlock.getPositionCount(); leftBlockPosition++) { + // hash code of position must match block hash + assertThat(hashStrategy.hashPosition(leftBlockIndex, leftBlockPosition)).isEqualTo(hashCodeOperator.hashCodeNullSafe(leftBlock, leftBlockPosition)); + + // position must be equal to itself + assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, leftBlockIndex, leftBlockPosition)).isTrue(); + + // check equality of every position against every other position in the block + for (int rightBlockIndex = 0; rightBlockIndex < channel.size(); rightBlockIndex++) { + Block rightBlock = channel.get(rightBlockIndex); + for (int rightBlockPosition = 0; rightBlockPosition < rightBlock.getPositionCount(); rightBlockPosition++) { + boolean expected = equalOperator.equalNullSafe(leftBlock, leftBlockPosition, rightBlock, rightBlockPosition); + boolean expectedIdentical = identicalOperator.isIdentical(leftBlock, leftBlockPosition, rightBlock, rightBlockPosition); + assertThat(hashStrategy.positionEqualsRow(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); + assertThat(hashStrategy.positionIdenticalToRow(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expectedIdentical); + assertThat(hashStrategy.rowEqualsRow(leftBlockPosition, new Page(leftBlock), rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); + assertThat(hashStrategy.rowIdenticalToRow(leftBlockPosition, new Page(leftBlock), rightBlockPosition, new Page(rightBlock))).isEqualTo(expectedIdentical); + assertThat(hashStrategy.positionEqualsRowIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); + assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expected); + assertThat(hashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expected); + assertThat(hashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedIdentical); } + } - // write position to output block - pageBuilder.declarePosition(); - hashStrategy.appendTo(leftBlockIndex, leftBlockPosition, pageBuilder, 0); + // check equality of every position against every other position in the block cursor + for (int rightBlockIndex = 0; rightBlockIndex < channel.size(); rightBlockIndex++) { + Block rightBlock = channel.get(rightBlockIndex); + for (int rightBlockPosition = 0; rightBlockPosition < rightBlock.getPositionCount(); rightBlockPosition++) { + boolean expected = equalOperator.equalNullSafe(leftBlock, leftBlockPosition, rightBlock, rightBlockPosition); + boolean expectedIdentical = identicalOperator.isIdentical(leftBlock, leftBlockPosition, rightBlock, rightBlockPosition); + assertThat(hashStrategy.positionEqualsRow(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); + assertThat(hashStrategy.positionIdenticalToRow(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expectedIdentical); + assertThat(hashStrategy.rowEqualsRow(leftBlockPosition, new Page(leftBlock), rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); + assertThat(hashStrategy.rowIdenticalToRow(leftBlockPosition, new Page(leftBlock), rightBlockPosition, new Page(rightBlock))).isEqualTo(expectedIdentical); + assertThat(hashStrategy.positionEqualsRowIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockPosition, new Page(rightBlock))).isEqualTo(expected); + assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expected); + assertThat(hashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expected); + assertThat(hashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedIdentical); + } } - // verify output block matches - assertBlockEquals(VARCHAR, pageBuilder.build().getBlock(0), leftBlock); + // write position to output block + pageBuilder.declarePosition(); + hashStrategy.appendTo(leftBlockIndex, leftBlockPosition, pageBuilder, 0); } + + // verify output block matches + assertBlockEquals(VARCHAR, pageBuilder.build().getBlock(0), leftBlock); } } @Test public void testMultiChannel() { - for (boolean hashEnabled : Arrays.asList(true, false)) { - // compile a single channel hash strategy - List types = ImmutableList.of(VARCHAR, VARCHAR, BIGINT, DOUBLE, BOOLEAN, VARCHAR); - List joinTypes = ImmutableList.of(VARCHAR, BIGINT, DOUBLE, BOOLEAN); - List outputTypes = ImmutableList.of(VARCHAR, BIGINT, DOUBLE, BOOLEAN, VARCHAR); - List joinChannels = Ints.asList(1, 2, 3, 4); - List outputChannels = Ints.asList(1, 2, 3, 4, 0); - - // crate hash strategy with a single channel blocks -- make sure there is some overlap in values - ObjectArrayList extraChannel = new ObjectArrayList<>(); - extraChannel.add(BlockAssertions.createStringSequenceBlock(10, 20)); - extraChannel.add(BlockAssertions.createStringSequenceBlock(20, 30)); - extraChannel.add(BlockAssertions.createStringSequenceBlock(15, 25)); - ObjectArrayList varcharChannel = new ObjectArrayList<>(); - varcharChannel.add(BlockAssertions.createStringSequenceBlock(10, 20)); - varcharChannel.add(BlockAssertions.createStringSequenceBlock(20, 30)); - varcharChannel.add(BlockAssertions.createStringSequenceBlock(15, 25)); - ObjectArrayList longChannel = new ObjectArrayList<>(); - longChannel.add(BlockAssertions.createLongSequenceBlock(10, 20)); - longChannel.add(BlockAssertions.createLongSequenceBlock(20, 30)); - longChannel.add(BlockAssertions.createLongSequenceBlock(15, 25)); - ObjectArrayList doubleChannel = new ObjectArrayList<>(); - doubleChannel.add(BlockAssertions.createDoubleSequenceBlock(10, 20)); - doubleChannel.add(BlockAssertions.createDoubleSequenceBlock(20, 30)); - doubleChannel.add(BlockAssertions.createDoubleSequenceBlock(15, 25)); - ObjectArrayList booleanChannel = new ObjectArrayList<>(); - booleanChannel.add(BlockAssertions.createBooleanSequenceBlock(10, 20)); - booleanChannel.add(BlockAssertions.createBooleanSequenceBlock(20, 30)); - booleanChannel.add(BlockAssertions.createBooleanSequenceBlock(15, 25)); - ObjectArrayList extraUnusedChannel = new ObjectArrayList<>(); - extraUnusedChannel.add(BlockAssertions.createBooleanSequenceBlock(10, 20)); - extraUnusedChannel.add(BlockAssertions.createBooleanSequenceBlock(20, 30)); - extraUnusedChannel.add(BlockAssertions.createBooleanSequenceBlock(15, 25)); - - OptionalInt hashChannel = OptionalInt.empty(); - List> channels = ImmutableList.of(extraChannel, varcharChannel, longChannel, doubleChannel, booleanChannel, extraUnusedChannel); - ObjectArrayList precomputedHash = new ObjectArrayList<>(); - if (hashEnabled) { - for (int i = 0; i < 3; i++) { - precomputedHash.add(TypeTestUtils.getHashBlock(joinTypes, varcharChannel.get(i), longChannel.get(i), doubleChannel.get(i), booleanChannel.get(i))); - } - hashChannel = OptionalInt.of(6); - channels = ImmutableList.of(extraChannel, varcharChannel, longChannel, doubleChannel, booleanChannel, extraUnusedChannel, precomputedHash); - types = ImmutableList.of(VARCHAR, VARCHAR, BIGINT, DOUBLE, BOOLEAN, VARCHAR, BIGINT); - outputTypes = ImmutableList.of(VARCHAR, BIGINT, DOUBLE, BOOLEAN, VARCHAR, BIGINT); - outputChannels = Ints.asList(1, 2, 3, 4, 0, 6); - } + // compile a single channel hash strategy + List types = ImmutableList.of(VARCHAR, VARCHAR, BIGINT, DOUBLE, BOOLEAN, VARCHAR); + List outputTypes = ImmutableList.of(VARCHAR, BIGINT, DOUBLE, BOOLEAN, VARCHAR); + List joinChannels = Ints.asList(1, 2, 3, 4); + List outputChannels = Ints.asList(1, 2, 3, 4, 0); + + // crate hash strategy with a single channel blocks -- make sure there is some overlap in values + ObjectArrayList extraChannel = new ObjectArrayList<>(); + extraChannel.add(BlockAssertions.createStringSequenceBlock(10, 20)); + extraChannel.add(BlockAssertions.createStringSequenceBlock(20, 30)); + extraChannel.add(BlockAssertions.createStringSequenceBlock(15, 25)); + ObjectArrayList varcharChannel = new ObjectArrayList<>(); + varcharChannel.add(BlockAssertions.createStringSequenceBlock(10, 20)); + varcharChannel.add(BlockAssertions.createStringSequenceBlock(20, 30)); + varcharChannel.add(BlockAssertions.createStringSequenceBlock(15, 25)); + ObjectArrayList longChannel = new ObjectArrayList<>(); + longChannel.add(BlockAssertions.createLongSequenceBlock(10, 20)); + longChannel.add(BlockAssertions.createLongSequenceBlock(20, 30)); + longChannel.add(BlockAssertions.createLongSequenceBlock(15, 25)); + ObjectArrayList doubleChannel = new ObjectArrayList<>(); + doubleChannel.add(BlockAssertions.createDoubleSequenceBlock(10, 20)); + doubleChannel.add(BlockAssertions.createDoubleSequenceBlock(20, 30)); + doubleChannel.add(BlockAssertions.createDoubleSequenceBlock(15, 25)); + ObjectArrayList booleanChannel = new ObjectArrayList<>(); + booleanChannel.add(BlockAssertions.createBooleanSequenceBlock(10, 20)); + booleanChannel.add(BlockAssertions.createBooleanSequenceBlock(20, 30)); + booleanChannel.add(BlockAssertions.createBooleanSequenceBlock(15, 25)); + ObjectArrayList extraUnusedChannel = new ObjectArrayList<>(); + extraUnusedChannel.add(BlockAssertions.createBooleanSequenceBlock(10, 20)); + extraUnusedChannel.add(BlockAssertions.createBooleanSequenceBlock(20, 30)); + extraUnusedChannel.add(BlockAssertions.createBooleanSequenceBlock(15, 25)); + + List> channels = ImmutableList.of(extraChannel, varcharChannel, longChannel, doubleChannel, booleanChannel, extraUnusedChannel); + PagesHashStrategyFactory pagesHashStrategyFactory = joinCompiler.compilePagesHashStrategyFactory(types, joinChannels, Optional.of(outputChannels)); + PagesHashStrategy hashStrategy = pagesHashStrategyFactory.createPagesHashStrategy(channels); + // todo add tests for filter function + PagesHashStrategy expectedHashStrategy = new SimplePagesHashStrategy(types, outputChannels, channels, joinChannels, Optional.empty(), blockTypeOperators); - PagesHashStrategyFactory pagesHashStrategyFactory = joinCompiler.compilePagesHashStrategyFactory(types, joinChannels, Optional.of(outputChannels)); - PagesHashStrategy hashStrategy = pagesHashStrategyFactory.createPagesHashStrategy(channels, hashChannel); - // todo add tests for filter function - PagesHashStrategy expectedHashStrategy = new SimplePagesHashStrategy(types, outputChannels, channels, joinChannels, hashChannel, Optional.empty(), blockTypeOperators); - - // verify channel count - assertThat(hashStrategy.getChannelCount()).isEqualTo(outputChannels.size()); - // verify size - int instanceSize = instanceSize(hashStrategy.getClass()); - long sizeInBytes = instanceSize + - (channels.size() > 0 ? sizeOf(channels.get(0).elements()) * channels.size() : 0) + - channels.stream() - .flatMap(List::stream) - .mapToLong(Block::getRetainedSizeInBytes) - .sum(); - assertThat(hashStrategy.getSizeInBytes()).isEqualTo(sizeInBytes); - - // verify hashStrategy is consistent with equals and hash code from block - for (int leftBlockIndex = 0; leftBlockIndex < varcharChannel.size(); leftBlockIndex++) { - PageBuilder pageBuilder = new PageBuilder(outputTypes); - - Block[] leftBlocks = new Block[4]; - leftBlocks[0] = varcharChannel.get(leftBlockIndex); - leftBlocks[1] = longChannel.get(leftBlockIndex); - leftBlocks[2] = doubleChannel.get(leftBlockIndex); - leftBlocks[3] = booleanChannel.get(leftBlockIndex); - - int leftPositionCount = varcharChannel.get(leftBlockIndex).getPositionCount(); - for (int leftBlockPosition = 0; leftBlockPosition < leftPositionCount; leftBlockPosition++) { - // hash code of position must match block hash - assertThat(hashStrategy.hashPosition(leftBlockIndex, leftBlockPosition)).isEqualTo(expectedHashStrategy.hashPosition(leftBlockIndex, leftBlockPosition)); - - // position must be equal to itself - assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, leftBlockIndex, leftBlockPosition)).isTrue(); - assertThat(hashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, leftBlockIndex, leftBlockPosition)).isTrue(); - assertThat(hashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, leftBlockIndex, leftBlockPosition)).isTrue(); - - // check equality of every position against every other position in the block - for (int rightBlockIndex = 0; rightBlockIndex < varcharChannel.size(); rightBlockIndex++) { - Block rightBlock = varcharChannel.get(rightBlockIndex); - for (int rightBlockPosition = 0; rightBlockPosition < rightBlock.getPositionCount(); rightBlockPosition++) { - assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedHashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)); - assertThat(hashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedHashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)); - assertThat(hashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedHashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)); - } - } + // verify channel count + assertThat(hashStrategy.getChannelCount()).isEqualTo(outputChannels.size()); + // verify size + int instanceSize = instanceSize(hashStrategy.getClass()); + long sizeInBytes = instanceSize + + (channels.size() > 0 ? sizeOf(channels.get(0).elements()) * channels.size() : 0) + + channels.stream() + .flatMap(List::stream) + .mapToLong(Block::getRetainedSizeInBytes) + .sum(); + assertThat(hashStrategy.getSizeInBytes()).isEqualTo(sizeInBytes); + + // verify hashStrategy is consistent with equals and hash code from block + for (int leftBlockIndex = 0; leftBlockIndex < varcharChannel.size(); leftBlockIndex++) { + PageBuilder pageBuilder = new PageBuilder(outputTypes); + + Block[] leftBlocks = new Block[4]; + leftBlocks[0] = varcharChannel.get(leftBlockIndex); + leftBlocks[1] = longChannel.get(leftBlockIndex); + leftBlocks[2] = doubleChannel.get(leftBlockIndex); + leftBlocks[3] = booleanChannel.get(leftBlockIndex); + + int leftPositionCount = varcharChannel.get(leftBlockIndex).getPositionCount(); + for (int leftBlockPosition = 0; leftBlockPosition < leftPositionCount; leftBlockPosition++) { + // hash code of position must match block hash + assertThat(hashStrategy.hashPosition(leftBlockIndex, leftBlockPosition)).isEqualTo(expectedHashStrategy.hashPosition(leftBlockIndex, leftBlockPosition)); + + // position must be equal to itself + assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, leftBlockIndex, leftBlockPosition)).isTrue(); + assertThat(hashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, leftBlockIndex, leftBlockPosition)).isTrue(); + assertThat(hashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, leftBlockIndex, leftBlockPosition)).isTrue(); - // check equality of every position against every other position in the block cursor - for (int rightBlockIndex = 0; rightBlockIndex < varcharChannel.size(); rightBlockIndex++) { - Block[] rightBlocks = new Block[4]; - rightBlocks[0] = varcharChannel.get(rightBlockIndex); - rightBlocks[1] = longChannel.get(rightBlockIndex); - rightBlocks[2] = doubleChannel.get(rightBlockIndex); - rightBlocks[3] = booleanChannel.get(rightBlockIndex); - - int rightPositionCount = varcharChannel.get(rightBlockIndex).getPositionCount(); - for (int rightPosition = 0; rightPosition < rightPositionCount; rightPosition++) { - boolean expected = expectedHashStrategy.positionEqualsRow(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks)); - boolean expectedIdentical = expectedHashStrategy.positionIdenticalToRow(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks)); - - assertThat(hashStrategy.positionEqualsRow(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks))).isEqualTo(expected); - assertThat(hashStrategy.positionIdenticalToRow(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks))).isEqualTo(expectedIdentical); - assertThat(hashStrategy.rowEqualsRow(leftBlockPosition, new Page(leftBlocks), rightPosition, new Page(rightBlocks))).isEqualTo(expected); - assertThat(hashStrategy.rowIdenticalToRow(leftBlockPosition, new Page(leftBlocks), rightPosition, new Page(rightBlocks))).isEqualTo(expectedIdentical); - assertThat(hashStrategy.positionEqualsRowIgnoreNulls(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks))).isEqualTo(expected); - } + // check equality of every position against every other position in the block + for (int rightBlockIndex = 0; rightBlockIndex < varcharChannel.size(); rightBlockIndex++) { + Block rightBlock = varcharChannel.get(rightBlockIndex); + for (int rightBlockPosition = 0; rightBlockPosition < rightBlock.getPositionCount(); rightBlockPosition++) { + assertThat(hashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedHashStrategy.positionEqualsPositionIgnoreNulls(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)); + assertThat(hashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedHashStrategy.positionEqualsPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)); + assertThat(hashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)).isEqualTo(expectedHashStrategy.positionIdenticalToPosition(leftBlockIndex, leftBlockPosition, rightBlockIndex, rightBlockPosition)); } - - // write position to output block - pageBuilder.declarePosition(); - hashStrategy.appendTo(leftBlockIndex, leftBlockPosition, pageBuilder, 0); } - // verify output block matches - Page page = pageBuilder.build(); - if (hashEnabled) { - assertPageEquals(outputTypes, page, new Page( - varcharChannel.get(leftBlockIndex), - longChannel.get(leftBlockIndex), - doubleChannel.get(leftBlockIndex), - booleanChannel.get(leftBlockIndex), - extraChannel.get(leftBlockIndex), - precomputedHash.get(leftBlockIndex))); - } - else { - assertPageEquals(outputTypes, page, new Page( - varcharChannel.get(leftBlockIndex), - longChannel.get(leftBlockIndex), - doubleChannel.get(leftBlockIndex), - booleanChannel.get(leftBlockIndex), - extraChannel.get(leftBlockIndex))); + // check equality of every position against every other position in the block cursor + for (int rightBlockIndex = 0; rightBlockIndex < varcharChannel.size(); rightBlockIndex++) { + Block[] rightBlocks = new Block[4]; + rightBlocks[0] = varcharChannel.get(rightBlockIndex); + rightBlocks[1] = longChannel.get(rightBlockIndex); + rightBlocks[2] = doubleChannel.get(rightBlockIndex); + rightBlocks[3] = booleanChannel.get(rightBlockIndex); + + int rightPositionCount = varcharChannel.get(rightBlockIndex).getPositionCount(); + for (int rightPosition = 0; rightPosition < rightPositionCount; rightPosition++) { + boolean expected = expectedHashStrategy.positionEqualsRow(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks)); + boolean expectedIdentical = expectedHashStrategy.positionIdenticalToRow(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks)); + + assertThat(hashStrategy.positionEqualsRow(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks))).isEqualTo(expected); + assertThat(hashStrategy.positionIdenticalToRow(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks))).isEqualTo(expectedIdentical); + assertThat(hashStrategy.rowEqualsRow(leftBlockPosition, new Page(leftBlocks), rightPosition, new Page(rightBlocks))).isEqualTo(expected); + assertThat(hashStrategy.rowIdenticalToRow(leftBlockPosition, new Page(leftBlocks), rightPosition, new Page(rightBlocks))).isEqualTo(expectedIdentical); + assertThat(hashStrategy.positionEqualsRowIgnoreNulls(leftBlockIndex, leftBlockPosition, rightPosition, new Page(rightBlocks))).isEqualTo(expected); + } } + + // write position to output block + pageBuilder.declarePosition(); + hashStrategy.appendTo(leftBlockIndex, leftBlockPosition, pageBuilder, 0); } + + // verify output block matches + Page page = pageBuilder.build(); + assertPageEquals(outputTypes, page, new Page( + varcharChannel.get(leftBlockIndex), + longChannel.get(leftBlockIndex), + doubleChannel.get(leftBlockIndex), + booleanChannel.get(leftBlockIndex), + extraChannel.get(leftBlockIndex))); } } @@ -310,7 +269,7 @@ public void testIdentical() channel.add(BlockAssertions.createDoubleSequenceBlock(20, 30)); channel.add(BlockAssertions.createDoubleSequenceBlock(15, 25)); - PagesHashStrategy hashStrategy = pagesHashStrategyFactory.createPagesHashStrategy(ImmutableList.of(channel), OptionalInt.empty()); + PagesHashStrategy hashStrategy = pagesHashStrategyFactory.createPagesHashStrategy(ImmutableList.of(channel)); // verify channel count assertThat(hashStrategy.getChannelCount()).isEqualTo(1); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java index 47645399afdd..50b8ece37f2b 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestEffectivePredicateExtractor.java @@ -748,8 +748,6 @@ public void testInnerJoin() Optional.of(lessThanOrEqual(new Reference(BIGINT, "b"), new Reference(BIGINT, "e"))), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); @@ -790,8 +788,6 @@ public void testInnerJoinPropagatesPredicatesViaEquiConditions() Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); @@ -821,8 +817,6 @@ public void testInnerJoinWithFalseFilter() Optional.of(FALSE), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); @@ -859,8 +853,6 @@ public void testLeftJoin() Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); @@ -906,8 +898,6 @@ public void testLeftJoinWithFalseInner() Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); @@ -957,8 +947,6 @@ public void testRightJoin() Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); @@ -1003,8 +991,6 @@ public void testRightJoinWithFalseInner() Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); @@ -1026,8 +1012,6 @@ public void testSemiJoin() filter(baseTableScan, greaterThan(new Reference(BIGINT, "a"), bigintLiteral(5))), new Symbol(BIGINT, "a"), new Symbol(BIGINT, "b"), new Symbol(DOUBLE, "c"), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.empty()); Expression effectivePredicate = effectivePredicateExtractor.extract(SESSION, node); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestIncrementalRefreshVisitor.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestIncrementalRefreshVisitor.java index cf3a53149931..50287946e9d2 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestIncrementalRefreshVisitor.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestIncrementalRefreshVisitor.java @@ -86,8 +86,6 @@ public void testFullRefreshForJoinNode() Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestLocalDynamicFilterConsumer.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestLocalDynamicFilterConsumer.java index d6cae19a06d7..0372c7db023a 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestLocalDynamicFilterConsumer.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestLocalDynamicFilterConsumer.java @@ -247,8 +247,6 @@ public void testDynamicFilterPruning() ImmutableList.of(), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(filter1, right1, filter2, right2, filter3, right3)); TestingDynamicFilterCollector collector = new TestingDynamicFilterCollector(); LocalDynamicFilterConsumer consumer = LocalDynamicFilterConsumer.create( diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestSchedulingOrderVisitor.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestSchedulingOrderVisitor.java index de836c8e8c1e..203c78169bd0 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestSchedulingOrderVisitor.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestSchedulingOrderVisitor.java @@ -25,7 +25,6 @@ import org.junit.jupiter.api.Test; import java.util.List; -import java.util.Optional; import static io.trino.SessionTestUtils.TEST_SESSION; import static io.trino.sql.planner.SchedulingOrderVisitor.scheduleOrder; @@ -68,8 +67,6 @@ public void testSemiJoinOrder() sourceJoin, filteringSource, planBuilder.symbol("semiJoinOutput"), - Optional.empty(), - Optional.empty(), a, b)); assertThat(order).isEqualTo(ImmutableList.of(b.getId(), a.getId())); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestTopologicalOrderSubPlanVisitor.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestTopologicalOrderSubPlanVisitor.java index 48bfef1503f4..a1f1cc071e3c 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestTopologicalOrderSubPlanVisitor.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestTopologicalOrderSubPlanVisitor.java @@ -130,8 +130,6 @@ private static JoinNode join(String id, PlanNode left, PlanNode right) Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); } @@ -146,8 +144,6 @@ private static SemiJoinNode semiJoin(String id, PlanNode left, PlanNode right) right.getOutputSymbols().get(0), new Symbol(UNKNOWN, id), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.empty()); } @@ -158,9 +154,7 @@ private static IndexJoinNode indexJoin(String id, PlanNode left, PlanNode right) IndexJoinNode.Type.INNER, left, right, - ImmutableList.of(), - Optional.empty(), - Optional.empty()); + ImmutableList.of()); } private static SpatialJoinNode spatialJoin(String id, PlanNode left, PlanNode right) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/IndexJoinMatcher.java b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/IndexJoinMatcher.java index 0cde20fa0760..bbb86ba2512f 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/IndexJoinMatcher.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/IndexJoinMatcher.java @@ -21,7 +21,6 @@ import io.trino.sql.planner.plan.PlanNode; import java.util.List; -import java.util.Optional; import java.util.Set; import static com.google.common.base.MoreObjects.toStringHelper; @@ -35,19 +34,13 @@ final class IndexJoinMatcher { private final IndexJoinNode.Type type; private final List> criteria; - private final Optional probeHashSymbol; - private final Optional indexHashSymbol; IndexJoinMatcher( IndexJoinNode.Type type, - List> criteria, - Optional probeHashSymbol, - Optional indexHashSymbol) + List> criteria) { this.type = requireNonNull(type, "type is null"); this.criteria = requireNonNull(criteria, "criteria is null"); - this.probeHashSymbol = requireNonNull(probeHashSymbol, "probeHashSymbol is null"); - this.indexHashSymbol = requireNonNull(indexHashSymbol, "indexHashSymbol is null"); } @Override @@ -77,14 +70,6 @@ public MatchResult detailMatches(PlanNode node, StatsProvider stats, Session ses return NO_MATCH; } - if (!indexJoinNode.getProbeHashSymbol().equals(probeHashSymbol.map(alias -> alias.toSymbol(symbolAliases)))) { - return NO_MATCH; - } - - if (!indexJoinNode.getIndexHashSymbol().equals(indexHashSymbol.map(alias -> alias.toSymbol(symbolAliases)))) { - return NO_MATCH; - } - return MatchResult.match(); } @@ -95,8 +80,6 @@ public String toString() .omitNullValues() .add("type", type) .add("criteria", criteria) - .add("probeHashSymbol", probeHashSymbol.orElse(null)) - .add("indexHashSymbol", indexHashSymbol.orElse(null)) .toString(); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/PlanMatchPattern.java b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/PlanMatchPattern.java index 780fedd205f9..03e0120a3889 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/PlanMatchPattern.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/PlanMatchPattern.java @@ -248,13 +248,11 @@ public static PlanMatchPattern constrainedTableScanWithTableLayout(String expect public static PlanMatchPattern indexJoin( IndexJoinNode.Type type, List> criteria, - Optional probeHashSymbol, - Optional indexHashSymbol, PlanMatchPattern probeSource, PlanMatchPattern indexSource) { return node(IndexJoinNode.class, probeSource, indexSource) - .with(new IndexJoinMatcher(type, criteria, probeHashSymbol.map(SymbolAlias::new), indexHashSymbol.map(SymbolAlias::new))); + .with(new IndexJoinMatcher(type, criteria)); } public static ExpectedValueProvider indexJoinEquiClause(String probe, String index) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestDetermineJoinDistributionType.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestDetermineJoinDistributionType.java index 3faf2ee294f6..dce9c6f20109 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestDetermineJoinDistributionType.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestDetermineJoinDistributionType.java @@ -237,8 +237,6 @@ public void testRetainDistributionType() ImmutableList.of(p.symbol("A1", BIGINT)), ImmutableList.of(p.symbol("B1", BIGINT)), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(DistributionType.REPLICATED), ImmutableMap.of())) .doesNotFire(); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestDetermineSemiJoinDistributionType.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestDetermineSemiJoinDistributionType.java index 513371f39087..97f88ba5edb1 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestDetermineSemiJoinDistributionType.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestDetermineSemiJoinDistributionType.java @@ -84,8 +84,6 @@ public void testRetainDistributionType() p.symbol("A1"), p.symbol("B1"), p.symbol("output"), - Optional.empty(), - Optional.empty(), Optional.of(REPLICATED))) .doesNotFire(); } @@ -115,8 +113,6 @@ public void testPartitionWhenRequiredBySession() a1, b1, p.symbol("output"), - Optional.empty(), - Optional.empty(), Optional.empty()); }) .matches(semiJoin( @@ -151,8 +147,6 @@ public void testReplicatesWhenRequiredBySession() p.symbol("A1"), p.symbol("B1"), p.symbol("output"), - Optional.empty(), - Optional.empty(), Optional.empty())) .matches(semiJoin( "A1", @@ -185,8 +179,6 @@ public void testPartitionsWhenBothTablesEqual() p.symbol("A1"), p.symbol("B1"), p.symbol("output"), - Optional.empty(), - Optional.empty(), Optional.empty())) .matches(semiJoin( "A1", @@ -219,8 +211,6 @@ public void testReplicatesWhenFilterMuchSmaller() p.symbol("A1"), p.symbol("B1"), p.symbol("output"), - Optional.empty(), - Optional.empty(), Optional.empty())) .matches(semiJoin( "A1", @@ -262,8 +252,6 @@ public void testReplicatesWhenNotRestricted() a1, b1, p.symbol("output"), - Optional.empty(), - Optional.empty(), Optional.empty()); }) .matches(semiJoin( @@ -298,8 +286,6 @@ public void testReplicatesWhenNotRestricted() a1, b1, p.symbol("output", BIGINT), - Optional.empty(), - Optional.empty(), Optional.empty()); }) .matches(semiJoin( @@ -351,8 +337,6 @@ public void testReplicatesWhenSourceIsSmall() a1, b1, p.symbol("output"), - Optional.empty(), - Optional.empty(), Optional.empty()); }) .matches(semiJoin( diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestEliminateCrossJoins.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestEliminateCrossJoins.java index 191f911b5700..ecd9380de30f 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestEliminateCrossJoins.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestEliminateCrossJoins.java @@ -341,8 +341,6 @@ private JoinNode joinNode(PlanNode left, PlanNode right, String... symbols) Optional.empty(), Optional.empty(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(), Optional.empty()); } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneIndexJoinColumns.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneIndexJoinColumns.java index 1322f177fbd7..4a99c8a48510 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneIndexJoinColumns.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneIndexJoinColumns.java @@ -22,8 +22,6 @@ import io.trino.sql.planner.plan.IndexJoinNode.EquiJoinClause; import org.junit.jupiter.api.Test; -import java.util.Optional; - import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.sql.planner.assertions.PlanMatchPattern.expression; import static io.trino.sql.planner.assertions.PlanMatchPattern.indexJoin; @@ -49,9 +47,7 @@ public void testPruneInputColumn() INNER, p.values(a), p.values(b, c), - ImmutableList.of(new EquiJoinClause(a, b)), - Optional.empty(), - Optional.empty())); + ImmutableList.of(new EquiJoinClause(a, b)))); }) .matches( strictProject( @@ -59,8 +55,6 @@ public void testPruneInputColumn() indexJoin( INNER, ImmutableList.of(indexJoinEquiClause("a", "b")), - Optional.empty(), - Optional.empty(), values("a"), strictProject( ImmutableMap.of("b", expression(new Reference(BIGINT, "b"))), @@ -78,9 +72,7 @@ public void testPruneInputColumn() INNER, p.values(a, b), p.values(c, d), - ImmutableList.of(new EquiJoinClause(a, c)), - Optional.empty(), - Optional.empty())); + ImmutableList.of(new EquiJoinClause(a, c)))); }) .matches( strictProject( @@ -88,8 +80,6 @@ public void testPruneInputColumn() indexJoin( INNER, ImmutableList.of(indexJoinEquiClause("a", "c")), - Optional.empty(), - Optional.empty(), strictProject( ImmutableMap.of("a", expression(new Reference(BIGINT, "a"))), values("a", "b")), @@ -111,30 +101,7 @@ public void testDoNotPruneEquiClauseSymbol() INNER, p.values(a), p.values(b), - ImmutableList.of(new EquiJoinClause(a, b)), - Optional.empty(), - Optional.empty())); - }) - .doesNotFire(); - } - - @Test - public void testDoNotPruneHashSymbol() - { - tester().assertThat(new PruneIndexJoinColumns()) - .on(p -> { - Symbol a = p.symbol("a"); - Symbol b = p.symbol("b"); - Symbol h = p.symbol("h"); - return p.project( - Assignments.identity(a, b), - p.indexJoin( - INNER, - p.values(a), - p.values(b, h), - ImmutableList.of(new EquiJoinClause(a, b)), - Optional.empty(), - Optional.of(h))); + ImmutableList.of(new EquiJoinClause(a, b)))); }) .doesNotFire(); } @@ -153,9 +120,7 @@ public void testAllOutputsReferenced() INNER, p.values(a), p.values(b, c), - ImmutableList.of(new EquiJoinClause(a, b)), - Optional.empty(), - Optional.empty())); + ImmutableList.of(new EquiJoinClause(a, b)))); }) .doesNotFire(); } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneJoinChildrenColumns.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneJoinChildrenColumns.java index 8ac96d7a1b3b..a8c4b5727534 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneJoinChildrenColumns.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneJoinChildrenColumns.java @@ -51,13 +51,12 @@ public void testNotAllInputsReferenced() join(INNER, builder -> builder .equiCriteria("leftKey", "rightKey") .filter(new Comparison(GREATER_THAN, new Reference(BIGINT, "leftValue"), new Constant(BIGINT, 5L))) - .left(values("leftKey", "leftKeyHash", "leftValue")) + .left(values("leftKey", "leftValue")) .right( strictProject( ImmutableMap.of( - "rightKey", PlanMatchPattern.expression(new Reference(BIGINT, "rightKey")), - "rightKeyHash", PlanMatchPattern.expression(new Reference(BIGINT, "rightKeyHash"))), - values("rightKey", "rightKeyHash", "rightValue"))))); + "rightKey", PlanMatchPattern.expression(new Reference(BIGINT, "rightKey"))), + values("rightKey", "rightValue"))))); } @Test @@ -82,8 +81,6 @@ public void testCrossJoin() ImmutableList.of(), ImmutableList.of(leftValue), ImmutableList.of(), - Optional.empty(), - Optional.empty(), Optional.empty()); }) .matches( @@ -98,17 +95,15 @@ public void testCrossJoin() private static PlanNode buildJoin(PlanBuilder p, Predicate joinOutputFilter) { Symbol leftKey = p.symbol("leftKey", BIGINT); - Symbol leftKeyHash = p.symbol("leftKeyHash", BIGINT); Symbol leftValue = p.symbol("leftValue", BIGINT); Symbol rightKey = p.symbol("rightKey", BIGINT); - Symbol rightKeyHash = p.symbol("rightKeyHash", BIGINT); Symbol rightValue = p.symbol("rightValue", BIGINT); List leftOutputs = ImmutableList.of(leftValue); List rightOutputs = ImmutableList.of(rightValue); return p.join( INNER, - p.values(leftKey, leftKeyHash, leftValue), - p.values(rightKey, rightKeyHash, rightValue), + p.values(leftKey, leftValue), + p.values(rightKey, rightValue), ImmutableList.of(new JoinNode.EquiJoinClause(leftKey, rightKey)), leftOutputs.stream() .filter(joinOutputFilter) @@ -116,8 +111,6 @@ private static PlanNode buildJoin(PlanBuilder p, Predicate joinOutputFil rightOutputs.stream() .filter(joinOutputFilter) .collect(toImmutableList()), - Optional.of(new Comparison(GREATER_THAN, new Reference(BIGINT, "leftValue"), new Constant(BIGINT, 5L))), - Optional.of(leftKeyHash), - Optional.of(rightKeyHash)); + Optional.of(new Comparison(GREATER_THAN, new Reference(BIGINT, "leftValue"), new Constant(BIGINT, 5L)))); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneJoinColumns.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneJoinColumns.java index 4b6843fac492..fb750c5dc564 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneJoinColumns.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneJoinColumns.java @@ -79,8 +79,6 @@ public void testCrossJoin() ImmutableList.of(), ImmutableList.of(leftValue), ImmutableList.of(rightValue), - Optional.empty(), - Optional.empty(), Optional.empty())); }) .matches( @@ -112,8 +110,6 @@ private static PlanNode buildProjectedJoin(PlanBuilder p, Predicate proj ImmutableList.of(new JoinNode.EquiJoinClause(leftKey, rightKey)), leftOutputs, rightOutputs, - Optional.empty(), - Optional.empty(), Optional.empty())); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneSemiJoinColumns.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneSemiJoinColumns.java index 95c18391b8cf..6b5c6472bafe 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneSemiJoinColumns.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneSemiJoinColumns.java @@ -24,7 +24,6 @@ import org.junit.jupiter.api.Test; import java.util.List; -import java.util.Optional; import java.util.function.Predicate; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -46,7 +45,7 @@ public void testSemiJoinNotNeeded() .matches( strictProject( ImmutableMap.of("leftValue", expression(new Reference(BIGINT, "leftValue"))), - values("leftKey", "leftKeyHash", "leftValue"))); + values("leftKey", "leftValue"))); } @Test @@ -76,9 +75,8 @@ public void testValueNotNeeded() semiJoin("leftKey", "rightKey", "match", strictProject( ImmutableMap.of( - "leftKey", expression(new Reference(BIGINT, "leftKey")), - "leftKeyHash", expression(new Reference(BIGINT, "leftKeyHash"))), - values("leftKey", "leftKeyHash", "leftValue")), + "leftKey", expression(new Reference(BIGINT, "leftKey"))), + values("leftKey", "leftValue")), values("rightKey")))); } @@ -86,10 +84,9 @@ private static PlanNode buildProjectedSemiJoin(PlanBuilder p, Predicate { Symbol match = p.symbol("match"); Symbol leftKey = p.symbol("leftKey"); - Symbol leftKeyHash = p.symbol("leftKeyHash"); Symbol leftValue = p.symbol("leftValue"); Symbol rightKey = p.symbol("rightKey"); - List outputs = ImmutableList.of(match, leftKey, leftKeyHash, leftValue); + List outputs = ImmutableList.of(match, leftKey, leftValue); return p.project( Assignments.identity( outputs.stream() @@ -99,9 +96,7 @@ private static PlanNode buildProjectedSemiJoin(PlanBuilder p, Predicate leftKey, rightKey, match, - Optional.of(leftKeyHash), - Optional.empty(), - p.values(leftKey, leftKeyHash, leftValue), + p.values(leftKey, leftValue), p.values(rightKey))); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneSemiJoinFilteringSourceColumns.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneSemiJoinFilteringSourceColumns.java index 8bd40868ef61..8bf0d094166c 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneSemiJoinFilteringSourceColumns.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPruneSemiJoinFilteringSourceColumns.java @@ -23,7 +23,6 @@ import org.junit.jupiter.api.Test; import java.util.List; -import java.util.Optional; import java.util.function.Predicate; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -46,9 +45,8 @@ public void testNotAllColumnsReferenced() values("leftKey"), strictProject( ImmutableMap.of( - "rightKey", expression(new Reference(BIGINT, "rightKey")), - "rightKeyHash", expression(new Reference(BIGINT, "rightKeyHash"))), - values("rightKey", "rightKeyHash", "rightValue")))); + "rightKey", expression(new Reference(BIGINT, "rightKey"))), + values("rightKey", "rightValue")))); } @Test @@ -64,15 +62,12 @@ private static PlanNode buildSemiJoin(PlanBuilder p, Predicate filtering Symbol match = p.symbol("match"); Symbol leftKey = p.symbol("leftKey"); Symbol rightKey = p.symbol("rightKey"); - Symbol rightKeyHash = p.symbol("rightKeyHash"); Symbol rightValue = p.symbol("rightValue"); - List filteringSourceSymbols = ImmutableList.of(rightKey, rightKeyHash, rightValue); + List filteringSourceSymbols = ImmutableList.of(rightKey, rightValue); return p.semiJoin( leftKey, rightKey, match, - Optional.empty(), - Optional.of(rightKeyHash), p.values(leftKey), p.values( filteringSourceSymbols.stream() diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushAggregationThroughOuterJoin.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushAggregationThroughOuterJoin.java index bc1357e46fa4..e0e8a3279fc7 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushAggregationThroughOuterJoin.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushAggregationThroughOuterJoin.java @@ -63,8 +63,6 @@ public void testPushesAggregationThroughLeftJoin() ImmutableList.of(new EquiJoinClause(p.symbol("COL1", BIGINT), p.symbol("COL2", BIGINT))), ImmutableList.of(p.symbol("COL1", BIGINT)), ImmutableList.of(p.symbol("COL2", BIGINT)), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(p.symbol("AVG", DOUBLE), PlanBuilder.aggregation("avg", ImmutableList.of(new Reference(BIGINT, "COL2"))), ImmutableList.of(BIGINT)) .singleGroupingSet(p.symbol("COL1", BIGINT)))) @@ -104,8 +102,6 @@ public void testPushesAggregationThroughRightJoin() ImmutableList.of(new EquiJoinClause(p.symbol("COL2", BIGINT), p.symbol("COL1", BIGINT))), ImmutableList.of(p.symbol("COL2", BIGINT)), ImmutableList.of(p.symbol("COL1", BIGINT)), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(p.symbol("AVG", DOUBLE), PlanBuilder.aggregation("avg", ImmutableList.of(new Reference(BIGINT, "COL2"))), ImmutableList.of(BIGINT)) .singleGroupingSet(p.symbol("COL1", BIGINT)))) @@ -147,8 +143,6 @@ public void testPushesAggregationWithMask() ImmutableList.of(new EquiJoinClause(p.symbol("COL1", BIGINT), p.symbol("COL2", BIGINT))), ImmutableList.of(p.symbol("COL1", BIGINT)), ImmutableList.of(p.symbol("COL2", BIGINT), p.symbol("MASK", BOOLEAN)), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation( p.symbol("AVG", DOUBLE), @@ -198,8 +192,6 @@ public void testPushCountAllAggregation() ImmutableList.of(new EquiJoinClause(p.symbol("COL1", BIGINT), p.symbol("COL2", BIGINT))), ImmutableList.of(p.symbol("COL1", BIGINT)), ImmutableList.of(p.symbol("COL2", BIGINT)), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(p.symbol("COUNT", BIGINT), PlanBuilder.aggregation("count", ImmutableList.of()), ImmutableList.of()) .singleGroupingSet(p.symbol("COL1", BIGINT)))) @@ -245,8 +237,6 @@ public void testDoesNotFireWhenMultipleGroupingSets() ImmutableList.of(new EquiJoinClause(p.symbol("COL1"), p.symbol("COL3"))), ImmutableList.of(p.symbol("COL1")), ImmutableList.of(p.symbol("COL3")), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(p.symbol("COUNT"), PlanBuilder.aggregation("count", ImmutableList.of()), ImmutableList.of()) .groupingSets(groupingSets(ImmutableList.of(p.symbol("COL1"), p.symbol("COL2")), 2, ImmutableSet.of())))) @@ -265,8 +255,6 @@ public void testDoesNotFireWhenNotDistinct() ImmutableList.of(new EquiJoinClause(new Symbol(BIGINT, "COL1"), new Symbol(BIGINT, "COL2"))), ImmutableList.of(p.symbol("COL1")), ImmutableList.of(p.symbol("COL2")), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(new Symbol(DOUBLE, "AVG"), PlanBuilder.aggregation("avg", ImmutableList.of(new Reference(BIGINT, "COL2"))), ImmutableList.of(BIGINT)) .singleGroupingSet(new Symbol(BIGINT, "COL1")))) @@ -291,8 +279,6 @@ public void testDoesNotFireWhenNotDistinct() ImmutableList.of(new EquiJoinClause(p.symbol("COL1"), p.symbol("COL2"))), ImmutableList.of(p.symbol("COL1")), ImmutableList.of(p.symbol("COL2")), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(p.symbol("AVG", DOUBLE), PlanBuilder.aggregation("avg", ImmutableList.of(new Reference(DOUBLE, "COL2"))), ImmutableList.of(DOUBLE)) .singleGroupingSet(p.symbol("COL1")))) @@ -310,8 +296,6 @@ public void testDoesNotFireWhenGroupingOnInner() ImmutableList.of(new EquiJoinClause(new Symbol(BIGINT, "COL1"), new Symbol(BIGINT, "COL2"))), ImmutableList.of(p.symbol("COL1")), ImmutableList.of(p.symbol("COL2")), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(new Symbol(DOUBLE, "AVG"), PlanBuilder.aggregation("avg", ImmutableList.of(new Reference(BIGINT, "COL2"))), ImmutableList.of(BIGINT)) .singleGroupingSet(new Symbol(BIGINT, "COL1"), new Symbol(BIGINT, "COL3")))) @@ -330,8 +314,6 @@ public void testDoesNotFireWhenAggregationDoesNotHaveSymbols() ImmutableList.of(new EquiJoinClause(new Symbol(BIGINT, "COL1"), new Symbol(BIGINT, "COL2"))), ImmutableList.of(p.symbol("COL1")), ImmutableList.of(p.symbol("COL2")), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(new Symbol(BIGINT, "SUM"), PlanBuilder.aggregation("sum", ImmutableList.of(new Reference(BIGINT, "COL1"))), ImmutableList.of(BIGINT)) .singleGroupingSet(new Symbol(BIGINT, "COL1")))) @@ -350,8 +332,6 @@ public void testDoesNotFireWhenAggregationOnMultipleSymbolsDoesNotHaveSomeSymbol ImmutableList.of(new EquiJoinClause(new Symbol(BIGINT, "COL1"), new Symbol(BIGINT, "COL2"))), ImmutableList.of(new Symbol(BIGINT, "COL1")), ImmutableList.of(new Symbol(BIGINT, "COL2")), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(new Symbol(BIGINT, "MIN_BY"), PlanBuilder.aggregation("min_by", ImmutableList.of(new Reference(BIGINT, "COL2"), new Reference(BIGINT, "COL1"))), ImmutableList.of(BIGINT, BIGINT)) .singleGroupingSet(new Symbol(BIGINT, "COL1")))) @@ -366,8 +346,6 @@ public void testDoesNotFireWhenAggregationOnMultipleSymbolsDoesNotHaveSomeSymbol ImmutableList.of(new EquiJoinClause(new Symbol(BIGINT, "COL1"), new Symbol(BIGINT, "COL2"))), ImmutableList.of(new Symbol(BIGINT, "COL1")), ImmutableList.of(new Symbol(BIGINT, "COL2")), - Optional.empty(), - Optional.empty(), Optional.empty())) .addAggregation(new Symbol(BIGINT, "SUM"), PlanBuilder.aggregation("sum", ImmutableList.of(new Reference(BIGINT, "COL2"))), ImmutableList.of(BIGINT)) .addAggregation(new Symbol(BIGINT, "MIN_BY"), PlanBuilder.aggregation("min_by", ImmutableList.of(new Reference(BIGINT, "COL2"), new Reference(BIGINT, "COL3"))), ImmutableList.of(BIGINT, BIGINT)) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushDownDereferencesRules.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushDownDereferencesRules.java index cd53c64c5572..6e174fbc17d4 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushDownDereferencesRules.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushDownDereferencesRules.java @@ -236,8 +236,6 @@ public void testPushdownDereferencesThroughSemiJoin() p.symbol("msg2", ROW_TYPE), p.symbol("filtering_msg", ROW_TYPE), p.symbol("match"), - Optional.empty(), - Optional.empty(), p.values(p.symbol("msg1", ROW_TYPE), p.symbol("msg2", ROW_TYPE)), p.values(p.symbol("filtering_msg", ROW_TYPE))))) .matches( diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushLimitThroughSemiJoin.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushLimitThroughSemiJoin.java index d1219f8850f9..c96d7cfbe4e0 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushLimitThroughSemiJoin.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushLimitThroughSemiJoin.java @@ -20,8 +20,6 @@ import io.trino.sql.planner.plan.PlanNode; import org.junit.jupiter.api.Test; -import java.util.Optional; - import static io.trino.sql.planner.assertions.PlanMatchPattern.limit; import static io.trino.sql.planner.assertions.PlanMatchPattern.semiJoin; import static io.trino.sql.planner.assertions.PlanMatchPattern.sort; @@ -91,8 +89,6 @@ public void testDoesNotFire() p.symbol("leftKey"), p.symbol("rightKey"), p.symbol("output"), - Optional.empty(), - Optional.empty(), p.values(p.symbol("leftKey")), p.limit(1, p.values(p.symbol("rightKey"))))) @@ -123,8 +119,6 @@ private static PlanNode buildSemiJoin(PlanBuilder p) leftKey, rightKey, p.symbol("match"), - Optional.empty(), - Optional.empty(), p.values(leftKey), p.values(rightKey)); } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushPartialAggregationThroughJoin.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushPartialAggregationThroughJoin.java index 375eb1f4ab4d..9513d9466300 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushPartialAggregationThroughJoin.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushPartialAggregationThroughJoin.java @@ -144,8 +144,6 @@ public void testDoesNotPushPartialAggregationForExpandingJoin() ImmutableList.of(), Optional.of(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI"))), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of())) .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR"))), ImmutableList.of(DOUBLE)) .singleGroupingSet(p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI")) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestRemoveAggregationInSemiJoin.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestRemoveAggregationInSemiJoin.java index 58972485424c..63bfd7a7afc5 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestRemoveAggregationInSemiJoin.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestRemoveAggregationInSemiJoin.java @@ -21,8 +21,6 @@ import io.trino.sql.planner.plan.PlanNode; import org.junit.jupiter.api.Test; -import java.util.Optional; - import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.sql.planner.assertions.PlanMatchPattern.semiJoin; import static io.trino.sql.planner.assertions.PlanMatchPattern.values; @@ -58,8 +56,6 @@ private static PlanNode semiJoinWithDistinctAsFilteringSource(PlanBuilder p) leftKey, rightKey, p.symbol("match"), - Optional.empty(), - Optional.empty(), p.values(leftKey), p.aggregation(builder -> builder .singleGroupingSet(rightKey) @@ -74,8 +70,6 @@ private static PlanNode semiJoinWithAggregationAsFilteringSource(PlanBuilder p) leftKey, rightKey, p.symbol("match"), - Optional.empty(), - Optional.empty(), p.values(leftKey), p.aggregation(builder -> builder .globalGrouping() diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestTransformFilteringSemiJoinToInnerJoin.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestTransformFilteringSemiJoinToInnerJoin.java index 5108566fd711..2d87d376e685 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestTransformFilteringSemiJoinToInnerJoin.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestTransformFilteringSemiJoinToInnerJoin.java @@ -60,8 +60,6 @@ public void testTransformSemiJoinToInnerJoin() a, b, aInB, - Optional.empty(), - Optional.empty(), Optional.empty())); }) .matches(project( @@ -97,8 +95,6 @@ public void testRemoveRedundantFilter() a, b, aInB, - Optional.empty(), - Optional.empty(), Optional.empty())); }) .matches(project( @@ -131,8 +127,6 @@ public void testFilterNotMatching() a, b, aInB, - Optional.empty(), - Optional.empty(), Optional.empty())); }) .doesNotFire(); @@ -156,8 +150,6 @@ public void testDoNotRewriteInContextOfDelete() a, b, aInB, - Optional.empty(), - Optional.empty(), Optional.empty())); }) .doesNotFire(); @@ -180,8 +172,6 @@ public void testDoNotRewriteInContextOfDelete() a, b, aInB, - Optional.empty(), - Optional.empty(), Optional.empty())); }) .doesNotFire(); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java index ccc3d69346fa..df55b1f056f7 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java @@ -814,8 +814,6 @@ public SemiJoinNode semiJoin( Symbol sourceJoinSymbol, Symbol filteringSourceJoinSymbol, Symbol semiJoinOutput, - Optional sourceHashSymbol, - Optional filteringSourceHashSymbol, PlanNode source, PlanNode filteringSource) { @@ -825,8 +823,6 @@ public SemiJoinNode semiJoin( sourceJoinSymbol, filteringSourceJoinSymbol, semiJoinOutput, - sourceHashSymbol, - filteringSourceHashSymbol, Optional.empty(), Optional.empty()); } @@ -837,8 +833,6 @@ public SemiJoinNode semiJoin( Symbol sourceJoinSymbol, Symbol filteringSourceJoinSymbol, Symbol semiJoinOutput, - Optional sourceHashSymbol, - Optional filteringSourceHashSymbol, Optional distributionType) { return semiJoin( @@ -847,8 +841,6 @@ public SemiJoinNode semiJoin( sourceJoinSymbol, filteringSourceJoinSymbol, semiJoinOutput, - sourceHashSymbol, - filteringSourceHashSymbol, distributionType, Optional.empty()); } @@ -859,8 +851,6 @@ public SemiJoinNode semiJoin( Symbol sourceJoinSymbol, Symbol filteringSourceJoinSymbol, Symbol semiJoinOutput, - Optional sourceHashSymbol, - Optional filteringSourceHashSymbol, Optional distributionType, Optional dynamicFilterId) { @@ -871,8 +861,6 @@ public SemiJoinNode semiJoin( sourceJoinSymbol, filteringSourceJoinSymbol, semiJoinOutput, - sourceHashSymbol, - filteringSourceHashSymbol, distributionType, dynamicFilterId); } @@ -1029,16 +1017,9 @@ private JoinNode join(JoinType joinType, PlanNode left, PlanNode right, Optional left.getOutputSymbols(), right.getOutputSymbols(), filter, - Optional.empty(), - Optional.empty(), ImmutableMap.of()); } - public JoinNode join(JoinType type, PlanNode left, PlanNode right, List criteria, List leftOutputSymbols, List rightOutputSymbols, Optional filter) - { - return join(type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, filter, Optional.empty(), Optional.empty()); - } - public JoinNode join(JoinType type, JoinNode.DistributionType distributionType, PlanNode left, PlanNode right, JoinNode.EquiJoinClause... criteria) { return join( @@ -1049,8 +1030,6 @@ public JoinNode join(JoinType type, JoinNode.DistributionType distributionType, left.getOutputSymbols(), right.getOutputSymbols(), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(distributionType), ImmutableMap.of()); } @@ -1062,11 +1041,9 @@ public JoinNode join( List criteria, List leftOutputSymbols, List rightOutputSymbols, - Optional filter, - Optional leftHashSymbol, - Optional rightHashSymbol) + Optional filter) { - return join(type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, filter, leftHashSymbol, rightHashSymbol, Optional.empty(), ImmutableMap.of()); + return join(type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, filter, Optional.empty(), ImmutableMap.of()); } public JoinNode join( @@ -1077,11 +1054,9 @@ public JoinNode join( List leftOutputSymbols, List rightOutputSymbols, Optional filter, - Optional leftHashSymbol, - Optional rightHashSymbol, Map dynamicFilters) { - return join(type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, filter, leftHashSymbol, rightHashSymbol, Optional.empty(), dynamicFilters); + return join(type, left, right, criteria, leftOutputSymbols, rightOutputSymbols, filter, Optional.empty(), dynamicFilters); } public JoinNode join( @@ -1092,8 +1067,6 @@ public JoinNode join( List leftOutputSymbols, List rightOutputSymbols, Optional filter, - Optional leftHashSymbol, - Optional rightHashSymbol, Optional distributionType, Map dynamicFilters) { @@ -1105,8 +1078,6 @@ public JoinNode join( leftOutputSymbols, rightOutputSymbols, filter, - leftHashSymbol, - rightHashSymbol, distributionType, dynamicFilters); } @@ -1120,8 +1091,6 @@ public JoinNode join( List leftOutputSymbols, List rightOutputSymbols, Optional filter, - Optional leftHashSymbol, - Optional rightHashSymbol, Optional distributionType, Map dynamicFilters) { @@ -1135,8 +1104,6 @@ public JoinNode join( rightOutputSymbols, false, filter, - leftHashSymbol, - rightHashSymbol, distributionType, Optional.empty(), dynamicFilters, @@ -1145,25 +1112,21 @@ public JoinNode join( public PlanNode indexJoin(IndexJoinNode.Type type, PlanNode probe, PlanNode index) { - return indexJoin(type, probe, index, emptyList(), Optional.empty(), Optional.empty()); + return indexJoin(type, probe, index, emptyList()); } public PlanNode indexJoin( IndexJoinNode.Type type, PlanNode probe, PlanNode index, - List criteria, - Optional probeHashSymbol, - Optional indexHashSymbol) + List criteria) { return new IndexJoinNode( idAllocator.getNextId(), type, probe, index, - criteria, - probeHashSymbol, - indexHashSymbol); + criteria); } public PlanNode spatialJoin( diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestRemoveUnsupportedDynamicFilters.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestRemoveUnsupportedDynamicFilters.java index 1e9554824391..5d52c3cf9bee 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestRemoveUnsupportedDynamicFilters.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestRemoveUnsupportedDynamicFilters.java @@ -129,8 +129,6 @@ public void testUnconsumedDynamicFilterInJoin() ImmutableList.of(ordersOrderKeySymbol), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), lineitemOrderKeySymbol)); assertPlan( removeUnsupportedDynamicFilters(root), @@ -161,8 +159,6 @@ public void testDynamicFilterConsumedOnBuildSide() ImmutableList.of(ordersOrderKeySymbol), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), lineitemOrderKeySymbol)); assertPlan( removeUnsupportedDynamicFilters(root), @@ -196,8 +192,6 @@ public void testUnmatchedDynamicFilter() ImmutableList.of(), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of())); assertPlan( removeUnsupportedDynamicFilters(root), @@ -230,8 +224,6 @@ public void testRemoveDynamicFilterNotAboveTableScan() ImmutableList.of(), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), ordersOrderKeySymbol))); assertPlan( removeUnsupportedDynamicFilters(root), @@ -268,8 +260,6 @@ public void testNestedDynamicFilterDisjunctionRewrite() ImmutableList.of(ordersOrderKeySymbol), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of())); assertPlan( removeUnsupportedDynamicFilters(root), @@ -302,8 +292,6 @@ public void testNestedDynamicFilterConjunctionRewrite() ImmutableList.of(ordersOrderKeySymbol), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of())); assertPlan( removeUnsupportedDynamicFilters(root), @@ -340,8 +328,6 @@ public void testRemoveUnsupportedCast() ImmutableList.of(lineitemDoubleOrderKeySymbol), ImmutableList.of(ordersOrderKeySymbol), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), ordersOrderKeySymbol))); assertPlan( removeUnsupportedDynamicFilters(root), @@ -389,8 +375,6 @@ public void testUnconsumedDynamicFilterInSemiJoin() lineitemOrderKeySymbol, new Symbol(UNKNOWN, "SEMIJOIN_OUTPUT"), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(new DynamicFilterId("DF"))); assertPlan( removeUnsupportedDynamicFilters(root), @@ -415,8 +399,6 @@ public void testDynamicFilterConsumedOnFilteringSourceSideInSemiJoin() lineitemOrderKeySymbol, new Symbol(UNKNOWN, "SEMIJOIN_OUTPUT"), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(new DynamicFilterId("DF"))); assertPlan( removeUnsupportedDynamicFilters(root), @@ -441,8 +423,6 @@ public void testUnmatchedDynamicFilterInSemiJoin() lineitemOrderKeySymbol, new Symbol(UNKNOWN, "SEMIJOIN_OUTPUT"), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.empty()); assertPlan( removeUnsupportedDynamicFilters(root), @@ -467,8 +447,6 @@ public void testRemoveDynamicFilterNotAboveTableScanWithSemiJoin() lineitemOrderKeySymbol, new Symbol(UNKNOWN, "SEMIJOIN_OUTPUT"), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(new DynamicFilterId("DF"))); assertPlan( diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestUnaliasSymbolReferences.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestUnaliasSymbolReferences.java index 94fd2557b1ac..9fb21da36d75 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestUnaliasSymbolReferences.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestUnaliasSymbolReferences.java @@ -99,8 +99,6 @@ probeColumn2, new TpchColumnHandle("suppkey", BIGINT))))), ImmutableList.of(), ImmutableList.of(buildAlias1, buildAlias2), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(dynamicFilterId1, buildAlias1, dynamicFilterId2, buildAlias2)); }, join(INNER, builder -> builder diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestAnonymizeJsonRepresentation.java b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestAnonymizeJsonRepresentation.java index 930bf771758c..46db3c2add6b 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestAnonymizeJsonRepresentation.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestAnonymizeJsonRepresentation.java @@ -143,15 +143,12 @@ public void testJoinPlan() ImmutableList.of(pb.symbol("b", BIGINT)), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), pb.symbol("d", BIGINT))), new JsonRenderedNode( "2", "InnerJoin", ImmutableMap.of( - "criteria", "(\"symbol_1\" = \"symbol_2\")", - "hash", "[]"), + "criteria", "(\"symbol_1\" = \"symbol_2\")"), ImmutableList.of(new Symbol(BIGINT, "symbol_3")), ImmutableList.of("dynamicFilterAssignments = {symbol_2 -> #DF}"), ImmutableList.of(), diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestJsonRepresentation.java b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestJsonRepresentation.java index 6cad6ce92c09..61f61a9ac34b 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestJsonRepresentation.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestJsonRepresentation.java @@ -163,13 +163,11 @@ public void testJoinPlan() ImmutableList.of(pb.symbol("b", BIGINT)), ImmutableList.of(), Optional.of(new Comparison(LESS_THAN, new Reference(BIGINT, "a"), new Reference(BIGINT, "c"))), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), pb.symbol("d", BIGINT))), new JsonRenderedNode( "2", "InnerJoin", - ImmutableMap.of("criteria", "(a = d)", "filter", "(a < c)", "hash", "[]"), + ImmutableMap.of("criteria", "(a = d)", "filter", "(a < c)"), ImmutableList.of(new Symbol(BIGINT, "b")), ImmutableList.of("dynamicFilterAssignments = {d -> #DF}"), ImmutableList.of(), diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestDynamicFiltersChecker.java b/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestDynamicFiltersChecker.java index e4793c56170b..684981f67e20 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestDynamicFiltersChecker.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/sanity/TestDynamicFiltersChecker.java @@ -105,8 +105,6 @@ public void testUnconsumedDynamicFilterInJoin() ImmutableList.of(ordersOrderKeySymbol), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), lineitemOrderKeySymbol)); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class) @@ -128,8 +126,6 @@ public void testDynamicFilterConsumedOnBuildSide() ImmutableList.of(ordersOrderKeySymbol), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), lineitemOrderKeySymbol)); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class) @@ -154,8 +150,6 @@ public void testUnmatchedDynamicFilter() ImmutableList.of(ordersOrderKeySymbol), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of())); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class) @@ -180,8 +174,6 @@ public void testDynamicFilterNotAboveTableScan() ImmutableList.of(), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), ordersOrderKeySymbol))); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class) @@ -210,8 +202,6 @@ public void testUnmatchedNestedDynamicFilter() ImmutableList.of(ordersOrderKeySymbol), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of())); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class) @@ -232,8 +222,6 @@ public void testUnsupportedDynamicFilterExpression() ImmutableList.of(lineitemOrderKeySymbol), ImmutableList.of(ordersOrderKeySymbol), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), ordersOrderKeySymbol))); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class) @@ -254,8 +242,6 @@ public void testUnsupportedCastExpression() ImmutableList.of(lineitemOrderKeySymbol), ImmutableList.of(ordersOrderKeySymbol), Optional.empty(), - Optional.empty(), - Optional.empty(), ImmutableMap.of(new DynamicFilterId("DF"), ordersOrderKeySymbol))); assertThatThrownBy(() -> validatePlan(root)).isInstanceOf(VerifyException.class).hasMessageMatching("The expression CAST\\(LINEITEM_OK AS integer\\) within in a CAST in dynamic filter must be a SymbolReference."); } @@ -270,8 +256,6 @@ public void testUnconsumedDynamicFilterInSemiJoin() lineitemOrderKeySymbol, new Symbol(UNKNOWN, "SEMIJOIN_OUTPUT"), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(new DynamicFilterId("DF"))); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class) @@ -296,8 +280,6 @@ public void testDynamicFilterConsumedOnFilteringSourceSideInSemiJoin() lineitemOrderKeySymbol, new Symbol(UNKNOWN, "SEMIJOIN_OUTPUT"), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(new DynamicFilterId("DF"))); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class) @@ -321,8 +303,6 @@ public void testUnmatchedDynamicFilterInSemiJoin() lineitemOrderKeySymbol, new Symbol(UNKNOWN, "SEMIJOIN_OUTPUT"), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.empty())); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class) @@ -343,8 +323,6 @@ public void testDynamicFilterNotAboveTableScanWithSemiJoin() lineitemOrderKeySymbol, new Symbol(UNKNOWN, "SEMIJOIN_OUTPUT"), Optional.empty(), - Optional.empty(), - Optional.empty(), Optional.of(new DynamicFilterId("DF"))); assertThatThrownBy(() -> validatePlan(root)) .isInstanceOf(VerifyException.class)