diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 4214f00f7a0..5714faa12e6 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -484,7 +484,8 @@ public class SqlBenchmark AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java index 0e4ba5b4f00..56617e45f6c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java @@ -321,7 +321,8 @@ public class SqlExpressionBenchmark AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); try { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java index 358c0bb1969..721fdef76ff 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java @@ -310,7 +310,8 @@ public class SqlNestedDataBenchmark AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); try { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java index 8b0ed6c96c2..b7c450f3127 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java @@ -127,7 +127,8 @@ public class SqlVsNativeBenchmark AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); groupByQuery = GroupByQuery .builder() diff --git a/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java b/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java index bb03146a05b..d2a3fd0d52f 100644 --- a/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java +++ b/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java @@ -22,7 +22,6 @@ package org.apache.druid.guice; import com.fasterxml.jackson.databind.AnnotationIntrospector; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; -import com.google.common.annotations.VisibleForTesting; import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Injector; @@ -96,14 +95,12 @@ public class DruidSecondaryModule implements Module return smileMapper; } - @VisibleForTesting public static void setupJackson(Injector injector, final ObjectMapper mapper) { mapper.setInjectableValues(new GuiceInjectableValues(injector)); setupAnnotationIntrospector(mapper, new GuiceAnnotationIntrospector()); } - @VisibleForTesting public static void setupAnnotationIntrospector( final ObjectMapper mapper, final AnnotationIntrospector annotationIntrospector diff --git a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java index d12497e0196..a02bf3d054a 100644 --- a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java +++ b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java @@ -22,19 +22,27 @@ package org.apache.druid.guice; import com.fasterxml.jackson.databind.BeanProperty; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.InjectableValues; +import com.google.inject.ConfigurationException; import com.google.inject.Injector; import com.google.inject.Key; import org.apache.druid.java.util.common.IAE; +import javax.annotation.Nullable; +import java.util.HashSet; +import java.util.concurrent.atomic.AtomicReference; + /** + * */ public class GuiceInjectableValues extends InjectableValues { private final Injector injector; + private final AtomicReference> nullables; public GuiceInjectableValues(Injector injector) { this.injector = injector; + this.nullables = new AtomicReference<>(new HashSet<>()); } @Override @@ -49,8 +57,22 @@ public class GuiceInjectableValues extends InjectableValues // whatever provider needs" // Currently we should only be dealing with `Key` instances, and anything more advanced should be handled with // great care - if (valueId instanceof Key) { - return injector.getInstance((Key) valueId); + if (nullables.get().contains((Key) valueId)) { + return null; + } else if (valueId instanceof Key) { + try { + return injector.getInstance((Key) valueId); + } + catch (ConfigurationException ce) { + // check if nullable annotation is present for this + if (forProperty.getAnnotation(Nullable.class) != null) { + HashSet encounteredNullables = new HashSet<>(nullables.get()); + encounteredNullables.add((Key) valueId); + nullables.set(encounteredNullables); + return null; + } + throw ce; + } } throw new IAE( "Unknown class type [%s] for valueId [%s]", diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index f971a4cc73f..e8b761c8007 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -47,6 +47,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; public abstract class BaseLeafFrameProcessor implements FrameProcessor @@ -88,6 +89,58 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor this.broadcastJoinHelper = inputChannelsAndBroadcastJoinHelper.rhs; } + /** + * Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up their primary and side channels. + */ + private static Pair, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper( + final DataSource dataSource, + final ReadableInput baseInput, + final Int2ObjectMap sideChannels, + final JoinableFactoryWrapper joinableFactory, + final long memoryReservedForBroadcastJoin + ) + { + if (!(dataSource instanceof JoinDataSource) && !sideChannels.isEmpty()) { + throw new ISE("Did not expect side channels for dataSource [%s]", dataSource); + } + + final List inputChannels = new ArrayList<>(); + final BroadcastJoinHelper broadcastJoinHelper; + + if (baseInput.hasChannel()) { + inputChannels.add(baseInput.getChannel()); + } + + if (dataSource instanceof JoinDataSource) { + final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap(); + final List channelReaders = new ArrayList<>(); + + if (baseInput.hasChannel()) { + // BroadcastJoinHelper doesn't need to read the base channel, so stub in a null reader. + channelReaders.add(null); + } + + for (Int2ObjectMap.Entry sideChannelEntry : sideChannels.int2ObjectEntrySet()) { + final int inputNumber = sideChannelEntry.getIntKey(); + inputNumberToProcessorChannelMap.put(inputNumber, inputChannels.size()); + inputChannels.add(sideChannelEntry.getValue().getChannel()); + channelReaders.add(sideChannelEntry.getValue().getChannelFrameReader()); + } + + broadcastJoinHelper = new BroadcastJoinHelper( + inputNumberToProcessorChannelMap, + inputChannels, + channelReaders, + joinableFactory, + memoryReservedForBroadcastJoin + ); + } else { + broadcastJoinHelper = null; + } + + return Pair.of(inputChannels, broadcastJoinHelper); + } + @Override public List inputChannels() { @@ -146,6 +199,7 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor private boolean initializeSegmentMapFn(final IntSet readableInputs) { + final AtomicLong cpuAccumulator = new AtomicLong(); if (segmentMapFn != null) { return true; } else if (broadcastJoinHelper == null) { @@ -153,64 +207,11 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor return true; } else { final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); - + DataSource inlineChannelDataSource = broadcastJoinHelper.inlineChannelData(query.getDataSource()); if (retVal) { - segmentMapFn = broadcastJoinHelper.makeSegmentMapFn(query); + segmentMapFn = inlineChannelDataSource.createSegmentMapFunction(query, cpuAccumulator); } - return retVal; } } - - /** - * Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up their primary and side channels. - */ - private static Pair, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper( - final DataSource dataSource, - final ReadableInput baseInput, - final Int2ObjectMap sideChannels, - final JoinableFactoryWrapper joinableFactory, - final long memoryReservedForBroadcastJoin - ) - { - if (!(dataSource instanceof JoinDataSource) && !sideChannels.isEmpty()) { - throw new ISE("Did not expect side channels for dataSource [%s]", dataSource); - } - - final List inputChannels = new ArrayList<>(); - final BroadcastJoinHelper broadcastJoinHelper; - - if (baseInput.hasChannel()) { - inputChannels.add(baseInput.getChannel()); - } - - if (dataSource instanceof JoinDataSource) { - final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap(); - final List channelReaders = new ArrayList<>(); - - if (baseInput.hasChannel()) { - // BroadcastJoinHelper doesn't need to read the base channel, so stub in a null reader. - channelReaders.add(null); - } - - for (Int2ObjectMap.Entry sideChannelEntry : sideChannels.int2ObjectEntrySet()) { - final int inputNumber = sideChannelEntry.getIntKey(); - inputNumberToProcessorChannelMap.put(inputNumber, inputChannels.size()); - inputChannels.add(sideChannelEntry.getValue().getChannel()); - channelReaders.add(sideChannelEntry.getValue().getChannelFrameReader()); - } - - broadcastJoinHelper = new BroadcastJoinHelper( - inputNumberToProcessorChannelMap, - inputChannels, - channelReaders, - joinableFactory, - memoryReservedForBroadcastJoin - ); - } else { - broadcastJoinHelper = null; - } - - return Pair.of(inputChannels, broadcastJoinHelper); - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java index 07e73012457..d6e7679023b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java @@ -19,7 +19,6 @@ package org.apache.druid.msq.querykit; -import com.google.common.annotations.VisibleForTesting; import it.unimi.dsi.fastutil.ints.Int2IntMap; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntOpenHashSet; @@ -32,18 +31,12 @@ import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault; import org.apache.druid.msq.indexing.error.MSQException; import org.apache.druid.query.DataSource; import org.apache.druid.query.InlineDataSource; -import org.apache.druid.query.Query; -import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; -import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.join.JoinableFactoryWrapper; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Function; import java.util.stream.Collectors; public class BroadcastJoinHelper @@ -138,21 +131,9 @@ public class BroadcastJoinHelper return sideChannelNumbers; } - public Function makeSegmentMapFn(final Query query) - { - final DataSource dataSourceWithInlinedChannelData = inlineChannelData(query.getDataSource()); - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(dataSourceWithInlinedChannelData); - return joinableFactory.createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - new AtomicLong(), - analysis.getBaseQuery().orElse(query) - ); - } - @VisibleForTesting - DataSource inlineChannelData(final DataSource originalDataSource) + public DataSource inlineChannelData(final DataSource originalDataSource) { if (originalDataSource instanceof InputNumberDataSource) { final int inputNumber = ((InputNumberDataSource) originalDataSource).getInputNumber(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java index fc715d3544b..a5c61c5bd75 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java @@ -315,9 +315,9 @@ public class DataSourcePlan clause.getPrefix(), clause.getCondition(), clause.getJoinType(), - // First JoinDataSource (i == 0) involves the base table, so we need to propagate the base table filter. - i == 0 ? analysis.getJoinBaseTableFilter().orElse(null) : null + i == 0 ? analysis.getJoinBaseTableFilter().orElse(null) : null, + dataSource.getJoinableFactoryWrapper() ); inputSpecs.addAll(clausePlan.getInputSpecs()); clausePlan.getBroadcastInputs().intStream().forEach(n -> broadcastInputs.add(n + shift)); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index df3d9fc3b06..2fb4a61ee8c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -24,11 +24,15 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.DataSource; +import org.apache.druid.query.Query; +import org.apache.druid.segment.SegmentReference; import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; @JsonTypeName("inputNumber") public class InputNumberDataSource implements DataSource @@ -81,6 +85,27 @@ public class InputNumberDataSource implements DataSource return false; } + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTimeAcc + ) + { + return Function.identity(); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + + @Override + public byte[] getCacheKey() + { + return null; + } + @JsonProperty public int getInputNumber() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 71d3a95201c..de7038700b8 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -34,6 +34,7 @@ import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; @@ -438,6 +439,122 @@ public class MSQSelectTest extends MSQTestBase .verifyResults(); } + @Test + public void testBroadcastJoin() + { + final RowSignature resultSignature = RowSignature.builder() + .add("dim2", ColumnType.STRING) + .add("EXPR$1", ColumnType.DOUBLE) + .build(); + + final ImmutableList expectedResults; + + if (NullHandling.sqlCompatible()) { + expectedResults = ImmutableList.of( + new Object[]{null, 4.0}, + new Object[]{"", 3.0}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ); + } else { + expectedResults = ImmutableList.of( + new Object[]{null, 3.6666666666666665}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ); + } + + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2", "m1", "m2") + .context( + defaultScanQueryContext( + RowSignature.builder() + .add("dim2", ColumnType.STRING) + .add("m1", ColumnType.FLOAT) + .add("m2", ColumnType.DOUBLE) + .build() + ) + ) + .limit(10) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.ofColumn(ColumnType.FLOAT, "m1"), + DruidExpression.ofColumn(ColumnType.FLOAT, "j0.m1") + ), + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimensions(new DefaultDimensionSpec("j0.dim2", "d0", ColumnType.STRING)) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + useDefault + ? aggregators( + new DoubleSumAggregatorFactory("a0:sum", "j0.m2"), + new CountAggregatorFactory("a0:count") + ) + : aggregators( + new DoubleSumAggregatorFactory("a0:sum", "j0.m2"), + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0:count"), + not(selector("j0.m2", null, null)), + + // Not sure why the name is only set in SQL-compatible null mode. Seems strange. + // May be due to JSON serialization: name is set on the serialized aggregator even + // if it was originally created with no name. + NullHandling.sqlCompatible() ? "a0:count" : null + ) + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "a0:sum"), + new FieldAccessPostAggregator(null, "a0:count") + ) + ) + + ) + ) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql( + "SELECT t1.dim2, AVG(t1.m2) FROM " + + "foo " + + "INNER JOIN (SELECT * FROM foo LIMIT 10) AS t1 " + + "ON t1.m1 = foo.m1 " + + "GROUP BY t1.dim2" + ) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(query) + .columnMappings(new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "dim2"), + new ColumnMapping("a0", "EXPR$1") + ))) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(expectedResults) + .verifyResults(); + } + @Test public void testGroupByOrderByAggregation() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java index 3246938b16a..4a32144d274 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java @@ -189,6 +189,7 @@ public class BroadcastJoinHelperTest extends InitializedNullHandlingTest "j.", JoinConditionAnalysis.forExpression("x == \"j.x\"", "j.", ExprMacroTable.nil()), JoinType.INNER, + null, null ) ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 405c1493206..7aebaa476da 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.test; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; @@ -30,10 +29,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.io.ByteStreams; -import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; -import com.google.inject.Module; import com.google.inject.TypeLiteral; import com.google.inject.util.Modules; import com.google.inject.util.Providers; @@ -43,6 +40,7 @@ import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.discovery.NodeRole; import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.guice.DruidSecondaryModule; import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.guice.JoinableFactoryModule; @@ -113,7 +111,6 @@ import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.loading.DataSegmentPusher; -import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.LocalLoadSpec; @@ -162,7 +159,6 @@ import org.mockito.Mockito; import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.io.Closeable; import java.io.File; import java.io.IOException; @@ -253,8 +249,14 @@ public class MSQTestBase extends BaseCalciteQueryTest groupByBuffers = TestGroupByBuffers.createDefault(); SqlTestFramework qf = queryFramework(); - Injector secondInjector = GuiceInjectors.makeStartupInjector(); - + Injector secondInjector = GuiceInjectors.makeStartupInjectorWithModules( + ImmutableList.of( + binder -> { + binder.bind(ExprMacroTable.class).toInstance(CalciteTests.createExprMacroTable()); + binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT); + } + ) + ); ObjectMapper secondMapper = setupObjectMapper(secondInjector); indexIO = new IndexIO(secondMapper, () -> 0); @@ -271,70 +273,69 @@ public class MSQTestBase extends BaseCalciteQueryTest segmentManager = new MSQTestSegmentManager(segmentCacheManager, indexIO); Injector injector = GuiceInjectors.makeStartupInjectorWithModules(ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) + binder -> { + DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() { - DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() + @Override + public String getFormatString() { - @Override - public String getFormatString() - { - return "test"; - } - }; - - GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); - - binder.bind(DruidProcessingConfig.class).toInstance(druidProcessingConfig); - binder.bind(new TypeLiteral>() - { - }).annotatedWith(Self.class).toInstance(ImmutableSet.of(NodeRole.PEON)); - binder.bind(QueryProcessingPool.class) - .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); - binder.bind(DataSegmentProvider.class) - .toInstance((dataSegment, channelCounters) -> - new LazyResourceHolder<>(getSupplierForSegment(dataSegment))); - binder.bind(IndexIO.class).toInstance(indexIO); - binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker()); - - binder.bind(GroupByStrategySelector.class) - .toInstance(GroupByQueryRunnerTest.makeQueryRunnerFactory(groupByQueryConfig, groupByBuffers) - .getStrategySelector()); - - LocalDataSegmentPusherConfig config = new LocalDataSegmentPusherConfig(); - try { - config.storageDirectory = tmpFolder.newFolder("localsegments"); - } - catch (IOException e) { - throw new ISE(e, "Unable to create folder"); - } - binder.bind(DataSegmentPusher.class).toInstance(new MSQTestDelegateDataSegmentPusher( - new LocalDataSegmentPusher(config), - segmentManager - )); - binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer()); - binder.bindConstant().annotatedWith(PruneLoadSpec.class).to(false); - // Client is not used in tests - binder.bind(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)) - .toProvider(Providers.of(null)); - // fault tolerance module - try { - JsonConfigProvider.bind( - binder, - MSQDurableStorageModule.MSQ_INTERMEDIATE_STORAGE_PREFIX, - StorageConnectorProvider.class, - MultiStageQuery.class - ); - localFileStorageDir = tmpFolder.newFolder("fault"); - binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class)) - .toProvider(new LocalFileStorageConnectorProvider(localFileStorageDir)); - } - catch (IOException e) { - throw new ISE(e, "Unable to create setup storage connector"); + return "test"; } + }; + + GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); + + + binder.bind(DruidProcessingConfig.class).toInstance(druidProcessingConfig); + binder.bind(new TypeLiteral>() + { + }).annotatedWith(Self.class).toInstance(ImmutableSet.of(NodeRole.PEON)); + binder.bind(QueryProcessingPool.class) + .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); + binder.bind(DataSegmentProvider.class) + .toInstance((dataSegment, channelCounters) -> + new LazyResourceHolder<>(getSupplierForSegment(dataSegment))); + binder.bind(IndexIO.class).toInstance(indexIO); + binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker()); + + binder.bind(GroupByStrategySelector.class) + .toInstance(GroupByQueryRunnerTest.makeQueryRunnerFactory(groupByQueryConfig, groupByBuffers) + .getStrategySelector()); + + LocalDataSegmentPusherConfig config = new LocalDataSegmentPusherConfig(); + try { + config.storageDirectory = tmpFolder.newFolder("localsegments"); } + catch (IOException e) { + throw new ISE(e, "Unable to create folder"); + } + binder.bind(DataSegmentPusher.class).toInstance(new MSQTestDelegateDataSegmentPusher( + new LocalDataSegmentPusher(config), + segmentManager + )); + binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer()); + binder.bindConstant().annotatedWith(PruneLoadSpec.class).to(false); + // Client is not used in tests + binder.bind(Key.get(ServiceClientFactory.class, EscalatedGlobal.class)) + .toProvider(Providers.of(null)); + // fault tolerance module + try { + JsonConfigProvider.bind( + binder, + MSQDurableStorageModule.MSQ_INTERMEDIATE_STORAGE_PREFIX, + StorageConnectorProvider.class, + MultiStageQuery.class + ); + localFileStorageDir = tmpFolder.newFolder("fault"); + binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class)) + .toProvider(new LocalFileStorageConnectorProvider(localFileStorageDir)); + } + catch (IOException e) { + throw new ISE(e, "Unable to create setup storage connector"); + } + + binder.bind(ExprMacroTable.class).toInstance(CalciteTests.createExprMacroTable()); + binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT); }, binder -> { // Requirements of JoinableFactoryModule @@ -393,7 +394,8 @@ public class MSQTestBase extends BaseCalciteQueryTest AuthTestUtils.TEST_AUTHORIZER_MAPPER, objectMapper, CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); sqlStatementFactory = CalciteTests.createSqlStatementFactory(engine, plannerFactory); @@ -568,14 +570,7 @@ public class MSQTestBase extends BaseCalciteQueryTest "compaction" ) ).registerSubtypes(ExternalDataSource.class)); - mapper.setInjectableValues( - new InjectableValues.Std() - .addValue(ObjectMapper.class, mapper) - .addValue(Injector.class, injector) - .addValue(DataSegment.PruneSpecsHolder.class, DataSegment.PruneSpecsHolder.DEFAULT) - .addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller()) - .addValue(ExprMacroTable.class, CalciteTests.createExprMacroTable()) - ); + DruidSecondaryModule.setupJackson(injector, mapper); mapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local")); diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 10d8e9806c4..f56a3550a34 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -21,9 +21,14 @@ package org.apache.druid.query; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.planning.PreJoinableClause; +import org.apache.druid.segment.SegmentReference; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; /** * Represents a source... of data... for a query. Analogous to the "FROM" clause in SQL. @@ -83,8 +88,39 @@ public interface DataSource * Returns true if this datasource represents concrete data that can be scanned via a * {@link org.apache.druid.segment.Segment} adapter of some kind. True for e.g. 'table' but not for 'query' or 'join'. * - * @see org.apache.druid.query.planning.DataSourceAnalysis#isConcreteBased() which uses this - * @see org.apache.druid.query.planning.DataSourceAnalysis#isConcreteTableBased() which uses this + * @see DataSourceAnalysis#isConcreteBased() which uses this + * @see DataSourceAnalysis#isConcreteTableBased() which uses this */ boolean isConcrete(); + + /** + * Returns a segment function on to how to segment should be modified. + * + * @param query the input query + * @param cpuTimeAcc the cpu time accumulator + * @return the segment function + */ + Function createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc); + + /** + * Returns an updated datasource based on the specified new source. + * + * @param newSource the new datasource to be used to update an existing query + * @return the updated datasource to be used + */ + DataSource withUpdatedDataSource(DataSource newSource); + + /** + * Compute a cache key prefix for a data source. This includes the data sources that participate in the RHS of a + * join as well as any query specific constructs associated with join data source such as base table filter. This key prefix + * can be used in segment level cache or result level cache. The function can return following + * - Non-empty byte array - If there is join datasource involved and caching is possible. The result includes + * join condition expression, join type and cache key returned by joinable factory for each {@link PreJoinableClause} + * - NULL - There is a join but caching is not possible. It may happen if one of the participating datasource + * in the JOIN is not cacheable. + * + * @return the cache key to be used as part of query cache key + */ + byte[] getCacheKey(); + } diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index 163bb986f91..040d6885782 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -27,6 +27,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.RowAdapter; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -36,12 +37,14 @@ import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; /** * Represents an inline datasource, where the rows are embedded within the DataSource object itself. - * + *

* The rows are backed by an Iterable, which can be lazy or not. Lazy datasources will only be iterated if someone calls * {@link #getRows()} and iterates the result, or until someone calls {@link #getRowsAsList()}. */ @@ -102,6 +105,55 @@ public class InlineDataSource implements DataSource return new InlineDataSource(rows, signature); } + /** + * A very zealous equality checker for "rows" that respects deep equality of arrays, but nevertheless refrains + * from materializing things needlessly. Useful for unit tests that want to compare equality of different + * InlineDataSource instances. + */ + private static boolean rowsEqual(final Iterable rowsA, final Iterable rowsB) + { + if (rowsA instanceof List && rowsB instanceof List) { + final List listA = (List) rowsA; + final List listB = (List) rowsB; + + if (listA.size() != listB.size()) { + return false; + } + + for (int i = 0; i < listA.size(); i++) { + final Object[] rowA = listA.get(i); + final Object[] rowB = listB.get(i); + + if (!Arrays.equals(rowA, rowB)) { + return false; + } + } + + return true; + } else { + return Objects.equals(rowsA, rowsB); + } + } + + /** + * A very zealous hash code computer for "rows" that is compatible with {@link #rowsEqual}. + */ + private static int rowsHashCode(final Iterable rows) + { + if (rows instanceof List) { + final List list = (List) rows; + + int code = 1; + for (final Object[] row : list) { + code = 31 * code + Arrays.hashCode(row); + } + + return code; + } else { + return Objects.hash(rows); + } + } + @Override public Set getTableNames() { @@ -182,6 +234,27 @@ public class InlineDataSource implements DataSource return true; } + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTimeAcc + ) + { + return Function.identity(); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + + @Override + public byte[] getCacheKey() + { + return null; + } + /** * Returns the row signature (map of column name to type) for this inline datasource. Note that types may * be null, meaning we know we have a column with a certain name, but we don't know what its type is. @@ -232,53 +305,4 @@ public class InlineDataSource implements DataSource "signature=" + signature + '}'; } - - /** - * A very zealous equality checker for "rows" that respects deep equality of arrays, but nevertheless refrains - * from materializing things needlessly. Useful for unit tests that want to compare equality of different - * InlineDataSource instances. - */ - private static boolean rowsEqual(final Iterable rowsA, final Iterable rowsB) - { - if (rowsA instanceof List && rowsB instanceof List) { - final List listA = (List) rowsA; - final List listB = (List) rowsB; - - if (listA.size() != listB.size()) { - return false; - } - - for (int i = 0; i < listA.size(); i++) { - final Object[] rowA = listA.get(i); - final Object[] rowB = listB.get(i); - - if (!Arrays.equals(rowA, rowB)) { - return false; - } - } - - return true; - } else { - return Objects.equals(rowsA, rowsB); - } - } - - /** - * A very zealous hash code computer for "rows" that is compatible with {@link #rowsEqual}. - */ - private static int rowsHashCode(final Iterable rows) - { - if (rows instanceof List) { - final List list = (List) rows; - - int code = 1; - for (final Object[] row : list) { - code = 31 * code + Arrays.hashCode(row); - } - - return code; - } else { - return Objects.hash(rows); - } - } } diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index a6206bc4619..e17e33c9a61 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -26,36 +26,61 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.primitives.Ints; +import org.apache.druid.common.guava.GuavaUtils; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.planning.PreJoinableClause; +import org.apache.druid.segment.SegmentReference; +import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.join.HashJoinSegment; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinPrefixUtils; import org.apache.druid.segment.join.JoinType; +import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; +import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey; +import org.apache.druid.segment.join.filter.JoinableClauses; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; +import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; - +import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; import java.util.stream.Collectors; /** * Represents a join of two datasources. - * + *

* Logically, this datasource contains the result of: - * + *

* (1) prefixing all right-side columns with "rightPrefix" * (2) then, joining the left and (prefixed) right sides using the provided type and condition - * + *

* Any columns from the left-hand side that start with "rightPrefix", and are at least one character longer than * the prefix, will be shadowed. It is up to the caller to ensure that no important columns are shadowed by the * chosen prefix. - * - * When analyzed by {@link org.apache.druid.query.planning.DataSourceAnalysis}, the right-hand side of this datasource - * will become a {@link org.apache.druid.query.planning.PreJoinableClause} object. + *

+ * When analyzed by {@link DataSourceAnalysis}, the right-hand side of this datasource + * will become a {@link PreJoinableClause} object. */ public class JoinDataSource implements DataSource { @@ -67,6 +92,12 @@ public class JoinDataSource implements DataSource // An optional filter on the left side if left is direct table access @Nullable private final DimFilter leftFilter; + @Nullable + private final JoinableFactoryWrapper joinableFactoryWrapper; + private static final Logger log = new Logger(JoinDataSource.class); + private final DataSourceAnalysis analysis; + + private JoinDataSource( DataSource left, @@ -74,7 +105,8 @@ public class JoinDataSource implements DataSource String rightPrefix, JoinConditionAnalysis conditionAnalysis, JoinType joinType, - @Nullable DimFilter leftFilter + @Nullable DimFilter leftFilter, + @Nullable JoinableFactoryWrapper joinableFactoryWrapper ) { this.left = Preconditions.checkNotNull(left, "left"); @@ -88,6 +120,8 @@ public class JoinDataSource implements DataSource "left filter is only supported if left data source is direct table access" ); this.leftFilter = leftFilter; + this.joinableFactoryWrapper = joinableFactoryWrapper; + this.analysis = DataSourceAnalysis.forDataSource(this); } /** @@ -101,7 +135,8 @@ public class JoinDataSource implements DataSource @JsonProperty("condition") String condition, @JsonProperty("joinType") JoinType joinType, @Nullable @JsonProperty("leftFilter") DimFilter leftFilter, - @JacksonInject ExprMacroTable macroTable + @JacksonInject ExprMacroTable macroTable, + @Nullable @JacksonInject JoinableFactoryWrapper joinableFactoryWrapper ) { return new JoinDataSource( @@ -114,7 +149,8 @@ public class JoinDataSource implements DataSource macroTable ), joinType, - leftFilter + leftFilter, + joinableFactoryWrapper ); } @@ -127,12 +163,22 @@ public class JoinDataSource implements DataSource final String rightPrefix, final JoinConditionAnalysis conditionAnalysis, final JoinType joinType, - final DimFilter leftFilter + final DimFilter leftFilter, + @Nullable final JoinableFactoryWrapper joinableFactoryWrapper ) { - return new JoinDataSource(left, right, rightPrefix, conditionAnalysis, joinType, leftFilter); + return new JoinDataSource( + left, + right, + rightPrefix, + conditionAnalysis, + joinType, + leftFilter, + joinableFactoryWrapper + ); } + @Override public Set getTableNames() { @@ -185,6 +231,12 @@ public class JoinDataSource implements DataSource return leftFilter; } + @Nullable + public JoinableFactoryWrapper getJoinableFactoryWrapper() + { + return joinableFactoryWrapper; + } + @Override public List getChildren() { @@ -204,7 +256,8 @@ public class JoinDataSource implements DataSource rightPrefix, conditionAnalysis, joinType, - leftFilter + leftFilter, + joinableFactoryWrapper ); } @@ -230,6 +283,7 @@ public class JoinDataSource implements DataSource * Computes a set of column names for left table expressions in join condition which may already have been defined as * a virtual column in the virtual column registry. It helps to remove any extraenous virtual columns created and only * use the relevant ones. + * * @return a set of column names which might be virtual columns on left table in join condition */ public Set getVirtualColumnCandidates() @@ -278,4 +332,153 @@ public class JoinDataSource implements DataSource ", leftFilter=" + leftFilter + '}'; } + + + /** + * Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join + * clauses is > 0). If mapping is not needed, this method will return {@link Function#identity()}. + * + * @param baseFilter Filter to apply before the join takes place + * @param clauses Pre-joinable clauses + * @param cpuTimeAccumulator An accumulator that we will add CPU nanos to; this is part of the function to encourage + * callers to remember to track metrics on CPU time required for creation of Joinables + * @param query The query that will be run on the mapped segments. Usually this should be + * {@code analysis.getBaseQuery().orElse(query)}, where "analysis" is a + * {@link DataSourceAnalysis} and "query" is the original + * query from the end user. + */ + public Function createSegmentMapFn( + @Nullable final Filter baseFilter, + final List clauses, + final AtomicLong cpuTimeAccumulator, + final Query query + ) + { + // compute column correlations here and RHS correlated values + return JvmUtils.safeAccumulateThreadCpuTime( + cpuTimeAccumulator, + () -> { + if (clauses.isEmpty()) { + return Function.identity(); + } else { + final JoinableClauses joinableClauses = JoinableClauses.createClauses( + clauses, + joinableFactoryWrapper.getJoinableFactory() + ); + final JoinFilterRewriteConfig filterRewriteConfig = JoinFilterRewriteConfig.forQuery(query); + + // Pick off any join clauses that can be converted into filters. + final Set requiredColumns = query.getRequiredColumns(); + final Filter baseFilterToUse; + final List clausesToUse; + + if (requiredColumns != null && filterRewriteConfig.isEnableRewriteJoinToFilter()) { + final Pair, List> conversionResult = JoinableFactoryWrapper.convertJoinsToFilters( + joinableClauses.getJoinableClauses(), + requiredColumns, + Ints.checkedCast(Math.min(filterRewriteConfig.getFilterRewriteMaxSize(), Integer.MAX_VALUE)) + ); + + baseFilterToUse = + Filters.maybeAnd( + Lists.newArrayList( + Iterables.concat( + Collections.singleton(baseFilter), + conversionResult.lhs + ) + ) + ).orElse(null); + clausesToUse = conversionResult.rhs; + } else { + baseFilterToUse = baseFilter; + clausesToUse = joinableClauses.getJoinableClauses(); + } + + // Analyze remaining join clauses to see if filters on them can be pushed down. + final JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + new JoinFilterPreAnalysisKey( + filterRewriteConfig, + clausesToUse, + query.getVirtualColumns(), + Filters.maybeAnd(Arrays.asList(baseFilterToUse, Filters.toFilter(query.getFilter()))) + .orElse(null) + ) + ); + + return baseSegment -> + new HashJoinSegment( + baseSegment, + baseFilterToUse, + GuavaUtils.firstNonNull(clausesToUse, ImmutableList.of()), + joinFilterPreAnalysis + ); + } + } + ); + } + + + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTimeAccumulator + ) + { + final Function segmentMapFn = createSegmentMapFn( + analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), + analysis.getPreJoinableClauses(), + cpuTimeAccumulator, + analysis.getBaseQuery().orElse(query) + ); + return segmentMapFn; + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + DataSource current = newSource; + DimFilter joinBaseFilter = analysis.getJoinBaseTableFilter().orElse(null); + + for (final PreJoinableClause clause : analysis.getPreJoinableClauses()) { + current = JoinDataSource.create( + current, + clause.getDataSource(), + clause.getPrefix(), + clause.getCondition(), + clause.getJoinType(), + joinBaseFilter, + this.joinableFactoryWrapper + ); + joinBaseFilter = null; + } + return current; + } + + @Override + public byte[] getCacheKey() + { + final List clauses = analysis.getPreJoinableClauses(); + if (clauses.isEmpty()) { + throw new IAE("No join clauses to build the cache key for data source [%s]", analysis.getDataSource()); + } + + final CacheKeyBuilder keyBuilder; + keyBuilder = new CacheKeyBuilder(JoinableFactoryWrapper.JOIN_OPERATION); + if (analysis.getJoinBaseTableFilter().isPresent()) { + keyBuilder.appendCacheable(analysis.getJoinBaseTableFilter().get()); + } + for (PreJoinableClause clause : clauses) { + Optional bytes = joinableFactoryWrapper.getJoinableFactory().computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); + if (!bytes.isPresent()) { + // Encountered a data source which didn't support cache yet + log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); + return new byte[]{}; + } + keyBuilder.appendByteArray(bytes.get()); + keyBuilder.appendString(clause.getCondition().getOriginalExpression()); + keyBuilder.appendString(clause.getPrefix()); + keyBuilder.appendString(clause.getJoinType().name()); + } + return keyBuilder.build(); + } } diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index 4b2b394554f..0b40aea020d 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -23,18 +23,21 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.SegmentReference; import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; /** * Represents a lookup. - * + *

* Currently, this datasource is not actually queryable, and attempts to do so will lead to errors. It is here as a * placeholder for a future time in which it will become queryable. - * + *

* The "lookupName" referred to here should be provided by a * {@link org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider}. */ @@ -96,6 +99,27 @@ public class LookupDataSource implements DataSource return true; } + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTime + ) + { + return Function.identity(); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + + @Override + public byte[] getCacheKey() + { + return null; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/Queries.java b/processing/src/main/java/org/apache/druid/query/Queries.java index 9a758b35112..f1a46f3b498 100644 --- a/processing/src/main/java/org/apache/druid/query/Queries.java +++ b/processing/src/main/java/org/apache/druid/query/Queries.java @@ -30,7 +30,6 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.query.planning.PreJoinableClause; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; @@ -191,40 +190,7 @@ public class Queries */ public static Query withBaseDataSource(final Query query, final DataSource newBaseDataSource) { - final Query retVal; - - if (query.getDataSource() instanceof QueryDataSource) { - final Query subQuery = ((QueryDataSource) query.getDataSource()).getQuery(); - retVal = query.withDataSource(new QueryDataSource(withBaseDataSource(subQuery, newBaseDataSource))); - } else { - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); - - DataSource current = newBaseDataSource; - DimFilter joinBaseFilter = analysis.getJoinBaseTableFilter().orElse(null); - - for (final PreJoinableClause clause : analysis.getPreJoinableClauses()) { - current = JoinDataSource.create( - current, - clause.getDataSource(), - clause.getPrefix(), - clause.getCondition(), - clause.getJoinType(), - joinBaseFilter - ); - joinBaseFilter = null; - } - - retVal = query.withDataSource(current); - } - - // Verify postconditions, just in case. - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(retVal.getDataSource()); - - if (!newBaseDataSource.equals(analysis.getBaseDataSource())) { - throw new ISE("Unable to replace base dataSource"); - } - - return retVal; + return query.withDataSource(query.getDataSource().withUpdatedDataSource(newBaseDataSource)); } /** diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 475368112c0..c495d0832f2 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -25,10 +25,13 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.SegmentReference; import java.util.Collections; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; @JsonTypeName("query") public class QueryDataSource implements DataSource @@ -88,6 +91,28 @@ public class QueryDataSource implements DataSource return false; } + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTime + ) + { + return Function.identity(); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return new QueryDataSource(query.withDataSource(query.getDataSource().withUpdatedDataSource(newSource))); + } + + @Override + public byte[] getCacheKey() + { + return null; + } + + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index dfe785a6265..2fbc6ca15f2 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -24,11 +24,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.SegmentReference; import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; @JsonTypeName("table") public class TableDataSource implements DataSource @@ -93,6 +96,27 @@ public class TableDataSource implements DataSource return true; } + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTime + ) + { + return Function.identity(); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + + @Override + public byte[] getCacheKey() + { + return new byte[0]; + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index 902c8c8f6e6..50802dc7252 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -26,9 +26,12 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.SegmentReference; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; import java.util.stream.Collectors; public class UnionDataSource implements DataSource @@ -105,6 +108,27 @@ public class UnionDataSource implements DataSource return dataSources.stream().allMatch(DataSource::isConcrete); } + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTime + ) + { + return Function.identity(); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + + @Override + public byte[] getCacheKey() + { + return null; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java index c726a8335af..55dd5ebdc01 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java @@ -39,7 +39,7 @@ import java.util.Optional; /** * Represents a deep, left-heavy join of a left-hand side baseSegment onto a series of right-hand side clauses. - * + *

* In other words, logically the operation is: join(join(join(baseSegment, clauses[0]), clauses[1]), clauses[2]) etc. */ public class HashJoinSegment implements SegmentReference @@ -148,15 +148,4 @@ public class HashJoinSegment implements SegmentReference return Optional.empty(); } } - - @Nullable - public Filter getBaseFilter() - { - return baseFilter; - } - - public List getClauses() - { - return clauses; - } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java index ca5bb44ac9d..974b2dea131 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java @@ -34,7 +34,7 @@ import java.util.stream.Collectors; * clause is "t1 JOIN t2 ON t1.x = t2.x" then this class represents "JOIN t2 ON x = t2.x" -- it does not include * references to the left-hand "t1". *

- * Created from {@link org.apache.druid.query.planning.PreJoinableClause} by {@link JoinableFactoryWrapper#createSegmentMapFn}. + * Created from {@link org.apache.druid.query.planning.PreJoinableClause}. */ public class JoinableClause implements ReferenceCountedObject { diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java index 5530d7243db..020d23317f7 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java @@ -22,42 +22,19 @@ package org.apache.druid.segment.join; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.HashMultiset; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.collect.Multiset; import com.google.common.collect.Sets; -import com.google.common.primitives.Ints; import com.google.inject.Inject; -import org.apache.druid.common.guava.GuavaUtils; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.query.Query; -import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; -import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.query.planning.PreJoinableClause; -import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.filter.FalseFilter; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey; -import org.apache.druid.segment.join.filter.JoinableClauses; -import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; -import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Function; import java.util.stream.Collectors; /** @@ -66,8 +43,7 @@ import java.util.stream.Collectors; public class JoinableFactoryWrapper { - private static final byte JOIN_OPERATION = 0x1; - private static final Logger log = new Logger(JoinableFactoryWrapper.class); + public static final byte JOIN_OPERATION = 0x1; private final JoinableFactory joinableFactory; @@ -77,144 +53,12 @@ public class JoinableFactoryWrapper this.joinableFactory = Preconditions.checkNotNull(joinableFactory, "joinableFactory"); } - public JoinableFactory getJoinableFactory() - { - return joinableFactory; - } - - /** - * Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join - * clauses is > 0). If mapping is not needed, this method will return {@link Function#identity()}. - * - * @param baseFilter Filter to apply before the join takes place - * @param clauses Pre-joinable clauses - * @param cpuTimeAccumulator An accumulator that we will add CPU nanos to; this is part of the function to encourage - * callers to remember to track metrics on CPU time required for creation of Joinables - * @param query The query that will be run on the mapped segments. Usually this should be - * {@code analysis.getBaseQuery().orElse(query)}, where "analysis" is a - * {@link DataSourceAnalysis} and "query" is the original - * query from the end user. - */ - public Function createSegmentMapFn( - @Nullable final Filter baseFilter, - final List clauses, - final AtomicLong cpuTimeAccumulator, - final Query query - ) - { - // compute column correlations here and RHS correlated values - return JvmUtils.safeAccumulateThreadCpuTime( - cpuTimeAccumulator, - () -> { - if (clauses.isEmpty()) { - return Function.identity(); - } else { - final JoinableClauses joinableClauses = JoinableClauses.createClauses(clauses, joinableFactory); - final JoinFilterRewriteConfig filterRewriteConfig = JoinFilterRewriteConfig.forQuery(query); - - // Pick off any join clauses that can be converted into filters. - final Set requiredColumns = query.getRequiredColumns(); - final Filter baseFilterToUse; - final List clausesToUse; - - if (requiredColumns != null && filterRewriteConfig.isEnableRewriteJoinToFilter()) { - final Pair, List> conversionResult = convertJoinsToFilters( - joinableClauses.getJoinableClauses(), - requiredColumns, - Ints.checkedCast(Math.min(filterRewriteConfig.getFilterRewriteMaxSize(), Integer.MAX_VALUE)) - ); - - baseFilterToUse = - Filters.maybeAnd( - Lists.newArrayList( - Iterables.concat( - Collections.singleton(baseFilter), - conversionResult.lhs - ) - ) - ).orElse(null); - clausesToUse = conversionResult.rhs; - } else { - baseFilterToUse = baseFilter; - clausesToUse = joinableClauses.getJoinableClauses(); - } - - // Analyze remaining join clauses to see if filters on them can be pushed down. - final JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - new JoinFilterPreAnalysisKey( - filterRewriteConfig, - clausesToUse, - query.getVirtualColumns(), - Filters.maybeAnd(Arrays.asList(baseFilterToUse, Filters.toFilter(query.getFilter()))) - .orElse(null) - ) - ); - - return baseSegment -> - new HashJoinSegment( - baseSegment, - baseFilterToUse, - GuavaUtils.firstNonNull(clausesToUse, ImmutableList.of()), - joinFilterPreAnalysis - ); - } - } - ); - } - - /** - * Compute a cache key prefix for a join data source. This includes the data sources that participate in the RHS of a - * join as well as any query specific constructs associated with join data source such as base table filter. This key prefix - * can be used in segment level cache or result level cache. The function can return following wrapped in an - * Optional - * - Non-empty byte array - If there is join datasource involved and caching is possible. The result includes - * join condition expression, join type and cache key returned by joinable factory for each {@link PreJoinableClause} - * - NULL - There is a join but caching is not possible. It may happen if one of the participating datasource - * in the JOIN is not cacheable. - * - * @param dataSourceAnalysis for the join datasource - * - * @return the optional cache key to be used as part of query cache key - * - * @throws {@link IAE} if this operation is called on a non-join data source - */ - public Optional computeJoinDataSourceCacheKey( - final DataSourceAnalysis dataSourceAnalysis - ) - { - final List clauses = dataSourceAnalysis.getPreJoinableClauses(); - if (clauses.isEmpty()) { - throw new IAE("No join clauses to build the cache key for data source [%s]", dataSourceAnalysis.getDataSource()); - } - - final CacheKeyBuilder keyBuilder; - keyBuilder = new CacheKeyBuilder(JOIN_OPERATION); - if (dataSourceAnalysis.getJoinBaseTableFilter().isPresent()) { - keyBuilder.appendCacheable(dataSourceAnalysis.getJoinBaseTableFilter().get()); - } - for (PreJoinableClause clause : clauses) { - Optional bytes = joinableFactory.computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); - if (!bytes.isPresent()) { - // Encountered a data source which didn't support cache yet - log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); - return Optional.empty(); - } - keyBuilder.appendByteArray(bytes.get()); - keyBuilder.appendString(clause.getCondition().getOriginalExpression()); - keyBuilder.appendString(clause.getPrefix()); - keyBuilder.appendString(clause.getJoinType().name()); - } - return Optional.of(keyBuilder.build()); - } - - /** * Converts any join clauses to filters that can be converted, and returns the rest as-is. - * + *

* See {@link #convertJoinToFilter} for details on the logic. */ - @VisibleForTesting - static Pair, List> convertJoinsToFilters( + public static Pair, List> convertJoinsToFilters( final List clauses, final Set requiredColumns, final int maxNumFilterValues @@ -271,16 +115,16 @@ public class JoinableFactoryWrapper /** * Converts a join clause into an "in" filter if possible. - * + *

* The requirements are: - * + *

* - it must be an INNER equi-join * - the right-hand columns referenced by the condition must not have any duplicate values. If there are duplicates - * values in the column, then the join is tried to be converted to a filter while maintaining the join clause on top - * as well for correct results. + * values in the column, then the join is tried to be converted to a filter while maintaining the join clause on top + * as well for correct results. * - no columns from the right-hand side can appear in "requiredColumns". If the columns from right side are required - * (ie they are directly or indirectly projected in the join output), then the join is tried to be converted to a - * filter while maintaining the join clause on top as well for correct results. + * (ie they are directly or indirectly projected in the join output), then the join is tried to be converted to a + * filter while maintaining the join clause on top as well for correct results. * * @return {@link JoinClauseToFilterConversion} object which contains the converted filter for the clause and a boolean * to represent whether the converted filter encapsulates the whole clause or not. More semantics of the object are @@ -340,12 +184,17 @@ public class JoinableFactoryWrapper return new JoinClauseToFilterConversion(null, false); } + public JoinableFactory getJoinableFactory() + { + return joinableFactory; + } + /** * Encapsulates the conversion which happened for a joinable clause. * convertedFilter represents the filter which got generated from the conversion. * joinClauseFullyConverted represents whether convertedFilter fully encapsulated the joinable clause or not. * Encapsulation of the clause means that the filter can replace the whole joinable clause. - * + *

* If convertedFilter is null and joinClauseFullyConverted is true, it means that all parts of the joinable clause can * be broken into filters. Further, all the clause conditions are on columns where the right side is only null values. * In that case, we replace joinable with a FalseFilter. diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java b/processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java similarity index 100% rename from processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java rename to processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java diff --git a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java index ff374ee2509..814250e1d25 100644 --- a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java @@ -22,28 +22,39 @@ package org.apache.druid.query; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterators; import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.TrueDimFilter; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; +import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.join.NoopJoinableFactory; +import org.easymock.Mock; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.mockito.Mockito; +import java.util.Arrays; import java.util.Collections; +import java.util.Optional; + public class JoinDataSourceTest { - @Rule - public ExpectedException expectedException = ExpectedException.none(); + public static final JoinableFactoryWrapper NOOP_JOINABLE_FACTORY_WRAPPER = new JoinableFactoryWrapper( + NoopJoinableFactory.INSTANCE + ); private final TableDataSource fooTable = new TableDataSource("foo"); private final TableDataSource barTable = new TableDataSource("bar"); private final LookupDataSource lookylooLookup = new LookupDataSource("lookyloo"); - private final JoinDataSource joinTableToLookup = JoinDataSource.create( fooTable, lookylooLookup, @@ -51,9 +62,9 @@ public class JoinDataSourceTest "x == \"j.x\"", JoinType.LEFT, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); - private final JoinDataSource joinTableToTable = JoinDataSource.create( fooTable, barTable, @@ -61,8 +72,13 @@ public class JoinDataSourceTest "x == \"j.x\"", JoinType.LEFT, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); + @Rule + public ExpectedException expectedException = ExpectedException.none(); + @Mock + private JoinableFactoryWrapper joinableFactoryWrapper; @Test public void test_getTableNames_tableToTable() @@ -149,13 +165,14 @@ public class JoinDataSourceTest EqualsVerifier.forClass(JoinDataSource.class) .usingGetClass() .withNonnullFields("left", "right", "rightPrefix", "conditionAnalysis", "joinType") + .withIgnoredFields("joinableFactoryWrapper", "analysis") .verify(); } @Test public void test_serde() throws Exception { - final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final ObjectMapper jsonMapper = TestHelper.makeJsonMapperForJoinable(joinableFactoryWrapper); JoinDataSource joinDataSource = JoinDataSource.create( new TableDataSource("table1"), new TableDataSource("table2"), @@ -163,7 +180,8 @@ public class JoinDataSourceTest "x == \"j.x\"", JoinType.LEFT, TrueDimFilter.instance(), - ExprMacroTable.nil() + ExprMacroTable.nil(), + joinableFactoryWrapper ); final JoinDataSource deserialized = (JoinDataSource) jsonMapper.readValue( @@ -186,7 +204,8 @@ public class JoinDataSourceTest "x == \"j.x\"", JoinType.LEFT, TrueDimFilter.instance(), - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); } @@ -200,7 +219,8 @@ public class JoinDataSourceTest "x == \"j.x\"", JoinType.LEFT, TrueDimFilter.instance(), - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); Assert.assertEquals(TrueDimFilter.instance(), dataSource.getLeftFilter()); } @@ -215,8 +235,275 @@ public class JoinDataSourceTest "x == \"j.x\"", JoinType.LEFT, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); Assert.assertEquals(dataSource.getVirtualColumnCandidates(), ImmutableSet.of("x")); } + + @Test + public void test_computeJoinDataSourceCacheKey_noHashJoin() + { + JoinDataSource dataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + NOOP_JOINABLE_FACTORY_WRAPPER + ); + + Optional cacheKey = Optional.ofNullable(dataSource.getCacheKey()); + + Assert.assertTrue(cacheKey.isPresent()); + } + + @Test + public void test_computeJoinDataSourceCacheKey_sameKeyForSameJoin() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertTrue(Arrays.equals(cacheKey1, cacheKey2)); + + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithTables() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table11"), + new TableDataSource("table12"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithExpressions() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "y == \"j.y\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinType() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.INNER, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithPrefix() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "ab.", + "x == \"ab.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithBaseFilter() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + NullHandling.initializeForTests(); + final InDimFilter expectedInDimFilter = new InDimFilter("dimTest", Arrays.asList("good", "bad"), null); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + expectedInDimFilter, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_cachingUnsupported() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new LookupDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + Assert.assertEquals(cacheKey1.length, 0); + } + + + private static class JoinableFactoryWithCacheKey extends NoopJoinableFactory + { + @Override + public Optional computeJoinCacheKey(DataSource dataSource, JoinConditionAnalysis condition) + { + if (dataSource.isCacheable(false) && condition.canHashJoin()) { + String tableName = Iterators.getOnlyElement(dataSource.getTableNames().iterator()); + return Optional.of(StringUtils.toUtf8(tableName)); + } + return Optional.empty(); + } + } } diff --git a/processing/src/test/java/org/apache/druid/query/QueriesTest.java b/processing/src/test/java/org/apache/druid/query/QueriesTest.java index 210314b636b..b83e256d7a8 100644 --- a/processing/src/test/java/org/apache/druid/query/QueriesTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueriesTest.java @@ -425,7 +425,8 @@ public class QueriesTest "\"foo.x\" == \"bar.x\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) .intervals("2000/3000") @@ -462,7 +463,8 @@ public class QueriesTest "\"foo.x\" == \"bar.x\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) .intervals("2000/3000") @@ -500,14 +502,16 @@ public class QueriesTest "\"foo.x\" == \"bar.x\"", JoinType.INNER, TrueDimFilter.instance(), - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ), new TableDataSource("foo_outer"), "j0.", "\"foo_outer.x\" == \"bar.x\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) @@ -533,14 +537,16 @@ public class QueriesTest "\"foo.x\" == \"bar.x\"", JoinType.INNER, TrueDimFilter.instance(), - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ), new TableDataSource("foo_outer"), "j0.", "\"foo_outer.x\" == \"bar.x\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) diff --git a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java index aab7d0e0baa..4ea43979353 100644 --- a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java +++ b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java @@ -612,7 +612,8 @@ public class DataSourceAnalysisTest joinClause(rightPrefix).getOriginalExpression(), joinType, dimFilter, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); } diff --git a/processing/src/test/java/org/apache/druid/segment/TestHelper.java b/processing/src/test/java/org/apache/druid/segment/TestHelper.java index 4ee90243ddd..525fb61b7c3 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -42,6 +42,7 @@ import org.apache.druid.query.topn.TopNResultValue; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.data.ComparableList; import org.apache.druid.segment.data.ComparableStringArray; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.junit.Assert; @@ -95,9 +96,10 @@ public class TestHelper public static ObjectMapper makeJsonMapper() { final ObjectMapper mapper = new DefaultObjectMapper(); - AnnotationIntrospector introspector = makeAnnotationIntrospector(); + final AnnotationIntrospector introspector = makeAnnotationIntrospector(); DruidSecondaryModule.setupAnnotationIntrospector(mapper, introspector); + mapper.setInjectableValues( new InjectableValues.Std() .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) @@ -107,6 +109,23 @@ public class TestHelper return mapper; } + public static ObjectMapper makeJsonMapperForJoinable(JoinableFactoryWrapper joinableFactoryWrapper) + { + final ObjectMapper mapper = new DefaultObjectMapper(); + AnnotationIntrospector introspector = makeAnnotationIntrospector(); + DruidSecondaryModule.setupAnnotationIntrospector(mapper, introspector); + + + mapper.setInjectableValues( + new InjectableValues.Std() + .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) + .addValue(ObjectMapper.class.getName(), mapper) + .addValue(PruneSpecsHolder.class, PruneSpecsHolder.DEFAULT) + .addValue(JoinableFactoryWrapper.class, joinableFactoryWrapper) + ); + return mapper; + } + public static ObjectMapper makeSmileMapper() { final ObjectMapper mapper = new DefaultObjectMapper(); diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java index 8f5b699ee7f..3d8a01785a4 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java @@ -22,34 +22,17 @@ package org.apache.druid.segment.join; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterators; import com.google.common.collect.Sets; import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.config.NullHandlingTest; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.query.DataSource; -import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.InlineDataSource; -import org.apache.druid.query.LookupDataSource; -import org.apache.druid.query.QueryContexts; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.TestQuery; import org.apache.druid.query.extraction.MapLookupExtractor; -import org.apache.druid.query.filter.FalseDimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; -import org.apache.druid.query.filter.TrueDimFilter; -import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.query.planning.PreJoinableClause; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; -import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.FalseFilter; @@ -57,24 +40,17 @@ import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.join.table.RowBasedIndexedTable; -import org.apache.druid.timeline.SegmentId; -import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; -import java.io.IOException; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Function; import java.util.stream.Collectors; public class JoinableFactoryWrapperTest extends NullHandlingTest @@ -85,12 +61,12 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest private static final Map TEST_LOOKUP = ImmutableMap.builder() - .put("MX", "Mexico") - .put("NO", "Norway") - .put("SV", "El Salvador") - .put("US", "United States") - .put("", "Empty key") - .build(); + .put("MX", "Mexico") + .put("NO", "Norway") + .put("SV", "El Salvador") + .put("US", "United States") + .put("", "Empty key") + .build(); private static final Set TEST_LOOKUP_KEYS = NullHandling.sqlCompatible() @@ -137,340 +113,6 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest @Rule public ExpectedException expectedException = ExpectedException.none(); - @Test - public void test_createSegmentMapFn_noClauses() - { - final Function segmentMapFn = NOOP_JOINABLE_FACTORY_WRAPPER.createSegmentMapFn( - null, - ImmutableList.of(), - new AtomicLong(), - null - ); - - Assert.assertSame(Function.identity(), segmentMapFn); - } - - @Test - public void test_createSegmentMapFn_unusableClause() - { - final PreJoinableClause clause = makePreJoinableClause( - INDEXED_TABLE_DS, - "country == \"j.country\"", - "j.", - JoinType.LEFT - ); - - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("dataSource is not joinable"); - - final Function ignored = NOOP_JOINABLE_FACTORY_WRAPPER.createSegmentMapFn( - null, - ImmutableList.of(clause), - new AtomicLong(), - null - ); - } - - @Test - public void test_createSegmentMapFn_usableClause() - { - final PreJoinableClause clause = makePreJoinableClause( - INDEXED_TABLE_DS, - "country == \"j.country\"", - "j.", - JoinType.LEFT - ); - - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new InlineJoinableFactory()); - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - null, - ImmutableList.of(clause), - new AtomicLong(), - new TestQuery( - new TableDataSource("test"), - new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, - new HashMap<>() - ) - ); - - Assert.assertNotSame(Function.identity(), segmentMapFn); - } - - @Test - public void test_createSegmentMapFn_usableClause_joinToFilterEnabled() throws IOException - { - final PreJoinableClause clause = makePreJoinableClause( - INDEXED_TABLE_DS, - "country == \"j.country\"", - "j.", - JoinType.INNER - ); - // required columns are necessary for the rewrite - final TestQuery queryWithRequiredColumnsAndJoinFilterRewrite = (TestQuery) new TestQuery( - new TableDataSource("test"), - new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, - new HashMap<>() - ).withOverriddenContext(ImmutableMap.of(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, "true")); - queryWithRequiredColumnsAndJoinFilterRewrite.setRequiredColumns(ImmutableSet.of("country")); - - final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new InlineJoinableFactory()); - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - null, - ImmutableList.of(clause), - new AtomicLong(), - queryWithRequiredColumnsAndJoinFilterRewrite - ); - - // dummy segment - final SegmentReference baseSegmentReference = ReferenceCountingSegment.wrapRootGenerationSegment( - new QueryableIndexSegment( - JoinTestHelper.createFactIndexBuilder(temporaryFolder.newFolder()).buildMMappedIndex(), - SegmentId.dummy("facts") - ) - ); - - // check the output contains the conversion filter - Assert.assertNotSame(Function.identity(), segmentMapFn); - final SegmentReference joinSegmentReference = segmentMapFn.apply(baseSegmentReference); - Assert.assertTrue(joinSegmentReference instanceof HashJoinSegment); - HashJoinSegment hashJoinSegment = (HashJoinSegment) joinSegmentReference; - Assert.assertEquals( - hashJoinSegment.getBaseFilter(), - new InDimFilter( - "country", - INDEXED_TABLE_DS.getRowsAsList().stream().map(row -> row[0].toString()).collect(Collectors.toSet()) - ) - ); - // the returned clause list is not comparable with an expected clause list since the Joinable - // class member in JoinableClause doesn't implement equals method in its implementations - Assert.assertEquals(hashJoinSegment.getClauses().size(), 1); - } - - @Test - public void test_computeJoinDataSourceCacheKey_noClauses() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - DataSource dataSource = new NoopDataSource(); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.emptyList()); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()); - EasyMock.expect(analysis.getDataSource()).andReturn(dataSource); - EasyMock.replay(analysis); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - expectedException.expect(IAE.class); - expectedException.expectMessage(StringUtils.format( - "No join clauses to build the cache key for data source [%s]", - dataSource - )); - joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - } - - @Test - public void test_computeJoinDataSourceCacheKey_noHashJoin() - { - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x != \"h.x\"", "h."); - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes(); - EasyMock.replay(analysis); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - Optional cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - - Assert.assertFalse(cacheKey.isPresent()); - } - - @Test - public void test_computeJoinDataSourceCacheKey_cachingUnsupported() - { - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - DataSource dataSource = new LookupDataSource("lookup"); - PreJoinableClause clause2 = makePreJoinableClause(dataSource, "x == \"h.x\"", "h.", JoinType.LEFT); - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes(); - EasyMock.replay(analysis); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - Optional cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - - Assert.assertFalse(cacheKey.isPresent()); - } - - @Test - public void test_computeJoinDataSourceCacheKey_usableClauses() - { - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"h.x\"", "h."); - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - Optional cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - - Assert.assertTrue(cacheKey.isPresent()); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithExpression() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "y == \"j.y\"", "j."); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinType() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.LEFT); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.INNER); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithPrefix() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "xy"); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithBaseFilter() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(FalseDimFilter.instance())).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinable() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"j.x\"", "j."); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_sameKeyForSameJoin() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertArrayEquals(cacheKey1.get(), cacheKey2.get()); - } - @Test public void test_checkClausePrefixesForDuplicatesAndShadowing_noConflicts() { @@ -563,10 +205,12 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest Assert.assertEquals( Pair.of( ImmutableList.of(new InDimFilter( - "x", - INDEXED_TABLE_DS.getRowsAsList().stream().map(row -> row[0].toString()).collect(Collectors.toSet())) + "x", + INDEXED_TABLE_DS.getRowsAsList().stream().map(row -> row[0].toString()).collect(Collectors.toSet()) + ) ), - ImmutableList.of(joinableClause) // the joinable clause remains intact since we've duplicates in country column + ImmutableList.of(joinableClause) + // the joinable clause remains intact since we've duplicates in country column ), conversion ); @@ -946,53 +590,4 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest conversion ); } - - private PreJoinableClause makeGlobalPreJoinableClause(String tableName, String expression, String prefix) - { - return makeGlobalPreJoinableClause(tableName, expression, prefix, JoinType.LEFT); - } - - private PreJoinableClause makeGlobalPreJoinableClause( - String tableName, - String expression, - String prefix, - JoinType joinType - ) - { - GlobalTableDataSource dataSource = new GlobalTableDataSource(tableName); - return makePreJoinableClause(dataSource, expression, prefix, joinType); - } - - private PreJoinableClause makePreJoinableClause( - DataSource dataSource, - String expression, - String prefix, - JoinType joinType - ) - { - JoinConditionAnalysis conditionAnalysis = JoinConditionAnalysis.forExpression( - expression, - prefix, - ExprMacroTable.nil() - ); - return new PreJoinableClause( - prefix, - dataSource, - joinType, - conditionAnalysis - ); - } - - private static class JoinableFactoryWithCacheKey extends NoopJoinableFactory - { - @Override - public Optional computeJoinCacheKey(DataSource dataSource, JoinConditionAnalysis condition) - { - if (dataSource.isCacheable(false) && condition.canHashJoin()) { - String tableName = Iterators.getOnlyElement(dataSource.getTableNames().iterator()); - return Optional.of(StringUtils.toUtf8(tableName)); - } - return Optional.empty(); - } - } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java index 6ad1cb1f891..b6eb168b624 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java @@ -20,9 +20,13 @@ package org.apache.druid.segment.join; import org.apache.druid.query.DataSource; +import org.apache.druid.query.Query; +import org.apache.druid.segment.SegmentReference; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; /** * A datasource that returns nothing. Only used to test un-registered datasources. @@ -64,4 +68,25 @@ public class NoopDataSource implements DataSource { return false; } + + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTime + ) + { + return Function.identity(); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + + @Override + public byte[] getCacheKey() + { + return new byte[]{}; + } } diff --git a/server/src/main/java/org/apache/druid/client/CacheUtil.java b/server/src/main/java/org/apache/druid/client/CacheUtil.java index a5741ebd0db..42cfe171c7b 100644 --- a/server/src/main/java/org/apache/druid/client/CacheUtil.java +++ b/server/src/main/java/org/apache/druid/client/CacheUtil.java @@ -108,9 +108,9 @@ public class CacheUtil ServerType serverType ) { - return isQueryCacheable(query, cacheStrategy, cacheConfig, serverType) + return cacheConfig.isUseCache() && query.context().isUseCache() - && cacheConfig.isUseCache(); + && isQueryCacheable(query, cacheStrategy, cacheConfig, serverType); } /** diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 025d079ad33..de5fdc5db19 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -296,9 +296,7 @@ public class CachingClusteredClient implements QuerySegmentWalker query, strategy, useCache, - populateCache, - dataSourceAnalysis, - joinableFactoryWrapper + populateCache ); } @@ -761,24 +759,18 @@ public class CachingClusteredClient implements QuerySegmentWalker { private final Query query; private final CacheStrategy> strategy; - private final DataSourceAnalysis dataSourceAnalysis; - private final JoinableFactoryWrapper joinableFactoryWrapper; private final boolean isSegmentLevelCachingEnable; CacheKeyManager( final Query query, final CacheStrategy> strategy, final boolean useCache, - final boolean populateCache, - final DataSourceAnalysis dataSourceAnalysis, - final JoinableFactoryWrapper joinableFactoryWrapper + final boolean populateCache ) { this.query = query; this.strategy = strategy; - this.dataSourceAnalysis = dataSourceAnalysis; - this.joinableFactoryWrapper = joinableFactoryWrapper; this.isSegmentLevelCachingEnable = ((populateCache || useCache) && !query.context().isBySegment()); // explicit bySegment queries are never cached @@ -840,15 +832,14 @@ public class CachingClusteredClient implements QuerySegmentWalker private byte[] computeQueryCacheKeyWithJoin() { Preconditions.checkNotNull(strategy, "strategy cannot be null"); - if (dataSourceAnalysis.isJoin()) { - byte[] joinDataSourceCacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis) - .orElse(null); - if (null == joinDataSourceCacheKey) { - return null; // A join operation that does not support caching - } - return Bytes.concat(joinDataSourceCacheKey, strategy.computeCacheKey(query)); + byte[] dataSourceCacheKey = query.getDataSource().getCacheKey(); + if (null == dataSourceCacheKey) { + return null; + } else if (dataSourceCacheKey.length > 0) { + return Bytes.concat(dataSourceCacheKey, strategy.computeCacheKey(query)); + } else { + return strategy.computeCacheKey(query); } - return strategy.computeCacheKey(query); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index f9745c4bab6..389dfc48484 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -31,7 +31,6 @@ import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.FunctionalIterable; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -59,7 +58,6 @@ import org.apache.druid.query.spec.SpecificSegmentQueryRunner; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; @@ -170,17 +168,15 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker } // segmentMapFn maps each base Segment into a joined Segment if necessary. - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - cpuTimeAccumulator, - analysis.getBaseQuery().orElse(query) - ); + final Function segmentMapFn = analysis.getDataSource() + .createSegmentMapFunction( + query, + cpuTimeAccumulator + ); + // We compute the join cache key here itself so it doesn't need to be re-computed for every segment - final Optional cacheKeyPrefix = analysis.isJoin() - ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) - : Optional.of(StringUtils.EMPTY_BYTES); + final Optional cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey()); Iterable> perSegmentRunners = Iterables.transform( specs, diff --git a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java index 8ed289b8bce..730eb3121d4 100644 --- a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java @@ -36,7 +36,6 @@ import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SegmentWrangler; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.joda.time.Interval; @@ -94,12 +93,13 @@ public class LocalQuerySegmentWalker implements QuerySegmentWalker final AtomicLong cpuAccumulator = new AtomicLong(0L); - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - cpuAccumulator, - analysis.getBaseQuery().orElse(query) - ); + final Function segmentMapFn = analysis + .getDataSource() + .createSegmentMapFunction( + query, + cpuAccumulator + ); + final QueryRunnerFactory> queryRunnerFactory = conglomerate.findFactory(query); final QueryRunner baseRunner = queryRunnerFactory.mergeRunners( diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index 9a31000b468..26aabc2ddd6 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -59,7 +59,6 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SetAndVerifyContextQueryRunner; @@ -195,18 +194,10 @@ public class ServerManager implements QuerySegmentWalker } else { return new ReportTimelineMissingSegmentQueryRunner<>(Lists.newArrayList(specs)); } + final Function segmentMapFn = query.getDataSource().createSegmentMapFunction(query, cpuTimeAccumulator); - // segmentMapFn maps each base Segment into a joined Segment if necessary. - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - cpuTimeAccumulator, - analysis.getBaseQuery().orElse(query) - ); // We compute the join cache key here itself so it doesn't need to be re-computed for every segment - final Optional cacheKeyPrefix = analysis.isJoin() - ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) - : Optional.of(StringUtils.EMPTY_BYTES); + final Optional cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey()); final FunctionalIterable> queryRunners = FunctionalIterable .create(specs) diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientCacheKeyManagerTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientCacheKeyManagerTest.java index 14b757bfaf6..40980d48e3a 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientCacheKeyManagerTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientCacheKeyManagerTest.java @@ -25,11 +25,13 @@ import com.google.common.primitives.Bytes; import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.query.CacheStrategy; +import org.apache.druid.query.DataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.join.NoopDataSource; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -42,7 +44,6 @@ import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; -import java.util.Optional; import java.util.Set; import static org.easymock.EasyMock.expect; @@ -60,6 +61,8 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport private JoinableFactoryWrapper joinableFactoryWrapper; @Mock private DataSourceAnalysis dataSourceAnalysis; + @Mock + private DataSource dataSource; private static final byte[] QUERY_CACHE_KEY = new byte[]{1, 2, 3}; private static final byte[] JOIN_KEY = new byte[]{4, 5}; @@ -139,7 +142,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport @Test public void testComputeEtag_nonJoinDataSource() { - expect(dataSourceAnalysis.isJoin()).andReturn(false); + expect(query.getDataSource()).andReturn(new NoopDataSource()); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); Set selectors = ImmutableSet.of( @@ -161,8 +164,8 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport @Test public void testComputeEtag_joinWithUnsupportedCaching() { - expect(dataSourceAnalysis.isJoin()).andReturn(true); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.empty()); + expect(query.getDataSource()).andReturn(dataSource); + expect(dataSource.getCacheKey()).andReturn(null); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); Set selectors = ImmutableSet.of( @@ -173,38 +176,15 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport Assert.assertNull(actual); } - @Test - public void testComputeEtag_joinWithSupportedCaching() - { - expect(dataSourceAnalysis.isJoin()).andReturn(true).anyTimes(); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(JOIN_KEY)); - replayAll(); - CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); - Set selectors = ImmutableSet.of( - makeHistoricalServerSelector(1), - makeHistoricalServerSelector(1) - ); - String actual1 = keyManager.computeResultLevelCachingEtag(selectors, null); - Assert.assertNotNull(actual1); - - reset(joinableFactoryWrapper); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(new byte[]{9})); - replay(joinableFactoryWrapper); - selectors = ImmutableSet.of( - makeHistoricalServerSelector(1), - makeHistoricalServerSelector(1) - ); - String actual2 = keyManager.computeResultLevelCachingEtag(selectors, null); - Assert.assertNotNull(actual2); - Assert.assertNotEquals(actual1, actual2); - } - @Test public void testComputeEtag_noEffectifBySegment() { expect(dataSourceAnalysis.isJoin()).andReturn(false); reset(query); + + expect(query.getDataSource()).andReturn(new NoopDataSource()); expect(query.context()).andReturn(QueryContext.of(ImmutableMap.of(QueryContexts.BY_SEGMENT_KEY, true))).anyTimes(); + replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); Set selectors = ImmutableSet.of( @@ -219,14 +199,13 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport public void testComputeEtag_noEffectIfUseAndPopulateFalse() { expect(dataSourceAnalysis.isJoin()).andReturn(false); + expect(query.getDataSource()).andReturn(new NoopDataSource()); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = new CachingClusteredClient.CacheKeyManager<>( query, strategy, false, - false, - dataSourceAnalysis, - joinableFactoryWrapper + false ); Set selectors = ImmutableSet.of( makeHistoricalServerSelector(1), @@ -239,7 +218,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport @Test public void testSegmentQueryCacheKey_nonJoinDataSource() { - expect(dataSourceAnalysis.isJoin()).andReturn(false); + expect(query.getDataSource()).andReturn(new NoopDataSource()); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey(); @@ -249,20 +228,18 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport @Test public void testSegmentQueryCacheKey_joinWithUnsupportedCaching() { - expect(dataSourceAnalysis.isJoin()).andReturn(true); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.empty()); + expect(query.getDataSource()).andReturn(new NoopDataSource()); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey(); - Assert.assertNull(cacheKey); + Assert.assertNotNull(cacheKey); } @Test public void testSegmentQueryCacheKey_joinWithSupportedCaching() { - - expect(dataSourceAnalysis.isJoin()).andReturn(true); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(JOIN_KEY)); + expect(query.getDataSource()).andReturn(dataSource); + expect(dataSource.getCacheKey()).andReturn(JOIN_KEY); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey(); @@ -287,9 +264,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport query, strategy, false, - false, - dataSourceAnalysis, - joinableFactoryWrapper + false ).computeSegmentLevelQueryCacheKey()); } @@ -299,9 +274,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport query, strategy, true, - true, - dataSourceAnalysis, - joinableFactoryWrapper + true ); } diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index e19dfcf189a..5e0701680cd 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -127,6 +127,7 @@ public class ClientQuerySegmentWalkerTest private static final String VERSION = "A"; private static final ShardSpec SHARD_SPEC = new NumberedShardSpec(0, 1); + private static final InlineDataSource FOO_INLINE = InlineDataSource.fromIterable( ImmutableList.builder() .add(new Object[]{INTERVAL.getStartMillis(), "x", 1}) @@ -502,7 +503,11 @@ public class ClientQuerySegmentWalkerTest "\"j.s\" == \"s\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + new JoinableFactoryWrapper(QueryStackTests.makeJoinableFactoryFromDefault( + null, + null, + null)) ) ) .setGranularity(Granularities.ALL) @@ -570,7 +575,11 @@ public class ClientQuerySegmentWalkerTest "\"j.s\" == \"s\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + new JoinableFactoryWrapper(QueryStackTests.makeJoinableFactoryFromDefault( + null, + null, + null)) ) ) .setGranularity(Granularities.ALL) @@ -749,7 +758,8 @@ public class ClientQuerySegmentWalkerTest "\"j.s\" == \"s\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) .setGranularity(Granularities.ALL) diff --git a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java index 673a93be12c..2245b28169a 100644 --- a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java @@ -45,7 +45,6 @@ import org.apache.druid.query.spec.SpecificSegmentQueryRunner; import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentReference; -import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -138,11 +137,9 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker throw new ISE("Cannot handle subquery: %s", analysis.getDataSource()); } - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - new AtomicLong(), - analysis.getBaseQuery().orElse(query) + final Function segmentMapFn = analysis.getDataSource().createSegmentMapFunction( + query, + new AtomicLong() ); final QueryRunner baseRunner = new FinalizeResultsQueryRunner<>( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index f32c5c1498d..bd2c8cb7380 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -27,12 +27,16 @@ import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputSource; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.DataSource; +import org.apache.druid.query.Query; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.column.RowSignature; import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; /** * Represents external data for INSERT queries. Only used by the SQL layer, not by the query stack. @@ -119,6 +123,27 @@ public class ExternalDataSource implements DataSource return false; } + @Override + public Function createSegmentMapFunction( + Query query, + AtomicLong cpuTime + ) + { + return Function.identity(); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + + @Override + public byte[] getCacheKey() + { + return null; + } + @Override public boolean equals(Object o) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 797bec1fe96..b683e6937dc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.ResourceAction; @@ -48,7 +49,6 @@ import org.joda.time.DateTimeZone; import org.joda.time.Interval; import javax.annotation.Nullable; - import java.util.Collections; import java.util.List; import java.util.Map; @@ -78,6 +78,7 @@ public class PlannerContext private final String sql; private final DruidOperatorTable operatorTable; private final ExprMacroTable macroTable; + private final JoinableFactoryWrapper joinableFactoryWrapper; private final ObjectMapper jsonMapper; private final PlannerConfig plannerConfig; private final DateTime localNow; @@ -112,7 +113,8 @@ public class PlannerContext final DruidSchemaCatalog rootSchema, final SqlEngine engine, final Map queryContext, - final Set contextKeys + final Set contextKeys, + final JoinableFactoryWrapper joinableFactoryWrapper ) { this.sql = sql; @@ -125,6 +127,7 @@ public class PlannerContext this.queryContext = queryContext; this.localNow = Preconditions.checkNotNull(localNow, "localNow"); this.stringifyArrays = stringifyArrays; + this.joinableFactoryWrapper = joinableFactoryWrapper; String sqlQueryId = (String) this.queryContext.get(QueryContexts.CTX_SQL_QUERY_ID); // special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle @@ -143,7 +146,8 @@ public class PlannerContext final DruidSchemaCatalog rootSchema, final SqlEngine engine, final Map queryContext, - final Set contextKeys + final Set contextKeys, + final JoinableFactoryWrapper joinableFactoryWrapper ) { final DateTime utcNow; @@ -183,7 +187,8 @@ public class PlannerContext rootSchema, engine, queryContext, - contextKeys + contextKeys, + joinableFactoryWrapper ); } @@ -217,6 +222,11 @@ public class PlannerContext return localNow.getZone(); } + public JoinableFactoryWrapper getJoinableFactoryWrapper() + { + return joinableFactoryWrapper; + } + @Nullable public String getSchemaResourceType(String schema, String resourceName) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java index 742d98ac56e..1b0c7c9c352 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java @@ -39,6 +39,7 @@ import org.apache.calcite.tools.Frameworks; import org.apache.calcite.tools.ValidationException; import org.apache.druid.guice.annotations.Json; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.NoopEscalator; @@ -72,6 +73,7 @@ public class PlannerFactory private final AuthorizerMapper authorizerMapper; private final String druidSchemaName; private final CalciteRulesManager calciteRuleManager; + private final JoinableFactoryWrapper joinableFactoryWrapper; @Inject public PlannerFactory( @@ -82,7 +84,8 @@ public class PlannerFactory final AuthorizerMapper authorizerMapper, final @Json ObjectMapper jsonMapper, final @DruidSchemaName String druidSchemaName, - final CalciteRulesManager calciteRuleManager + final CalciteRulesManager calciteRuleManager, + final JoinableFactoryWrapper joinableFactoryWrapper ) { this.rootSchema = rootSchema; @@ -93,6 +96,7 @@ public class PlannerFactory this.jsonMapper = jsonMapper; this.druidSchemaName = druidSchemaName; this.calciteRuleManager = calciteRuleManager; + this.joinableFactoryWrapper = joinableFactoryWrapper; } /** @@ -114,7 +118,8 @@ public class PlannerFactory rootSchema, engine, queryContext, - contextKeys + contextKeys, + joinableFactoryWrapper ); return new DruidPlanner(buildFrameworkConfig(context), context, engine); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java index 137803bb46e..9ff0a3cf1ba 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java @@ -195,7 +195,8 @@ public class DruidJoinQueryRel extends DruidRel condition.getExpression(), toDruidJoinType(joinRel.getJoinType()), getDimFilter(getPlannerContext(), leftSignature, leftFilter), - getPlannerContext().getExprMacroTable() + getPlannerContext().getExprMacroTable(), + getPlannerContext().getJoinableFactoryWrapper() ), prefixSignaturePair.rhs, getPlannerContext(), diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 14197005d7a..8c5aa588f70 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -81,6 +81,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.DimensionExpression; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -748,7 +749,8 @@ public class DruidQuery static Pair getFiltration( DataSource dataSource, DimFilter filter, - VirtualColumnRegistry virtualColumnRegistry + VirtualColumnRegistry virtualColumnRegistry, + JoinableFactoryWrapper joinableFactoryWrapper ) { if (!(dataSource instanceof JoinDataSource)) { @@ -768,13 +770,16 @@ public class DruidQuery // Adds the intervals from the join left filter to query filtration Filtration queryFiltration = Filtration.create(filter, leftFiltration.getIntervals()) .optimize(virtualColumnRegistry.getFullRowSignature()); + + JoinDataSource newDataSource = JoinDataSource.create( joinDataSource.getLeft(), joinDataSource.getRight(), joinDataSource.getRightPrefix(), joinDataSource.getConditionAnalysis(), joinDataSource.getJoinType(), - leftFiltration.getDimFilter() + leftFiltration.getDimFilter(), + joinableFactoryWrapper ); return Pair.of(newDataSource, queryFiltration); } @@ -946,7 +951,8 @@ public class DruidQuery final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; @@ -1056,7 +1062,8 @@ public class DruidQuery final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; @@ -1154,7 +1161,8 @@ public class DruidQuery final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; @@ -1199,7 +1207,8 @@ public class DruidQuery final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; @@ -1324,7 +1333,8 @@ public class DruidQuery final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; diff --git a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java index 2311bbba433..fe3edecb109 100644 --- a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java @@ -35,6 +35,7 @@ import org.apache.druid.query.DefaultQueryConfig; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.initialization.ServerConfig; @@ -144,6 +145,7 @@ public class SqlStatementTest final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); testRequestLogger = new TestRequestLogger(); + final JoinableFactoryWrapper joinableFactoryWrapper = CalciteTests.createJoinableFactoryWrapper(); final PlannerFactory plannerFactory = new PlannerFactory( rootSchema, @@ -153,7 +155,8 @@ public class SqlStatementTest CalciteTests.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + joinableFactoryWrapper ); this.sqlStatementFactory = new SqlStatementFactory( diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java index 9cf65fe1966..ab9ca2547f6 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -54,6 +54,7 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DefaultQueryConfig; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryScheduler; @@ -289,6 +290,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase binder.install(new SqlModule.SqlStatementFactoryModule()); binder.bind(new TypeLiteral>(){}).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of()))); binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of())); + binder.bind(JoinableFactoryWrapper.class).toInstance(CalciteTests.createJoinableFactoryWrapper()); } ) .build(); @@ -984,7 +986,8 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ) ); } @@ -999,6 +1002,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase final List frames = new ArrayList<>(); final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame"); + DruidMeta smallFrameDruidMeta = new DruidMeta( makeStatementFactory(), config, @@ -1058,6 +1062,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase final List frames = new ArrayList<>(); final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame"); + DruidMeta smallFrameDruidMeta = new DruidMeta( makeStatementFactory(), config, diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java index f9b0718d1e1..05a3676c304 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AllowAllAuthenticator; import org.apache.druid.server.security.AuthTestUtils; @@ -103,6 +104,7 @@ public class DruidStatementTest extends CalciteTestBase final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); + final JoinableFactoryWrapper joinableFactoryWrapper = CalciteTests.createJoinableFactoryWrapper(); final PlannerFactory plannerFactory = new PlannerFactory( rootSchema, operatorTable, @@ -111,7 +113,8 @@ public class DruidStatementTest extends CalciteTestBase AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + joinableFactoryWrapper ); this.sqlStatementFactory = CalciteTests.createSqlStatementFactory( CalciteTests.createMockSqlEngine(walker, conglomerate), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index da9218cb90d..13b13dc9b2b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -106,7 +106,6 @@ import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import javax.annotation.Nullable; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -420,7 +419,8 @@ public class BaseCalciteQueryTest extends CalciteTestBase implements QueryCompon condition, joinType, filter, - CalciteTests.createExprMacroTable() + CalciteTests.createExprMacroTable(), + CalciteTests.createJoinableFactoryWrapper() ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index 88a5ec3fa20..801b4c8b36e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -142,7 +142,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest "(\"_d0\" == \"j0.d0\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .columns("_d0") @@ -199,7 +200,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest "(\"dim4\" == \"j0._d0\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .context(context) @@ -3248,7 +3250,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest "(\"dim2\" == \"j0.dim2\")", JoinType.LEFT, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .setInterval(querySegmentSpec(Filtration.eternity())) @@ -5088,7 +5091,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest "(\"l1\" == \"j0.ROW_VALUE\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .columns("l1") diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 6ff6e06bd93..badb943117f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -3944,7 +3944,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest "(\"dim1\" == \"j0.ROW_VALUE\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -13090,7 +13091,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest "(\"__time\" == \"j0.__time\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() )) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java index 5124ce64edb..4c3d70cc477 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.sql.calcite.planner.CalciteRulesManager; @@ -140,6 +141,7 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe final PlannerConfig plannerConfig = new PlannerConfig(); final DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema(CONGLOMERATE, WALKER, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); + final JoinableFactoryWrapper joinableFactoryWrapper = CalciteTests.createJoinableFactoryWrapper(); ENGINE = CalciteTests.createMockSqlEngine(WALKER, CONGLOMERATE); PLANNER_FACTORY = new PlannerFactory( rootSchema, @@ -149,7 +151,8 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + joinableFactoryWrapper ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java index e6f7d1519d1..340540d89ec 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.VirtualizedColumnSelectorFactory; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.DruidTypeSystem; @@ -62,7 +63,6 @@ import org.joda.time.DateTimeZone; import org.junit.Assert; import javax.annotation.Nullable; - import java.math.BigDecimal; import java.util.Arrays; import java.util.Collections; @@ -74,6 +74,7 @@ import java.util.stream.Collectors; class ExpressionTestHelper { + private static final JoinableFactoryWrapper JOINABLE_FACTORY_WRAPPER = CalciteTests.createJoinableFactoryWrapper(); private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create( "SELECT 1", // The actual query isn't important for this test CalciteTests.createOperatorTable(), @@ -89,7 +90,8 @@ class ExpressionTestHelper ), null /* Don't need engine */, Collections.emptyMap(), - Collections.emptySet() + Collections.emptySet(), + JOINABLE_FACTORY_WRAPPER ); private final RowSignature rowSignature; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java index 1a84d2080e5..59612da6b85 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java @@ -64,7 +64,8 @@ public class ExternalTableScanRuleTest ), engine, Collections.emptyMap(), - Collections.emptySet() + Collections.emptySet(), + CalciteTests.createJoinableFactoryWrapper() ); plannerContext.setQueryMaker( engine.buildQueryMakerForSelect(EasyMock.createMock(RelRoot.class), plannerContext) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java index 37410727fea..932d8bac0f3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java @@ -34,6 +34,7 @@ import org.apache.druid.guice.LazySingleton; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.jackson.JacksonModule; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ResourceType; @@ -54,7 +55,6 @@ import org.junit.runner.RunWith; import javax.validation.Validation; import javax.validation.Validator; - import java.util.Collections; import java.util.Set; @@ -81,6 +81,8 @@ public class CalcitePlannerModuleTest extends CalciteTestBase @Mock private ExprMacroTable macroTable; @Mock + private JoinableFactoryWrapper joinableFactoryWrapper; + @Mock private AuthorizerMapper authorizerMapper; @Mock private DruidSchemaCatalog rootSchema; @@ -125,6 +127,7 @@ public class CalcitePlannerModuleTest extends CalciteTestBase binder.bind(Key.get(new TypeLiteral>() {})).toInstance(aggregators); binder.bind(Key.get(new TypeLiteral>() {})).toInstance(operatorConversions); binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema); + binder.bind(JoinableFactoryWrapper.class).toInstance(joinableFactoryWrapper); }, target, binder -> { @@ -176,7 +179,8 @@ public class CalcitePlannerModuleTest extends CalciteTestBase rootSchema, null, Collections.emptyMap(), - Collections.emptySet() + Collections.emptySet(), + joinableFactoryWrapper ); boolean containsCustomRule = injector.getInstance(CalciteRulesManager.class) .druidConventionRuleSet(context) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java index 216634796b6..c1d3842e0d2 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java @@ -97,7 +97,8 @@ public class DruidRexExecutorTest extends InitializedNullHandlingTest ), null /* Don't need an engine */, Collections.emptyMap(), - Collections.emptySet() + Collections.emptySet(), + CalciteTests.createJoinableFactoryWrapper() ); private final RexBuilder rexBuilder = new RexBuilder(new JavaTypeFactoryImpl()); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java index 68edf3f1f28..844569d4ab6 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java @@ -35,6 +35,7 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinType; import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.util.CalciteTests; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -59,10 +60,12 @@ public class DruidQueryTest public void test_filtration_noJoinAndInterval() { DataSource dataSource = new TableDataSource("test"); + Pair pair = DruidQuery.getFiltration( dataSource, selectorFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, dataSource, selectorFilter, Intervals.ETERNITY); } @@ -74,7 +77,8 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, filterWithInterval, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200)); } @@ -86,7 +90,8 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, filterWithInterval, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200)); } @@ -99,7 +104,8 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); } @@ -112,7 +118,8 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); } @@ -125,7 +132,8 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); } @@ -138,7 +146,8 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); } @@ -156,7 +165,8 @@ public class DruidQueryTest Pair pair = DruidQuery.getFiltration( dataSource, queryFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(150, 200)); } @@ -170,7 +180,8 @@ public class DruidQueryTest "c == \"r.c\"", joinType, filter, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 40cbfb7c064..b51fd090285 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -283,6 +283,11 @@ public class CalciteTests return QueryFrameworkUtils.createExprMacroTable(INJECTOR); } + public static JoinableFactoryWrapper createJoinableFactoryWrapper() + { + return new JoinableFactoryWrapper(QueryFrameworkUtils.createDefaultJoinableFactory(INJECTOR)); + } + public static DruidOperatorTable createOperatorTable() { return QueryFrameworkUtils.createOperatorTable(INJECTOR); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java index e68c052637a..b250111e648 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SqlTestFramework.java @@ -373,7 +373,8 @@ public class SqlTestFramework authorizerMapper, queryJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); final SqlStatementFactory sqlStatementFactory = QueryFrameworkUtils.createSqlStatementFactory( engine, diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 10e9002c8b9..d3287530c55 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -222,6 +222,7 @@ public class SqlResourceTest extends CalciteTestBase ); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); + req = request(true); testRequestLogger = new TestRequestLogger(); @@ -234,7 +235,8 @@ public class SqlResourceTest extends CalciteTestBase CalciteTests.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); lifecycleManager = new SqlLifecycleManager()