Refactoring the data source before unnest (#13085)

* First set of changes for framework

* Second set of changes to move segment map function to data source

* Minot change to server manager

* Removing the createSegmentMapFunction from JoinableFactoryWrapper and moving to JoinDataSource

* Checkstyle fixes

* Patching Eric's fix for injection

* Checkstyle and fixing some CI issues

* Fixing code inspections and some failed tests and one injector for test in avatica

* Another set of changes for CI...almost there

* Equals and hashcode part update

* Fixing injector from Eric + refactoring for broadcastJoinHelper

* Updating second injector. Might revert later if better way found

* Fixing guice issue in JoinableFactory

* Addressing review comments part 1

* Temp changes refactoring

* Revert "Temp changes refactoring"

This reverts commit 9da42a9ef0.

* temp

* Temp discussions

* Refactoring temp

* Refatoring the query rewrite to refer to a datasource

* Refactoring getCacheKey by moving it inside data source

* Nullable annotation check in injector

* Addressing some comments, removing 2 analysis.isJoin() checks and correcting the benchmark files

* Minor changes for refactoring

* Addressing reviews part 1

* Refactoring part 2 with new test cases for broadcast join

* Set for nullables

* removing instance of checks

* Storing nullables in guice to avoid checking on reruns

* Fixing a test case and removing an irrelevant line

* Addressing the atomic reference review comments
This commit is contained in:
somu-imply 2022-10-26 15:58:58 -07:00 committed by GitHub
parent 72c16097ac
commit affc522b9f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
59 changed files with 1321 additions and 1025 deletions

View File

@ -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()
);
}

View File

@ -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 {

View File

@ -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 {

View File

@ -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()

View File

@ -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

View File

@ -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<HashSet<Key>> 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<Key> 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]",

View File

@ -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<Long>
@ -88,6 +89,58 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
this.broadcastJoinHelper = inputChannelsAndBroadcastJoinHelper.rhs;
}
/**
* Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up their primary and side channels.
*/
private static Pair<List<ReadableFrameChannel>, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper(
final DataSource dataSource,
final ReadableInput baseInput,
final Int2ObjectMap<ReadableInput> 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<ReadableFrameChannel> inputChannels = new ArrayList<>();
final BroadcastJoinHelper broadcastJoinHelper;
if (baseInput.hasChannel()) {
inputChannels.add(baseInput.getChannel());
}
if (dataSource instanceof JoinDataSource) {
final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap();
final List<FrameReader> 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<ReadableInput> 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<ReadableFrameChannel> inputChannels()
{
@ -146,6 +199,7 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
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<Long>
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<List<ReadableFrameChannel>, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper(
final DataSource dataSource,
final ReadableInput baseInput,
final Int2ObjectMap<ReadableInput> 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<ReadableFrameChannel> inputChannels = new ArrayList<>();
final BroadcastJoinHelper broadcastJoinHelper;
if (baseInput.hasChannel()) {
inputChannels.add(baseInput.getChannel());
}
if (dataSource instanceof JoinDataSource) {
final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap();
final List<FrameReader> 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<ReadableInput> 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);
}
}

View File

@ -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<SegmentReference, SegmentReference> 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();

View File

@ -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));

View File

@ -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<SegmentReference, SegmentReference> 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()
{

View File

@ -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<Object[]> 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()
{

View File

@ -189,6 +189,7 @@ public class BroadcastJoinHelperTest extends InitializedNullHandlingTest
"j.",
JoinConditionAnalysis.forExpression("x == \"j.x\"", "j.", ExprMacroTable.nil()),
JoinType.INNER,
null,
null
)
);

View File

@ -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<Set<NodeRole>>()
{
}).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<Set<NodeRole>>()
{
}).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"));

View File

@ -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<SegmentReference, SegmentReference> 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();
}

View File

@ -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.
*
* <p>
* 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<Object[]> rowsA, final Iterable<Object[]> rowsB)
{
if (rowsA instanceof List && rowsB instanceof List) {
final List<Object[]> listA = (List<Object[]>) rowsA;
final List<Object[]> listB = (List<Object[]>) 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<Object[]> rows)
{
if (rows instanceof List) {
final List<Object[]> list = (List<Object[]>) 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<String> getTableNames()
{
@ -182,6 +234,27 @@ public class InlineDataSource implements DataSource
return true;
}
@Override
public Function<SegmentReference, SegmentReference> 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<Object[]> rowsA, final Iterable<Object[]> rowsB)
{
if (rowsA instanceof List && rowsB instanceof List) {
final List<Object[]> listA = (List<Object[]>) rowsA;
final List<Object[]> listB = (List<Object[]>) 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<Object[]> rows)
{
if (rows instanceof List) {
final List<Object[]> list = (List<Object[]>) rows;
int code = 1;
for (final Object[] row : list) {
code = 31 * code + Arrays.hashCode(row);
}
return code;
} else {
return Objects.hash(rows);
}
}
}

View File

@ -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.
*
* <p>
* Logically, this datasource contains the result of:
*
* <p>
* (1) prefixing all right-side columns with "rightPrefix"
* (2) then, joining the left and (prefixed) right sides using the provided type and condition
*
* <p>
* 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.
* <p>
* 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<String> getTableNames()
{
@ -185,6 +231,12 @@ public class JoinDataSource implements DataSource
return leftFilter;
}
@Nullable
public JoinableFactoryWrapper getJoinableFactoryWrapper()
{
return joinableFactoryWrapper;
}
@Override
public List<DataSource> 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<String> 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<SegmentReference, SegmentReference> createSegmentMapFn(
@Nullable final Filter baseFilter,
final List<PreJoinableClause> 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<String> requiredColumns = query.getRequiredColumns();
final Filter baseFilterToUse;
final List<JoinableClause> clausesToUse;
if (requiredColumns != null && filterRewriteConfig.isEnableRewriteJoinToFilter()) {
final Pair<List<Filter>, List<JoinableClause>> 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<SegmentReference, SegmentReference> createSegmentMapFunction(
Query query,
AtomicLong cpuTimeAccumulator
)
{
final Function<SegmentReference, SegmentReference> 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<PreJoinableClause> 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<byte[]> 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();
}
}

View File

@ -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.
*
* <p>
* 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.
*
* <p>
* 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<SegmentReference, SegmentReference> 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)
{

View File

@ -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 <T> Query<T> withBaseDataSource(final Query<T> query, final DataSource newBaseDataSource)
{
final Query<T> 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));
}
/**

View File

@ -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<SegmentReference, SegmentReference> 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()
{

View File

@ -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<SegmentReference, SegmentReference> 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()
{

View File

@ -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<SegmentReference, SegmentReference> 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)
{

View File

@ -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.
*
* <p>
* 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<JoinableClause> getClauses()
{
return clauses;
}
}

View File

@ -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".
* <p>
* 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
{

View File

@ -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<SegmentReference, SegmentReference> createSegmentMapFn(
@Nullable final Filter baseFilter,
final List<PreJoinableClause> 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<String> requiredColumns = query.getRequiredColumns();
final Filter baseFilterToUse;
final List<JoinableClause> clausesToUse;
if (requiredColumns != null && filterRewriteConfig.isEnableRewriteJoinToFilter()) {
final Pair<List<Filter>, List<JoinableClause>> 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<byte[]> computeJoinDataSourceCacheKey(
final DataSourceAnalysis dataSourceAnalysis
)
{
final List<PreJoinableClause> 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<byte[]> 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.
*
* <p>
* See {@link #convertJoinToFilter} for details on the logic.
*/
@VisibleForTesting
static Pair<List<Filter>, List<JoinableClause>> convertJoinsToFilters(
public static Pair<List<Filter>, List<JoinableClause>> convertJoinsToFilters(
final List<JoinableClause> clauses,
final Set<String> requiredColumns,
final int maxNumFilterValues
@ -271,16 +115,16 @@ public class JoinableFactoryWrapper
/**
* Converts a join clause into an "in" filter if possible.
*
* <p>
* The requirements are:
*
* <p>
* - 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.
*
* <p>
* 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.

View File

@ -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<byte[]> 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<byte[]> 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();
}
}
}

View File

@ -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
)
)

View File

@ -612,7 +612,8 @@ public class DataSourceAnalysisTest
joinClause(rightPrefix).getOriginalExpression(),
joinType,
dimFilter,
ExprMacroTable.nil()
ExprMacroTable.nil(),
null
);
}

View File

@ -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();

View File

@ -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<String, String> TEST_LOOKUP =
ImmutableMap.<String, String>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<String> 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<SegmentReference, SegmentReference> 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<SegmentReference, SegmentReference> 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<SegmentReference, SegmentReference> 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<SegmentReference, SegmentReference> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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<byte[]> 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();
}
}
}

View File

@ -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<SegmentReference, SegmentReference> createSegmentMapFunction(
Query query,
AtomicLong cpuTime
)
{
return Function.identity();
}
@Override
public DataSource withUpdatedDataSource(DataSource newSource)
{
return newSource;
}
@Override
public byte[] getCacheKey()
{
return new byte[]{};
}
}

View File

@ -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);
}
/**

View File

@ -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<T> query;
private final CacheStrategy<T, Object, Query<T>> strategy;
private final DataSourceAnalysis dataSourceAnalysis;
private final JoinableFactoryWrapper joinableFactoryWrapper;
private final boolean isSegmentLevelCachingEnable;
CacheKeyManager(
final Query<T> query,
final CacheStrategy<T, Object, Query<T>> 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);
}
}

View File

@ -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<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn(
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null),
analysis.getPreJoinableClauses(),
cpuTimeAccumulator,
analysis.getBaseQuery().orElse(query)
);
final Function<SegmentReference, SegmentReference> 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<byte[]> cacheKeyPrefix = analysis.isJoin()
? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis)
: Optional.of(StringUtils.EMPTY_BYTES);
final Optional<byte[]> cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey());
Iterable<QueryRunner<T>> perSegmentRunners = Iterables.transform(
specs,

View File

@ -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<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn(
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null),
analysis.getPreJoinableClauses(),
cpuAccumulator,
analysis.getBaseQuery().orElse(query)
);
final Function<SegmentReference, SegmentReference> segmentMapFn = analysis
.getDataSource()
.createSegmentMapFunction(
query,
cpuAccumulator
);
final QueryRunnerFactory<T, Query<T>> queryRunnerFactory = conglomerate.findFactory(query);
final QueryRunner<T> baseRunner = queryRunnerFactory.mergeRunners(

View File

@ -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<SegmentReference, SegmentReference> segmentMapFn = query.getDataSource().createSegmentMapFunction(query, cpuTimeAccumulator);
// segmentMapFn maps each base Segment into a joined Segment if necessary.
final Function<SegmentReference, SegmentReference> 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<byte[]> cacheKeyPrefix = analysis.isJoin()
? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis)
: Optional.of(StringUtils.EMPTY_BYTES);
final Optional<byte[]> cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey());
final FunctionalIterable<QueryRunner<T>> queryRunners = FunctionalIterable
.create(specs)

View File

@ -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<Object> keyManager = makeKeyManager();
Set<SegmentServerSelector> 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<Object> keyManager = makeKeyManager();
Set<SegmentServerSelector> 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<Object> keyManager = makeKeyManager();
Set<SegmentServerSelector> 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<Object> keyManager = makeKeyManager();
Set<SegmentServerSelector> 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<Object> keyManager = new CachingClusteredClient.CacheKeyManager<>(
query,
strategy,
false,
false,
dataSourceAnalysis,
joinableFactoryWrapper
false
);
Set<SegmentServerSelector> 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<Object> 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<Object> 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<Object> 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
);
}

View File

@ -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.<Object[]>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)

View File

@ -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<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn(
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null),
analysis.getPreJoinableClauses(),
new AtomicLong(),
analysis.getBaseQuery().orElse(query)
final Function<SegmentReference, SegmentReference> segmentMapFn = analysis.getDataSource().createSegmentMapFunction(
query,
new AtomicLong()
);
final QueryRunner<T> baseRunner = new FinalizeResultsQueryRunner<>(

View File

@ -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<SegmentReference, SegmentReference> 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)
{

View File

@ -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<String, Object> queryContext,
final Set<String> contextKeys
final Set<String> 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<String, Object> queryContext,
final Set<String> contextKeys
final Set<String> 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)
{

View File

@ -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);

View File

@ -195,7 +195,8 @@ public class DruidJoinQueryRel extends DruidRel<DruidJoinQueryRel>
condition.getExpression(),
toDruidJoinType(joinRel.getJoinType()),
getDimFilter(getPlannerContext(), leftSignature, leftFilter),
getPlannerContext().getExprMacroTable()
getPlannerContext().getExprMacroTable(),
getPlannerContext().getJoinableFactoryWrapper()
),
prefixSignaturePair.rhs,
getPlannerContext(),

View File

@ -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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
dataSource,
filter,
virtualColumnRegistry
virtualColumnRegistry,
plannerContext.getJoinableFactoryWrapper()
);
final DataSource newDataSource = dataSourceFiltrationPair.lhs;
final Filtration filtration = dataSourceFiltrationPair.rhs;

View File

@ -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(

View File

@ -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<Supplier<DefaultQueryConfig>>(){}).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<Meta.Frame> 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<Meta.Frame> frames = new ArrayList<>();
final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame");
DruidMeta smallFrameDruidMeta = new DruidMeta(
makeStatementFactory(),
config,

View File

@ -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),

View File

@ -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()
);
}

View File

@ -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")

View File

@ -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)

View File

@ -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
);
}

View File

@ -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;

View File

@ -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)

View File

@ -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<Set<SqlAggregator>>() {})).toInstance(aggregators);
binder.bind(Key.get(new TypeLiteral<Set<SqlOperatorConversion>>() {})).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)

View File

@ -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());

View File

@ -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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> 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<DataSource, Filtration> 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()
);
}

View File

@ -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);

View File

@ -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,

View File

@ -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()