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 9da42a9ef05bbeefddfc62b4019227fd7c975f93.

* 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, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(), CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, 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, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(), CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()) new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
); );
try { try {

View File

@ -310,7 +310,8 @@ public class SqlNestedDataBenchmark
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(), CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()) new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
); );
try { try {

View File

@ -127,7 +127,8 @@ public class SqlVsNativeBenchmark
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(), CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()) new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
); );
groupByQuery = GroupByQuery groupByQuery = GroupByQuery
.builder() .builder()

View File

@ -22,7 +22,6 @@ package org.apache.druid.guice;
import com.fasterxml.jackson.databind.AnnotationIntrospector; import com.fasterxml.jackson.databind.AnnotationIntrospector;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair;
import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Binder; import com.google.inject.Binder;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.google.inject.Injector; import com.google.inject.Injector;
@ -96,14 +95,12 @@ public class DruidSecondaryModule implements Module
return smileMapper; return smileMapper;
} }
@VisibleForTesting
public static void setupJackson(Injector injector, final ObjectMapper mapper) public static void setupJackson(Injector injector, final ObjectMapper mapper)
{ {
mapper.setInjectableValues(new GuiceInjectableValues(injector)); mapper.setInjectableValues(new GuiceInjectableValues(injector));
setupAnnotationIntrospector(mapper, new GuiceAnnotationIntrospector()); setupAnnotationIntrospector(mapper, new GuiceAnnotationIntrospector());
} }
@VisibleForTesting
public static void setupAnnotationIntrospector( public static void setupAnnotationIntrospector(
final ObjectMapper mapper, final ObjectMapper mapper,
final AnnotationIntrospector annotationIntrospector 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.BeanProperty;
import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.InjectableValues;
import com.google.inject.ConfigurationException;
import com.google.inject.Injector; import com.google.inject.Injector;
import com.google.inject.Key; import com.google.inject.Key;
import org.apache.druid.java.util.common.IAE; 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 public class GuiceInjectableValues extends InjectableValues
{ {
private final Injector injector; private final Injector injector;
private final AtomicReference<HashSet<Key>> nullables;
public GuiceInjectableValues(Injector injector) public GuiceInjectableValues(Injector injector)
{ {
this.injector = injector; this.injector = injector;
this.nullables = new AtomicReference<>(new HashSet<>());
} }
@Override @Override
@ -49,8 +57,22 @@ public class GuiceInjectableValues extends InjectableValues
// whatever provider needs" // whatever provider needs"
// Currently we should only be dealing with `Key` instances, and anything more advanced should be handled with // Currently we should only be dealing with `Key` instances, and anything more advanced should be handled with
// great care // great care
if (valueId instanceof Key) { if (nullables.get().contains((Key) valueId)) {
return injector.getInstance((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( throw new IAE(
"Unknown class type [%s] for valueId [%s]", "Unknown class type [%s] for valueId [%s]",

View File

@ -47,6 +47,7 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function; import java.util.function.Function;
public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long> public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
@ -88,6 +89,58 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
this.broadcastJoinHelper = inputChannelsAndBroadcastJoinHelper.rhs; 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 @Override
public List<ReadableFrameChannel> inputChannels() public List<ReadableFrameChannel> inputChannels()
{ {
@ -146,6 +199,7 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
private boolean initializeSegmentMapFn(final IntSet readableInputs) private boolean initializeSegmentMapFn(final IntSet readableInputs)
{ {
final AtomicLong cpuAccumulator = new AtomicLong();
if (segmentMapFn != null) { if (segmentMapFn != null) {
return true; return true;
} else if (broadcastJoinHelper == null) { } else if (broadcastJoinHelper == null) {
@ -153,64 +207,11 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
return true; return true;
} else { } else {
final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs);
DataSource inlineChannelDataSource = broadcastJoinHelper.inlineChannelData(query.getDataSource());
if (retVal) { if (retVal) {
segmentMapFn = broadcastJoinHelper.makeSegmentMapFn(query); segmentMapFn = inlineChannelDataSource.createSegmentMapFunction(query, cpuAccumulator);
} }
return retVal; 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; 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.Int2IntMap;
import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntIterator;
import it.unimi.dsi.fastutil.ints.IntOpenHashSet; 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.msq.indexing.error.MSQException;
import org.apache.druid.query.DataSource; import org.apache.druid.query.DataSource;
import org.apache.druid.query.InlineDataSource; 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.ColumnValueSelector;
import org.apache.druid.segment.Cursor; 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 org.apache.druid.segment.join.JoinableFactoryWrapper;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public class BroadcastJoinHelper public class BroadcastJoinHelper
@ -138,21 +131,9 @@ public class BroadcastJoinHelper
return sideChannelNumbers; 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 public DataSource inlineChannelData(final DataSource originalDataSource)
DataSource inlineChannelData(final DataSource originalDataSource)
{ {
if (originalDataSource instanceof InputNumberDataSource) { if (originalDataSource instanceof InputNumberDataSource) {
final int inputNumber = ((InputNumberDataSource) originalDataSource).getInputNumber(); final int inputNumber = ((InputNumberDataSource) originalDataSource).getInputNumber();

View File

@ -315,9 +315,9 @@ public class DataSourcePlan
clause.getPrefix(), clause.getPrefix(),
clause.getCondition(), clause.getCondition(),
clause.getJoinType(), clause.getJoinType(),
// First JoinDataSource (i == 0) involves the base table, so we need to propagate the base table filter. // 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()); inputSpecs.addAll(clausePlan.getInputSpecs());
clausePlan.getBroadcastInputs().intStream().forEach(n -> broadcastInputs.add(n + shift)); 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 com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.query.DataSource; 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.Collections;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
@JsonTypeName("inputNumber") @JsonTypeName("inputNumber")
public class InputNumberDataSource implements DataSource public class InputNumberDataSource implements DataSource
@ -81,6 +85,27 @@ public class InputNumberDataSource implements DataSource
return false; 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 @JsonProperty
public int getInputNumber() 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.msq.test.MSQTestBase;
import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.InlineDataSource;
import org.apache.druid.query.QueryDataSource; 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.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
@ -438,6 +439,122 @@ public class MSQSelectTest extends MSQTestBase
.verifyResults(); .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 @Test
public void testGroupByOrderByAggregation() public void testGroupByOrderByAggregation()
{ {

View File

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

View File

@ -20,7 +20,6 @@
package org.apache.druid.msq.test; package org.apache.druid.msq.test;
import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule; 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.ImmutableSet;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.io.ByteStreams; import com.google.common.io.ByteStreams;
import com.google.inject.Binder;
import com.google.inject.Injector; import com.google.inject.Injector;
import com.google.inject.Key; import com.google.inject.Key;
import com.google.inject.Module;
import com.google.inject.TypeLiteral; import com.google.inject.TypeLiteral;
import com.google.inject.util.Modules; import com.google.inject.util.Modules;
import com.google.inject.util.Providers; 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.data.input.impl.StringDimensionSchema;
import org.apache.druid.discovery.NodeRole; import org.apache.druid.discovery.NodeRole;
import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.frame.testutil.FrameTestUtil;
import org.apache.druid.guice.DruidSecondaryModule;
import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.GuiceInjectors;
import org.apache.druid.guice.IndexingServiceTuningConfigModule; import org.apache.druid.guice.IndexingServiceTuningConfigModule;
import org.apache.druid.guice.JoinableFactoryModule; 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.column.RowSignature;
import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.loading.DataSegmentPusher; 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.LocalDataSegmentPusher;
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
import org.apache.druid.segment.loading.LocalLoadSpec; import org.apache.druid.segment.loading.LocalLoadSpec;
@ -162,7 +159,6 @@ import org.mockito.Mockito;
import javax.annotation.Nonnull; import javax.annotation.Nonnull;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.Closeable; import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
@ -253,8 +249,14 @@ public class MSQTestBase extends BaseCalciteQueryTest
groupByBuffers = TestGroupByBuffers.createDefault(); groupByBuffers = TestGroupByBuffers.createDefault();
SqlTestFramework qf = queryFramework(); 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); ObjectMapper secondMapper = setupObjectMapper(secondInjector);
indexIO = new IndexIO(secondMapper, () -> 0); indexIO = new IndexIO(secondMapper, () -> 0);
@ -271,70 +273,69 @@ public class MSQTestBase extends BaseCalciteQueryTest
segmentManager = new MSQTestSegmentManager(segmentCacheManager, indexIO); segmentManager = new MSQTestSegmentManager(segmentCacheManager, indexIO);
Injector injector = GuiceInjectors.makeStartupInjectorWithModules(ImmutableList.of( Injector injector = GuiceInjectors.makeStartupInjectorWithModules(ImmutableList.of(
new Module() binder -> {
{ DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
@Override
public void configure(Binder binder)
{ {
DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig() @Override
public String getFormatString()
{ {
@Override return "test";
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");
} }
};
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 -> { binder -> {
// Requirements of JoinableFactoryModule // Requirements of JoinableFactoryModule
@ -393,7 +394,8 @@ public class MSQTestBase extends BaseCalciteQueryTest
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
objectMapper, objectMapper,
CalciteTests.DRUID_SCHEMA_NAME, CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()) new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
); );
sqlStatementFactory = CalciteTests.createSqlStatementFactory(engine, plannerFactory); sqlStatementFactory = CalciteTests.createSqlStatementFactory(engine, plannerFactory);
@ -568,14 +570,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
"compaction" "compaction"
) )
).registerSubtypes(ExternalDataSource.class)); ).registerSubtypes(ExternalDataSource.class));
mapper.setInjectableValues( DruidSecondaryModule.setupJackson(injector, mapper);
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())
);
mapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local")); 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.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo; 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.List;
import java.util.Set; 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. * 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 * 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'. * {@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 DataSourceAnalysis#isConcreteBased() which uses this
* @see org.apache.druid.query.planning.DataSourceAnalysis#isConcreteTableBased() which uses this * @see DataSourceAnalysis#isConcreteTableBased() which uses this
*/ */
boolean isConcrete(); 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 com.google.common.collect.Lists;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.RowAdapter; 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.ColumnType;
import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.RowSignature;
@ -36,12 +37,14 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
/** /**
* Represents an inline datasource, where the rows are embedded within the DataSource object itself. * 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 * 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()}. * {@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); 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 @Override
public Set<String> getTableNames() public Set<String> getTableNames()
{ {
@ -182,6 +234,27 @@ public class InlineDataSource implements DataSource
return true; 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 * 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. * 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 + "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.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList; 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.IAE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils; 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.math.expr.ExprMacroTable;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.filter.DimFilter; 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.JoinConditionAnalysis;
import org.apache.druid.segment.join.JoinPrefixUtils; import org.apache.druid.segment.join.JoinPrefixUtils;
import org.apache.druid.segment.join.JoinType; 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 javax.annotation.Nullable;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors; import java.util.stream.Collectors;
/** /**
* Represents a join of two datasources. * Represents a join of two datasources.
* * <p>
* Logically, this datasource contains the result of: * Logically, this datasource contains the result of:
* * <p>
* (1) prefixing all right-side columns with "rightPrefix" * (1) prefixing all right-side columns with "rightPrefix"
* (2) then, joining the left and (prefixed) right sides using the provided type and condition * (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 * 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 * the prefix, will be shadowed. It is up to the caller to ensure that no important columns are shadowed by the
* chosen prefix. * chosen prefix.
* * <p>
* When analyzed by {@link org.apache.druid.query.planning.DataSourceAnalysis}, the right-hand side of this datasource * When analyzed by {@link DataSourceAnalysis}, the right-hand side of this datasource
* will become a {@link org.apache.druid.query.planning.PreJoinableClause} object. * will become a {@link PreJoinableClause} object.
*/ */
public class JoinDataSource implements DataSource 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 // An optional filter on the left side if left is direct table access
@Nullable @Nullable
private final DimFilter leftFilter; private final DimFilter leftFilter;
@Nullable
private final JoinableFactoryWrapper joinableFactoryWrapper;
private static final Logger log = new Logger(JoinDataSource.class);
private final DataSourceAnalysis analysis;
private JoinDataSource( private JoinDataSource(
DataSource left, DataSource left,
@ -74,7 +105,8 @@ public class JoinDataSource implements DataSource
String rightPrefix, String rightPrefix,
JoinConditionAnalysis conditionAnalysis, JoinConditionAnalysis conditionAnalysis,
JoinType joinType, JoinType joinType,
@Nullable DimFilter leftFilter @Nullable DimFilter leftFilter,
@Nullable JoinableFactoryWrapper joinableFactoryWrapper
) )
{ {
this.left = Preconditions.checkNotNull(left, "left"); 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" "left filter is only supported if left data source is direct table access"
); );
this.leftFilter = leftFilter; 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("condition") String condition,
@JsonProperty("joinType") JoinType joinType, @JsonProperty("joinType") JoinType joinType,
@Nullable @JsonProperty("leftFilter") DimFilter leftFilter, @Nullable @JsonProperty("leftFilter") DimFilter leftFilter,
@JacksonInject ExprMacroTable macroTable @JacksonInject ExprMacroTable macroTable,
@Nullable @JacksonInject JoinableFactoryWrapper joinableFactoryWrapper
) )
{ {
return new JoinDataSource( return new JoinDataSource(
@ -114,7 +149,8 @@ public class JoinDataSource implements DataSource
macroTable macroTable
), ),
joinType, joinType,
leftFilter leftFilter,
joinableFactoryWrapper
); );
} }
@ -127,12 +163,22 @@ public class JoinDataSource implements DataSource
final String rightPrefix, final String rightPrefix,
final JoinConditionAnalysis conditionAnalysis, final JoinConditionAnalysis conditionAnalysis,
final JoinType joinType, 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 @Override
public Set<String> getTableNames() public Set<String> getTableNames()
{ {
@ -185,6 +231,12 @@ public class JoinDataSource implements DataSource
return leftFilter; return leftFilter;
} }
@Nullable
public JoinableFactoryWrapper getJoinableFactoryWrapper()
{
return joinableFactoryWrapper;
}
@Override @Override
public List<DataSource> getChildren() public List<DataSource> getChildren()
{ {
@ -204,7 +256,8 @@ public class JoinDataSource implements DataSource
rightPrefix, rightPrefix,
conditionAnalysis, conditionAnalysis,
joinType, 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 * 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 * a virtual column in the virtual column registry. It helps to remove any extraenous virtual columns created and only
* use the relevant ones. * use the relevant ones.
*
* @return a set of column names which might be virtual columns on left table in join condition * @return a set of column names which might be virtual columns on left table in join condition
*/ */
public Set<String> getVirtualColumnCandidates() public Set<String> getVirtualColumnCandidates()
@ -278,4 +332,153 @@ public class JoinDataSource implements DataSource
", leftFilter=" + leftFilter + ", 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.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.SegmentReference;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
/** /**
* Represents a lookup. * 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 * 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. * placeholder for a future time in which it will become queryable.
* * <p>
* The "lookupName" referred to here should be provided by a * The "lookupName" referred to here should be provided by a
* {@link org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider}. * {@link org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider}.
*/ */
@ -96,6 +99,27 @@ public class LookupDataSource implements DataSource
return true; 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 @Override
public boolean equals(Object o) 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.dimension.DimensionSpec;
import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.planning.DataSourceAnalysis; 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.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.VirtualColumns; 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) public static <T> Query<T> withBaseDataSource(final Query<T> query, final DataSource newBaseDataSource)
{ {
final Query<T> retVal; return query.withDataSource(query.getDataSource().withUpdatedDataSource(newBaseDataSource));
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;
} }
/** /**

View File

@ -25,10 +25,13 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.SegmentReference;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
@JsonTypeName("query") @JsonTypeName("query")
public class QueryDataSource implements DataSource public class QueryDataSource implements DataSource
@ -88,6 +91,28 @@ public class QueryDataSource implements DataSource
return false; 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 @Override
public String toString() public String toString()
{ {

View File

@ -24,11 +24,14 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.SegmentReference;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
@JsonTypeName("table") @JsonTypeName("table")
public class TableDataSource implements DataSource public class TableDataSource implements DataSource
@ -93,6 +96,27 @@ public class TableDataSource implements DataSource
return true; 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 @Override
public String toString() public String toString()
{ {

View File

@ -26,9 +26,12 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.SegmentReference;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public class UnionDataSource implements DataSource public class UnionDataSource implements DataSource
@ -105,6 +108,27 @@ public class UnionDataSource implements DataSource
return dataSources.stream().allMatch(DataSource::isConcrete); 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 @Override
public boolean equals(Object o) 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. * 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. * In other words, logically the operation is: join(join(join(baseSegment, clauses[0]), clauses[1]), clauses[2]) etc.
*/ */
public class HashJoinSegment implements SegmentReference public class HashJoinSegment implements SegmentReference
@ -148,15 +148,4 @@ public class HashJoinSegment implements SegmentReference
return Optional.empty(); 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 * 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". * references to the left-hand "t1".
* <p> * <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 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.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.HashMultiset; 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.Multiset;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import com.google.inject.Inject; 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.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.Filter;
import org.apache.druid.query.filter.InDimFilter; 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.FalseFilter;
import org.apache.druid.segment.filter.Filters; 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 javax.annotation.Nullable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors; import java.util.stream.Collectors;
/** /**
@ -66,8 +43,7 @@ import java.util.stream.Collectors;
public class JoinableFactoryWrapper public class JoinableFactoryWrapper
{ {
private static final byte JOIN_OPERATION = 0x1; public static final byte JOIN_OPERATION = 0x1;
private static final Logger log = new Logger(JoinableFactoryWrapper.class);
private final JoinableFactory joinableFactory; private final JoinableFactory joinableFactory;
@ -77,144 +53,12 @@ public class JoinableFactoryWrapper
this.joinableFactory = Preconditions.checkNotNull(joinableFactory, "joinableFactory"); 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. * 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. * See {@link #convertJoinToFilter} for details on the logic.
*/ */
@VisibleForTesting public static Pair<List<Filter>, List<JoinableClause>> convertJoinsToFilters(
static Pair<List<Filter>, List<JoinableClause>> convertJoinsToFilters(
final List<JoinableClause> clauses, final List<JoinableClause> clauses,
final Set<String> requiredColumns, final Set<String> requiredColumns,
final int maxNumFilterValues final int maxNumFilterValues
@ -271,16 +115,16 @@ public class JoinableFactoryWrapper
/** /**
* Converts a join clause into an "in" filter if possible. * Converts a join clause into an "in" filter if possible.
* * <p>
* The requirements are: * The requirements are:
* * <p>
* - it must be an INNER equi-join * - 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 * - 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 * 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. * as well for correct results.
* - no columns from the right-hand side can appear in "requiredColumns". If the columns from right side are required * - 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 * (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. * 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 * @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 * 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); return new JoinClauseToFilterConversion(null, false);
} }
public JoinableFactory getJoinableFactory()
{
return joinableFactory;
}
/** /**
* Encapsulates the conversion which happened for a joinable clause. * Encapsulates the conversion which happened for a joinable clause.
* convertedFilter represents the filter which got generated from the conversion. * convertedFilter represents the filter which got generated from the conversion.
* joinClauseFullyConverted represents whether convertedFilter fully encapsulated the joinable clause or not. * 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. * 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 * 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. * 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. * 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.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterators;
import nl.jqno.equalsverifier.EqualsVerifier; 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.math.expr.ExprMacroTable;
import org.apache.druid.query.filter.InDimFilter;
import org.apache.druid.query.filter.TrueDimFilter; import org.apache.druid.query.filter.TrueDimFilter;
import org.apache.druid.segment.TestHelper; 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.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.Assert;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.ExpectedException; import org.junit.rules.ExpectedException;
import org.mockito.Mockito; import org.mockito.Mockito;
import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.Optional;
public class JoinDataSourceTest 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 fooTable = new TableDataSource("foo");
private final TableDataSource barTable = new TableDataSource("bar"); private final TableDataSource barTable = new TableDataSource("bar");
private final LookupDataSource lookylooLookup = new LookupDataSource("lookyloo"); private final LookupDataSource lookylooLookup = new LookupDataSource("lookyloo");
private final JoinDataSource joinTableToLookup = JoinDataSource.create( private final JoinDataSource joinTableToLookup = JoinDataSource.create(
fooTable, fooTable,
lookylooLookup, lookylooLookup,
@ -51,9 +62,9 @@ public class JoinDataSourceTest
"x == \"j.x\"", "x == \"j.x\"",
JoinType.LEFT, JoinType.LEFT,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
null
); );
private final JoinDataSource joinTableToTable = JoinDataSource.create( private final JoinDataSource joinTableToTable = JoinDataSource.create(
fooTable, fooTable,
barTable, barTable,
@ -61,8 +72,13 @@ public class JoinDataSourceTest
"x == \"j.x\"", "x == \"j.x\"",
JoinType.LEFT, JoinType.LEFT,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
null
); );
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Mock
private JoinableFactoryWrapper joinableFactoryWrapper;
@Test @Test
public void test_getTableNames_tableToTable() public void test_getTableNames_tableToTable()
@ -149,13 +165,14 @@ public class JoinDataSourceTest
EqualsVerifier.forClass(JoinDataSource.class) EqualsVerifier.forClass(JoinDataSource.class)
.usingGetClass() .usingGetClass()
.withNonnullFields("left", "right", "rightPrefix", "conditionAnalysis", "joinType") .withNonnullFields("left", "right", "rightPrefix", "conditionAnalysis", "joinType")
.withIgnoredFields("joinableFactoryWrapper", "analysis")
.verify(); .verify();
} }
@Test @Test
public void test_serde() throws Exception public void test_serde() throws Exception
{ {
final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); final ObjectMapper jsonMapper = TestHelper.makeJsonMapperForJoinable(joinableFactoryWrapper);
JoinDataSource joinDataSource = JoinDataSource.create( JoinDataSource joinDataSource = JoinDataSource.create(
new TableDataSource("table1"), new TableDataSource("table1"),
new TableDataSource("table2"), new TableDataSource("table2"),
@ -163,7 +180,8 @@ public class JoinDataSourceTest
"x == \"j.x\"", "x == \"j.x\"",
JoinType.LEFT, JoinType.LEFT,
TrueDimFilter.instance(), TrueDimFilter.instance(),
ExprMacroTable.nil() ExprMacroTable.nil(),
joinableFactoryWrapper
); );
final JoinDataSource deserialized = (JoinDataSource) jsonMapper.readValue( final JoinDataSource deserialized = (JoinDataSource) jsonMapper.readValue(
@ -186,7 +204,8 @@ public class JoinDataSourceTest
"x == \"j.x\"", "x == \"j.x\"",
JoinType.LEFT, JoinType.LEFT,
TrueDimFilter.instance(), TrueDimFilter.instance(),
ExprMacroTable.nil() ExprMacroTable.nil(),
null
); );
} }
@ -200,7 +219,8 @@ public class JoinDataSourceTest
"x == \"j.x\"", "x == \"j.x\"",
JoinType.LEFT, JoinType.LEFT,
TrueDimFilter.instance(), TrueDimFilter.instance(),
ExprMacroTable.nil() ExprMacroTable.nil(),
null
); );
Assert.assertEquals(TrueDimFilter.instance(), dataSource.getLeftFilter()); Assert.assertEquals(TrueDimFilter.instance(), dataSource.getLeftFilter());
} }
@ -215,8 +235,275 @@ public class JoinDataSourceTest
"x == \"j.x\"", "x == \"j.x\"",
JoinType.LEFT, JoinType.LEFT,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
null
); );
Assert.assertEquals(dataSource.getVirtualColumnCandidates(), ImmutableSet.of("x")); 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\"", "\"foo.x\" == \"bar.x\"",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
null
) )
) )
.intervals("2000/3000") .intervals("2000/3000")
@ -462,7 +463,8 @@ public class QueriesTest
"\"foo.x\" == \"bar.x\"", "\"foo.x\" == \"bar.x\"",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
null
) )
) )
.intervals("2000/3000") .intervals("2000/3000")
@ -500,14 +502,16 @@ public class QueriesTest
"\"foo.x\" == \"bar.x\"", "\"foo.x\" == \"bar.x\"",
JoinType.INNER, JoinType.INNER,
TrueDimFilter.instance(), TrueDimFilter.instance(),
ExprMacroTable.nil() ExprMacroTable.nil(),
null
), ),
new TableDataSource("foo_outer"), new TableDataSource("foo_outer"),
"j0.", "j0.",
"\"foo_outer.x\" == \"bar.x\"", "\"foo_outer.x\" == \"bar.x\"",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
null
) )
) )
@ -533,14 +537,16 @@ public class QueriesTest
"\"foo.x\" == \"bar.x\"", "\"foo.x\" == \"bar.x\"",
JoinType.INNER, JoinType.INNER,
TrueDimFilter.instance(), TrueDimFilter.instance(),
ExprMacroTable.nil() ExprMacroTable.nil(),
null
), ),
new TableDataSource("foo_outer"), new TableDataSource("foo_outer"),
"j0.", "j0.",
"\"foo_outer.x\" == \"bar.x\"", "\"foo_outer.x\" == \"bar.x\"",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
null
) )
) )

View File

@ -612,7 +612,8 @@ public class DataSourceAnalysisTest
joinClause(rightPrefix).getOriginalExpression(), joinClause(rightPrefix).getOriginalExpression(),
joinType, joinType,
dimFilter, 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.column.ColumnConfig;
import org.apache.druid.segment.data.ComparableList; import org.apache.druid.segment.data.ComparableList;
import org.apache.druid.segment.data.ComparableStringArray; 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.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder;
import org.junit.Assert; import org.junit.Assert;
@ -95,9 +96,10 @@ public class TestHelper
public static ObjectMapper makeJsonMapper() public static ObjectMapper makeJsonMapper()
{ {
final ObjectMapper mapper = new DefaultObjectMapper(); final ObjectMapper mapper = new DefaultObjectMapper();
AnnotationIntrospector introspector = makeAnnotationIntrospector(); final AnnotationIntrospector introspector = makeAnnotationIntrospector();
DruidSecondaryModule.setupAnnotationIntrospector(mapper, introspector); DruidSecondaryModule.setupAnnotationIntrospector(mapper, introspector);
mapper.setInjectableValues( mapper.setInjectableValues(
new InjectableValues.Std() new InjectableValues.Std()
.addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
@ -107,6 +109,23 @@ public class TestHelper
return mapper; 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() public static ObjectMapper makeSmileMapper()
{ {
final ObjectMapper mapper = new DefaultObjectMapper(); 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.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterators;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.config.NullHandling;
import org.apache.druid.common.config.NullHandlingTest; import org.apache.druid.common.config.NullHandlingTest;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE; 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.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable; 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.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.extraction.MapLookupExtractor;
import org.apache.druid.query.filter.FalseDimFilter;
import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.filter.InDimFilter; 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.ColumnType;
import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.filter.FalseFilter; 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.IndexedTable;
import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.join.table.IndexedTableJoinable;
import org.apache.druid.segment.join.table.RowBasedIndexedTable; 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.Assert;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.ExpectedException; import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder; import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.Collectors; import java.util.stream.Collectors;
public class JoinableFactoryWrapperTest extends NullHandlingTest public class JoinableFactoryWrapperTest extends NullHandlingTest
@ -85,12 +61,12 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest
private static final Map<String, String> TEST_LOOKUP = private static final Map<String, String> TEST_LOOKUP =
ImmutableMap.<String, String>builder() ImmutableMap.<String, String>builder()
.put("MX", "Mexico") .put("MX", "Mexico")
.put("NO", "Norway") .put("NO", "Norway")
.put("SV", "El Salvador") .put("SV", "El Salvador")
.put("US", "United States") .put("US", "United States")
.put("", "Empty key") .put("", "Empty key")
.build(); .build();
private static final Set<String> TEST_LOOKUP_KEYS = private static final Set<String> TEST_LOOKUP_KEYS =
NullHandling.sqlCompatible() NullHandling.sqlCompatible()
@ -137,340 +113,6 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest
@Rule @Rule
public ExpectedException expectedException = ExpectedException.none(); 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 @Test
public void test_checkClausePrefixesForDuplicatesAndShadowing_noConflicts() public void test_checkClausePrefixesForDuplicatesAndShadowing_noConflicts()
{ {
@ -563,10 +205,12 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest
Assert.assertEquals( Assert.assertEquals(
Pair.of( Pair.of(
ImmutableList.of(new InDimFilter( ImmutableList.of(new InDimFilter(
"x", "x",
INDEXED_TABLE_DS.getRowsAsList().stream().map(row -> row[0].toString()).collect(Collectors.toSet())) 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 conversion
); );
@ -946,53 +590,4 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest
conversion 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; package org.apache.druid.segment.join;
import org.apache.druid.query.DataSource; 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.List;
import java.util.Set; 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. * A datasource that returns nothing. Only used to test un-registered datasources.
@ -64,4 +68,25 @@ public class NoopDataSource implements DataSource
{ {
return false; 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 ServerType serverType
) )
{ {
return isQueryCacheable(query, cacheStrategy, cacheConfig, serverType) return cacheConfig.isUseCache()
&& query.context().isUseCache() && query.context().isUseCache()
&& cacheConfig.isUseCache(); && isQueryCacheable(query, cacheStrategy, cacheConfig, serverType);
} }
/** /**

View File

@ -296,9 +296,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
query, query,
strategy, strategy,
useCache, useCache,
populateCache, populateCache
dataSourceAnalysis,
joinableFactoryWrapper
); );
} }
@ -761,24 +759,18 @@ public class CachingClusteredClient implements QuerySegmentWalker
{ {
private final Query<T> query; private final Query<T> query;
private final CacheStrategy<T, Object, Query<T>> strategy; private final CacheStrategy<T, Object, Query<T>> strategy;
private final DataSourceAnalysis dataSourceAnalysis;
private final JoinableFactoryWrapper joinableFactoryWrapper;
private final boolean isSegmentLevelCachingEnable; private final boolean isSegmentLevelCachingEnable;
CacheKeyManager( CacheKeyManager(
final Query<T> query, final Query<T> query,
final CacheStrategy<T, Object, Query<T>> strategy, final CacheStrategy<T, Object, Query<T>> strategy,
final boolean useCache, final boolean useCache,
final boolean populateCache, final boolean populateCache
final DataSourceAnalysis dataSourceAnalysis,
final JoinableFactoryWrapper joinableFactoryWrapper
) )
{ {
this.query = query; this.query = query;
this.strategy = strategy; this.strategy = strategy;
this.dataSourceAnalysis = dataSourceAnalysis;
this.joinableFactoryWrapper = joinableFactoryWrapper;
this.isSegmentLevelCachingEnable = ((populateCache || useCache) this.isSegmentLevelCachingEnable = ((populateCache || useCache)
&& !query.context().isBySegment()); // explicit bySegment queries are never cached && !query.context().isBySegment()); // explicit bySegment queries are never cached
@ -840,15 +832,14 @@ public class CachingClusteredClient implements QuerySegmentWalker
private byte[] computeQueryCacheKeyWithJoin() private byte[] computeQueryCacheKeyWithJoin()
{ {
Preconditions.checkNotNull(strategy, "strategy cannot be null"); Preconditions.checkNotNull(strategy, "strategy cannot be null");
if (dataSourceAnalysis.isJoin()) { byte[] dataSourceCacheKey = query.getDataSource().getCacheKey();
byte[] joinDataSourceCacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis) if (null == dataSourceCacheKey) {
.orElse(null); return null;
if (null == joinDataSourceCacheKey) { } else if (dataSourceCacheKey.length > 0) {
return null; // A join operation that does not support caching return Bytes.concat(dataSourceCacheKey, strategy.computeCacheKey(query));
} } else {
return Bytes.concat(joinDataSourceCacheKey, strategy.computeCacheKey(query)); 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.ISE;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair; 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.common.guava.FunctionalIterable;
import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; 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.query.spec.SpecificSegmentSpec;
import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SegmentReference;
import org.apache.druid.segment.StorageAdapter; 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.join.JoinableFactoryWrapper;
import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.FireHydrant;
import org.apache.druid.segment.realtime.plumber.Sink; 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. // segmentMapFn maps each base Segment into a joined Segment if necessary.
final Function<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( final Function<SegmentReference, SegmentReference> segmentMapFn = analysis.getDataSource()
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), .createSegmentMapFunction(
analysis.getPreJoinableClauses(), query,
cpuTimeAccumulator, 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 // 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() final Optional<byte[]> cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey());
? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis)
: Optional.of(StringUtils.EMPTY_BYTES);
Iterable<QueryRunner<T>> perSegmentRunners = Iterables.transform( Iterable<QueryRunner<T>> perSegmentRunners = Iterables.transform(
specs, 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.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SegmentReference;
import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.SegmentWrangler;
import org.apache.druid.segment.filter.Filters;
import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -94,12 +93,13 @@ public class LocalQuerySegmentWalker implements QuerySegmentWalker
final AtomicLong cpuAccumulator = new AtomicLong(0L); final AtomicLong cpuAccumulator = new AtomicLong(0L);
final Function<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( final Function<SegmentReference, SegmentReference> segmentMapFn = analysis
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), .getDataSource()
analysis.getPreJoinableClauses(), .createSegmentMapFunction(
cpuAccumulator, query,
analysis.getBaseQuery().orElse(query) cpuAccumulator
); );
final QueryRunnerFactory<T, Query<T>> queryRunnerFactory = conglomerate.findFactory(query); final QueryRunnerFactory<T, Query<T>> queryRunnerFactory = conglomerate.findFactory(query);
final QueryRunner<T> baseRunner = queryRunnerFactory.mergeRunners( 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.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SegmentReference;
import org.apache.druid.segment.StorageAdapter; 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.join.JoinableFactoryWrapper;
import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SegmentManager;
import org.apache.druid.server.SetAndVerifyContextQueryRunner; import org.apache.druid.server.SetAndVerifyContextQueryRunner;
@ -195,18 +194,10 @@ public class ServerManager implements QuerySegmentWalker
} else { } else {
return new ReportTimelineMissingSegmentQueryRunner<>(Lists.newArrayList(specs)); 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 // 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() final Optional<byte[]> cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey());
? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis)
: Optional.of(StringUtils.EMPTY_BYTES);
final FunctionalIterable<QueryRunner<T>> queryRunners = FunctionalIterable final FunctionalIterable<QueryRunner<T>> queryRunners = FunctionalIterable
.create(specs) .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.QueryableDruidServer;
import org.apache.druid.client.selector.ServerSelector; import org.apache.druid.client.selector.ServerSelector;
import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.CacheStrategy;
import org.apache.druid.query.DataSource;
import org.apache.druid.query.Query; import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.segment.join.JoinableFactoryWrapper; 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.DataSegment;
import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec;
@ -42,7 +44,6 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expect;
@ -60,6 +61,8 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
private JoinableFactoryWrapper joinableFactoryWrapper; private JoinableFactoryWrapper joinableFactoryWrapper;
@Mock @Mock
private DataSourceAnalysis dataSourceAnalysis; private DataSourceAnalysis dataSourceAnalysis;
@Mock
private DataSource dataSource;
private static final byte[] QUERY_CACHE_KEY = new byte[]{1, 2, 3}; private static final byte[] QUERY_CACHE_KEY = new byte[]{1, 2, 3};
private static final byte[] JOIN_KEY = new byte[]{4, 5}; private static final byte[] JOIN_KEY = new byte[]{4, 5};
@ -139,7 +142,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
@Test @Test
public void testComputeEtag_nonJoinDataSource() public void testComputeEtag_nonJoinDataSource()
{ {
expect(dataSourceAnalysis.isJoin()).andReturn(false); expect(query.getDataSource()).andReturn(new NoopDataSource());
replayAll(); replayAll();
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager(); CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
Set<SegmentServerSelector> selectors = ImmutableSet.of( Set<SegmentServerSelector> selectors = ImmutableSet.of(
@ -161,8 +164,8 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
@Test @Test
public void testComputeEtag_joinWithUnsupportedCaching() public void testComputeEtag_joinWithUnsupportedCaching()
{ {
expect(dataSourceAnalysis.isJoin()).andReturn(true); expect(query.getDataSource()).andReturn(dataSource);
expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.empty()); expect(dataSource.getCacheKey()).andReturn(null);
replayAll(); replayAll();
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager(); CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
Set<SegmentServerSelector> selectors = ImmutableSet.of( Set<SegmentServerSelector> selectors = ImmutableSet.of(
@ -173,38 +176,15 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
Assert.assertNull(actual); 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 @Test
public void testComputeEtag_noEffectifBySegment() public void testComputeEtag_noEffectifBySegment()
{ {
expect(dataSourceAnalysis.isJoin()).andReturn(false); expect(dataSourceAnalysis.isJoin()).andReturn(false);
reset(query); reset(query);
expect(query.getDataSource()).andReturn(new NoopDataSource());
expect(query.context()).andReturn(QueryContext.of(ImmutableMap.of(QueryContexts.BY_SEGMENT_KEY, true))).anyTimes(); expect(query.context()).andReturn(QueryContext.of(ImmutableMap.of(QueryContexts.BY_SEGMENT_KEY, true))).anyTimes();
replayAll(); replayAll();
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager(); CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
Set<SegmentServerSelector> selectors = ImmutableSet.of( Set<SegmentServerSelector> selectors = ImmutableSet.of(
@ -219,14 +199,13 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
public void testComputeEtag_noEffectIfUseAndPopulateFalse() public void testComputeEtag_noEffectIfUseAndPopulateFalse()
{ {
expect(dataSourceAnalysis.isJoin()).andReturn(false); expect(dataSourceAnalysis.isJoin()).andReturn(false);
expect(query.getDataSource()).andReturn(new NoopDataSource());
replayAll(); replayAll();
CachingClusteredClient.CacheKeyManager<Object> keyManager = new CachingClusteredClient.CacheKeyManager<>( CachingClusteredClient.CacheKeyManager<Object> keyManager = new CachingClusteredClient.CacheKeyManager<>(
query, query,
strategy, strategy,
false, false,
false, false
dataSourceAnalysis,
joinableFactoryWrapper
); );
Set<SegmentServerSelector> selectors = ImmutableSet.of( Set<SegmentServerSelector> selectors = ImmutableSet.of(
makeHistoricalServerSelector(1), makeHistoricalServerSelector(1),
@ -239,7 +218,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
@Test @Test
public void testSegmentQueryCacheKey_nonJoinDataSource() public void testSegmentQueryCacheKey_nonJoinDataSource()
{ {
expect(dataSourceAnalysis.isJoin()).andReturn(false); expect(query.getDataSource()).andReturn(new NoopDataSource());
replayAll(); replayAll();
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager(); CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey(); byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey();
@ -249,20 +228,18 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
@Test @Test
public void testSegmentQueryCacheKey_joinWithUnsupportedCaching() public void testSegmentQueryCacheKey_joinWithUnsupportedCaching()
{ {
expect(dataSourceAnalysis.isJoin()).andReturn(true); expect(query.getDataSource()).andReturn(new NoopDataSource());
expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.empty());
replayAll(); replayAll();
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager(); CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey(); byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey();
Assert.assertNull(cacheKey); Assert.assertNotNull(cacheKey);
} }
@Test @Test
public void testSegmentQueryCacheKey_joinWithSupportedCaching() public void testSegmentQueryCacheKey_joinWithSupportedCaching()
{ {
expect(query.getDataSource()).andReturn(dataSource);
expect(dataSourceAnalysis.isJoin()).andReturn(true); expect(dataSource.getCacheKey()).andReturn(JOIN_KEY);
expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(JOIN_KEY));
replayAll(); replayAll();
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager(); CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey(); byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey();
@ -287,9 +264,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
query, query,
strategy, strategy,
false, false,
false, false
dataSourceAnalysis,
joinableFactoryWrapper
).computeSegmentLevelQueryCacheKey()); ).computeSegmentLevelQueryCacheKey());
} }
@ -299,9 +274,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
query, query,
strategy, strategy,
true, true,
true, true
dataSourceAnalysis,
joinableFactoryWrapper
); );
} }

View File

@ -127,6 +127,7 @@ public class ClientQuerySegmentWalkerTest
private static final String VERSION = "A"; private static final String VERSION = "A";
private static final ShardSpec SHARD_SPEC = new NumberedShardSpec(0, 1); private static final ShardSpec SHARD_SPEC = new NumberedShardSpec(0, 1);
private static final InlineDataSource FOO_INLINE = InlineDataSource.fromIterable( private static final InlineDataSource FOO_INLINE = InlineDataSource.fromIterable(
ImmutableList.<Object[]>builder() ImmutableList.<Object[]>builder()
.add(new Object[]{INTERVAL.getStartMillis(), "x", 1}) .add(new Object[]{INTERVAL.getStartMillis(), "x", 1})
@ -502,7 +503,11 @@ public class ClientQuerySegmentWalkerTest
"\"j.s\" == \"s\"", "\"j.s\" == \"s\"",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
new JoinableFactoryWrapper(QueryStackTests.makeJoinableFactoryFromDefault(
null,
null,
null))
) )
) )
.setGranularity(Granularities.ALL) .setGranularity(Granularities.ALL)
@ -570,7 +575,11 @@ public class ClientQuerySegmentWalkerTest
"\"j.s\" == \"s\"", "\"j.s\" == \"s\"",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
new JoinableFactoryWrapper(QueryStackTests.makeJoinableFactoryFromDefault(
null,
null,
null))
) )
) )
.setGranularity(Granularities.ALL) .setGranularity(Granularities.ALL)
@ -749,7 +758,8 @@ public class ClientQuerySegmentWalkerTest
"\"j.s\" == \"s\"", "\"j.s\" == \"s\"",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
null
) )
) )
.setGranularity(Granularities.ALL) .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.query.spec.SpecificSegmentSpec;
import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SegmentReference;
import org.apache.druid.segment.filter.Filters;
import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.VersionedIntervalTimeline;
@ -138,11 +137,9 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker
throw new ISE("Cannot handle subquery: %s", analysis.getDataSource()); throw new ISE("Cannot handle subquery: %s", analysis.getDataSource());
} }
final Function<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( final Function<SegmentReference, SegmentReference> segmentMapFn = analysis.getDataSource().createSegmentMapFunction(
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), query,
analysis.getPreJoinableClauses(), new AtomicLong()
new AtomicLong(),
analysis.getBaseQuery().orElse(query)
); );
final QueryRunner<T> baseRunner = new FinalizeResultsQueryRunner<>( 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.data.input.InputSource;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.query.DataSource; 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 org.apache.druid.segment.column.RowSignature;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
import java.util.Set; 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. * 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; 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 @Override
public boolean equals(Object o) 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.math.expr.ExprMacroTable;
import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryContexts; 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.Access;
import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.AuthenticationResult;
import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceAction;
@ -48,7 +49,6 @@ import org.joda.time.DateTimeZone;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -78,6 +78,7 @@ public class PlannerContext
private final String sql; private final String sql;
private final DruidOperatorTable operatorTable; private final DruidOperatorTable operatorTable;
private final ExprMacroTable macroTable; private final ExprMacroTable macroTable;
private final JoinableFactoryWrapper joinableFactoryWrapper;
private final ObjectMapper jsonMapper; private final ObjectMapper jsonMapper;
private final PlannerConfig plannerConfig; private final PlannerConfig plannerConfig;
private final DateTime localNow; private final DateTime localNow;
@ -112,7 +113,8 @@ public class PlannerContext
final DruidSchemaCatalog rootSchema, final DruidSchemaCatalog rootSchema,
final SqlEngine engine, final SqlEngine engine,
final Map<String, Object> queryContext, final Map<String, Object> queryContext,
final Set<String> contextKeys final Set<String> contextKeys,
final JoinableFactoryWrapper joinableFactoryWrapper
) )
{ {
this.sql = sql; this.sql = sql;
@ -125,6 +127,7 @@ public class PlannerContext
this.queryContext = queryContext; this.queryContext = queryContext;
this.localNow = Preconditions.checkNotNull(localNow, "localNow"); this.localNow = Preconditions.checkNotNull(localNow, "localNow");
this.stringifyArrays = stringifyArrays; this.stringifyArrays = stringifyArrays;
this.joinableFactoryWrapper = joinableFactoryWrapper;
String sqlQueryId = (String) this.queryContext.get(QueryContexts.CTX_SQL_QUERY_ID); String sqlQueryId = (String) this.queryContext.get(QueryContexts.CTX_SQL_QUERY_ID);
// special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle // special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle
@ -143,7 +146,8 @@ public class PlannerContext
final DruidSchemaCatalog rootSchema, final DruidSchemaCatalog rootSchema,
final SqlEngine engine, final SqlEngine engine,
final Map<String, Object> queryContext, final Map<String, Object> queryContext,
final Set<String> contextKeys final Set<String> contextKeys,
final JoinableFactoryWrapper joinableFactoryWrapper
) )
{ {
final DateTime utcNow; final DateTime utcNow;
@ -183,7 +187,8 @@ public class PlannerContext
rootSchema, rootSchema,
engine, engine,
queryContext, queryContext,
contextKeys contextKeys,
joinableFactoryWrapper
); );
} }
@ -217,6 +222,11 @@ public class PlannerContext
return localNow.getZone(); return localNow.getZone();
} }
public JoinableFactoryWrapper getJoinableFactoryWrapper()
{
return joinableFactoryWrapper;
}
@Nullable @Nullable
public String getSchemaResourceType(String schema, String resourceName) 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.calcite.tools.ValidationException;
import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Json;
import org.apache.druid.math.expr.ExprMacroTable; 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.Access;
import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.server.security.NoopEscalator;
@ -72,6 +73,7 @@ public class PlannerFactory
private final AuthorizerMapper authorizerMapper; private final AuthorizerMapper authorizerMapper;
private final String druidSchemaName; private final String druidSchemaName;
private final CalciteRulesManager calciteRuleManager; private final CalciteRulesManager calciteRuleManager;
private final JoinableFactoryWrapper joinableFactoryWrapper;
@Inject @Inject
public PlannerFactory( public PlannerFactory(
@ -82,7 +84,8 @@ public class PlannerFactory
final AuthorizerMapper authorizerMapper, final AuthorizerMapper authorizerMapper,
final @Json ObjectMapper jsonMapper, final @Json ObjectMapper jsonMapper,
final @DruidSchemaName String druidSchemaName, final @DruidSchemaName String druidSchemaName,
final CalciteRulesManager calciteRuleManager final CalciteRulesManager calciteRuleManager,
final JoinableFactoryWrapper joinableFactoryWrapper
) )
{ {
this.rootSchema = rootSchema; this.rootSchema = rootSchema;
@ -93,6 +96,7 @@ public class PlannerFactory
this.jsonMapper = jsonMapper; this.jsonMapper = jsonMapper;
this.druidSchemaName = druidSchemaName; this.druidSchemaName = druidSchemaName;
this.calciteRuleManager = calciteRuleManager; this.calciteRuleManager = calciteRuleManager;
this.joinableFactoryWrapper = joinableFactoryWrapper;
} }
/** /**
@ -114,7 +118,8 @@ public class PlannerFactory
rootSchema, rootSchema,
engine, engine,
queryContext, queryContext,
contextKeys contextKeys,
joinableFactoryWrapper
); );
return new DruidPlanner(buildFrameworkConfig(context), context, engine); return new DruidPlanner(buildFrameworkConfig(context), context, engine);

View File

@ -195,7 +195,8 @@ public class DruidJoinQueryRel extends DruidRel<DruidJoinQueryRel>
condition.getExpression(), condition.getExpression(),
toDruidJoinType(joinRel.getJoinType()), toDruidJoinType(joinRel.getJoinType()),
getDimFilter(getPlannerContext(), leftSignature, leftFilter), getDimFilter(getPlannerContext(), leftSignature, leftFilter),
getPlannerContext().getExprMacroTable() getPlannerContext().getExprMacroTable(),
getPlannerContext().getJoinableFactoryWrapper()
), ),
prefixSignaturePair.rhs, prefixSignaturePair.rhs,
getPlannerContext(), 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.RowSignature;
import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.column.ValueType; 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.Aggregation;
import org.apache.druid.sql.calcite.aggregation.DimensionExpression; import org.apache.druid.sql.calcite.aggregation.DimensionExpression;
import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.DruidExpression;
@ -748,7 +749,8 @@ public class DruidQuery
static Pair<DataSource, Filtration> getFiltration( static Pair<DataSource, Filtration> getFiltration(
DataSource dataSource, DataSource dataSource,
DimFilter filter, DimFilter filter,
VirtualColumnRegistry virtualColumnRegistry VirtualColumnRegistry virtualColumnRegistry,
JoinableFactoryWrapper joinableFactoryWrapper
) )
{ {
if (!(dataSource instanceof JoinDataSource)) { if (!(dataSource instanceof JoinDataSource)) {
@ -768,13 +770,16 @@ public class DruidQuery
// Adds the intervals from the join left filter to query filtration // Adds the intervals from the join left filter to query filtration
Filtration queryFiltration = Filtration.create(filter, leftFiltration.getIntervals()) Filtration queryFiltration = Filtration.create(filter, leftFiltration.getIntervals())
.optimize(virtualColumnRegistry.getFullRowSignature()); .optimize(virtualColumnRegistry.getFullRowSignature());
JoinDataSource newDataSource = JoinDataSource.create( JoinDataSource newDataSource = JoinDataSource.create(
joinDataSource.getLeft(), joinDataSource.getLeft(),
joinDataSource.getRight(), joinDataSource.getRight(),
joinDataSource.getRightPrefix(), joinDataSource.getRightPrefix(),
joinDataSource.getConditionAnalysis(), joinDataSource.getConditionAnalysis(),
joinDataSource.getJoinType(), joinDataSource.getJoinType(),
leftFiltration.getDimFilter() leftFiltration.getDimFilter(),
joinableFactoryWrapper
); );
return Pair.of(newDataSource, queryFiltration); return Pair.of(newDataSource, queryFiltration);
} }
@ -946,7 +951,8 @@ public class DruidQuery
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration( final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
dataSource, dataSource,
filter, filter,
virtualColumnRegistry virtualColumnRegistry,
plannerContext.getJoinableFactoryWrapper()
); );
final DataSource newDataSource = dataSourceFiltrationPair.lhs; final DataSource newDataSource = dataSourceFiltrationPair.lhs;
final Filtration filtration = dataSourceFiltrationPair.rhs; final Filtration filtration = dataSourceFiltrationPair.rhs;
@ -1056,7 +1062,8 @@ public class DruidQuery
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration( final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
dataSource, dataSource,
filter, filter,
virtualColumnRegistry virtualColumnRegistry,
plannerContext.getJoinableFactoryWrapper()
); );
final DataSource newDataSource = dataSourceFiltrationPair.lhs; final DataSource newDataSource = dataSourceFiltrationPair.lhs;
final Filtration filtration = dataSourceFiltrationPair.rhs; final Filtration filtration = dataSourceFiltrationPair.rhs;
@ -1154,7 +1161,8 @@ public class DruidQuery
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration( final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
dataSource, dataSource,
filter, filter,
virtualColumnRegistry virtualColumnRegistry,
plannerContext.getJoinableFactoryWrapper()
); );
final DataSource newDataSource = dataSourceFiltrationPair.lhs; final DataSource newDataSource = dataSourceFiltrationPair.lhs;
final Filtration filtration = dataSourceFiltrationPair.rhs; final Filtration filtration = dataSourceFiltrationPair.rhs;
@ -1199,7 +1207,8 @@ public class DruidQuery
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration( final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
dataSource, dataSource,
filter, filter,
virtualColumnRegistry virtualColumnRegistry,
plannerContext.getJoinableFactoryWrapper()
); );
final DataSource newDataSource = dataSourceFiltrationPair.lhs; final DataSource newDataSource = dataSourceFiltrationPair.lhs;
final Filtration filtration = dataSourceFiltrationPair.rhs; final Filtration filtration = dataSourceFiltrationPair.rhs;
@ -1324,7 +1333,8 @@ public class DruidQuery
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration( final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
dataSource, dataSource,
filter, filter,
virtualColumnRegistry virtualColumnRegistry,
plannerContext.getJoinableFactoryWrapper()
); );
final DataSource newDataSource = dataSourceFiltrationPair.lhs; final DataSource newDataSource = dataSourceFiltrationPair.lhs;
final Filtration filtration = dataSourceFiltrationPair.rhs; 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.Query;
import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QueryScheduler;
import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.ServerConfig;
@ -144,6 +145,7 @@ public class SqlStatementTest
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
testRequestLogger = new TestRequestLogger(); testRequestLogger = new TestRequestLogger();
final JoinableFactoryWrapper joinableFactoryWrapper = CalciteTests.createJoinableFactoryWrapper();
final PlannerFactory plannerFactory = new PlannerFactory( final PlannerFactory plannerFactory = new PlannerFactory(
rootSchema, rootSchema,
@ -153,7 +155,8 @@ public class SqlStatementTest
CalciteTests.TEST_AUTHORIZER_MAPPER, CalciteTests.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(), CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()) new CalciteRulesManager(ImmutableSet.of()),
joinableFactoryWrapper
); );
this.sqlStatementFactory = new SqlStatementFactory( 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.BaseQuery;
import org.apache.druid.query.DefaultQueryConfig; import org.apache.druid.query.DefaultQueryConfig;
import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.server.DruidNode; import org.apache.druid.server.DruidNode;
import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryLifecycleFactory;
import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QueryScheduler;
@ -289,6 +290,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
binder.install(new SqlModule.SqlStatementFactoryModule()); binder.install(new SqlModule.SqlStatementFactoryModule());
binder.bind(new TypeLiteral<Supplier<DefaultQueryConfig>>(){}).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of()))); binder.bind(new TypeLiteral<Supplier<DefaultQueryConfig>>(){}).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of())));
binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of())); binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of()));
binder.bind(JoinableFactoryWrapper.class).toInstance(CalciteTests.createJoinableFactoryWrapper());
} }
) )
.build(); .build();
@ -984,7 +986,8 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(), CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, 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 List<Meta.Frame> frames = new ArrayList<>();
final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame"); final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame");
DruidMeta smallFrameDruidMeta = new DruidMeta( DruidMeta smallFrameDruidMeta = new DruidMeta(
makeStatementFactory(), makeStatementFactory(),
config, config,
@ -1058,6 +1062,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
final List<Meta.Frame> frames = new ArrayList<>(); final List<Meta.Frame> frames = new ArrayList<>();
final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame"); final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame");
DruidMeta smallFrameDruidMeta = new DruidMeta( DruidMeta smallFrameDruidMeta = new DruidMeta(
makeStatementFactory(), makeStatementFactory(),
config, 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.java.util.common.io.Closer;
import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.security.AllowAllAuthenticator; import org.apache.druid.server.security.AllowAllAuthenticator;
import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthTestUtils;
@ -103,6 +104,7 @@ public class DruidStatementTest extends CalciteTestBase
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
DruidSchemaCatalog rootSchema = DruidSchemaCatalog rootSchema =
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
final JoinableFactoryWrapper joinableFactoryWrapper = CalciteTests.createJoinableFactoryWrapper();
final PlannerFactory plannerFactory = new PlannerFactory( final PlannerFactory plannerFactory = new PlannerFactory(
rootSchema, rootSchema,
operatorTable, operatorTable,
@ -111,7 +113,8 @@ public class DruidStatementTest extends CalciteTestBase
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(), CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()) new CalciteRulesManager(ImmutableSet.of()),
joinableFactoryWrapper
); );
this.sqlStatementFactory = CalciteTests.createSqlStatementFactory( this.sqlStatementFactory = CalciteTests.createSqlStatementFactory(
CalciteTests.createMockSqlEngine(walker, conglomerate), CalciteTests.createMockSqlEngine(walker, conglomerate),

View File

@ -106,7 +106,6 @@ import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder; import org.junit.rules.TemporaryFolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
@ -420,7 +419,8 @@ public class BaseCalciteQueryTest extends CalciteTestBase implements QueryCompon
condition, condition,
joinType, joinType,
filter, filter,
CalciteTests.createExprMacroTable() CalciteTests.createExprMacroTable(),
CalciteTests.createJoinableFactoryWrapper()
); );
} }

View File

@ -142,7 +142,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
"(\"_d0\" == \"j0.d0\")", "(\"_d0\" == \"j0.d0\")",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
CalciteTests.createJoinableFactoryWrapper()
) )
) )
.columns("_d0") .columns("_d0")
@ -199,7 +200,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
"(\"dim4\" == \"j0._d0\")", "(\"dim4\" == \"j0._d0\")",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
CalciteTests.createJoinableFactoryWrapper()
) )
) )
.context(context) .context(context)
@ -3248,7 +3250,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
"(\"dim2\" == \"j0.dim2\")", "(\"dim2\" == \"j0.dim2\")",
JoinType.LEFT, JoinType.LEFT,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
CalciteTests.createJoinableFactoryWrapper()
) )
) )
.setInterval(querySegmentSpec(Filtration.eternity())) .setInterval(querySegmentSpec(Filtration.eternity()))
@ -5088,7 +5091,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
"(\"l1\" == \"j0.ROW_VALUE\")", "(\"l1\" == \"j0.ROW_VALUE\")",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
CalciteTests.createJoinableFactoryWrapper()
) )
) )
.columns("l1") .columns("l1")

View File

@ -3944,7 +3944,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
"(\"dim1\" == \"j0.ROW_VALUE\")", "(\"dim1\" == \"j0.ROW_VALUE\")",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
CalciteTests.createJoinableFactoryWrapper()
) )
) )
.intervals(querySegmentSpec(Filtration.eternity())) .intervals(querySegmentSpec(Filtration.eternity()))
@ -13090,7 +13091,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
"(\"__time\" == \"j0.__time\")", "(\"__time\" == \"j0.__time\")",
JoinType.INNER, JoinType.INNER,
null, null,
ExprMacroTable.nil() ExprMacroTable.nil(),
CalciteTests.createJoinableFactoryWrapper()
)) ))
.intervals(querySegmentSpec(Filtration.eternity())) .intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL) .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.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator; 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.QueryStackTests;
import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager; import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
@ -140,6 +141,7 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe
final PlannerConfig plannerConfig = new PlannerConfig(); final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchemaCatalog rootSchema = final DruidSchemaCatalog rootSchema =
CalciteTests.createMockRootSchema(CONGLOMERATE, WALKER, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); CalciteTests.createMockRootSchema(CONGLOMERATE, WALKER, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
final JoinableFactoryWrapper joinableFactoryWrapper = CalciteTests.createJoinableFactoryWrapper();
ENGINE = CalciteTests.createMockSqlEngine(WALKER, CONGLOMERATE); ENGINE = CalciteTests.createMockSqlEngine(WALKER, CONGLOMERATE);
PLANNER_FACTORY = new PlannerFactory( PLANNER_FACTORY = new PlannerFactory(
rootSchema, rootSchema,
@ -149,7 +151,8 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe
AuthTestUtils.TEST_AUTHORIZER_MAPPER, AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(), CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, 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.VirtualColumn;
import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.RowSignature; 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.segment.virtual.VirtualizedColumnSelectorFactory;
import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.DruidTypeSystem; import org.apache.druid.sql.calcite.planner.DruidTypeSystem;
@ -62,7 +63,6 @@ import org.joda.time.DateTimeZone;
import org.junit.Assert; import org.junit.Assert;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.math.BigDecimal; import java.math.BigDecimal;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
@ -74,6 +74,7 @@ import java.util.stream.Collectors;
class ExpressionTestHelper class ExpressionTestHelper
{ {
private static final JoinableFactoryWrapper JOINABLE_FACTORY_WRAPPER = CalciteTests.createJoinableFactoryWrapper();
private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create( private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create(
"SELECT 1", // The actual query isn't important for this test "SELECT 1", // The actual query isn't important for this test
CalciteTests.createOperatorTable(), CalciteTests.createOperatorTable(),
@ -89,7 +90,8 @@ class ExpressionTestHelper
), ),
null /* Don't need engine */, null /* Don't need engine */,
Collections.emptyMap(), Collections.emptyMap(),
Collections.emptySet() Collections.emptySet(),
JOINABLE_FACTORY_WRAPPER
); );
private final RowSignature rowSignature; private final RowSignature rowSignature;

View File

@ -64,7 +64,8 @@ public class ExternalTableScanRuleTest
), ),
engine, engine,
Collections.emptyMap(), Collections.emptyMap(),
Collections.emptySet() Collections.emptySet(),
CalciteTests.createJoinableFactoryWrapper()
); );
plannerContext.setQueryMaker( plannerContext.setQueryMaker(
engine.buildQueryMakerForSelect(EasyMock.createMock(RelRoot.class), plannerContext) 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.DefaultObjectMapper;
import org.apache.druid.jackson.JacksonModule; import org.apache.druid.jackson.JacksonModule;
import org.apache.druid.math.expr.ExprMacroTable; 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.QueryLifecycleFactory;
import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.ResourceType; import org.apache.druid.server.security.ResourceType;
@ -54,7 +55,6 @@ import org.junit.runner.RunWith;
import javax.validation.Validation; import javax.validation.Validation;
import javax.validation.Validator; import javax.validation.Validator;
import java.util.Collections; import java.util.Collections;
import java.util.Set; import java.util.Set;
@ -81,6 +81,8 @@ public class CalcitePlannerModuleTest extends CalciteTestBase
@Mock @Mock
private ExprMacroTable macroTable; private ExprMacroTable macroTable;
@Mock @Mock
private JoinableFactoryWrapper joinableFactoryWrapper;
@Mock
private AuthorizerMapper authorizerMapper; private AuthorizerMapper authorizerMapper;
@Mock @Mock
private DruidSchemaCatalog rootSchema; 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<SqlAggregator>>() {})).toInstance(aggregators);
binder.bind(Key.get(new TypeLiteral<Set<SqlOperatorConversion>>() {})).toInstance(operatorConversions); binder.bind(Key.get(new TypeLiteral<Set<SqlOperatorConversion>>() {})).toInstance(operatorConversions);
binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema); binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema);
binder.bind(JoinableFactoryWrapper.class).toInstance(joinableFactoryWrapper);
}, },
target, target,
binder -> { binder -> {
@ -176,7 +179,8 @@ public class CalcitePlannerModuleTest extends CalciteTestBase
rootSchema, rootSchema,
null, null,
Collections.emptyMap(), Collections.emptyMap(),
Collections.emptySet() Collections.emptySet(),
joinableFactoryWrapper
); );
boolean containsCustomRule = injector.getInstance(CalciteRulesManager.class) boolean containsCustomRule = injector.getInstance(CalciteRulesManager.class)
.druidConventionRuleSet(context) .druidConventionRuleSet(context)

View File

@ -97,7 +97,8 @@ public class DruidRexExecutorTest extends InitializedNullHandlingTest
), ),
null /* Don't need an engine */, null /* Don't need an engine */,
Collections.emptyMap(), Collections.emptyMap(),
Collections.emptySet() Collections.emptySet(),
CalciteTests.createJoinableFactoryWrapper()
); );
private final RexBuilder rexBuilder = new RexBuilder(new JavaTypeFactoryImpl()); 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.column.RowSignature;
import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.join.JoinType;
import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -59,10 +60,12 @@ public class DruidQueryTest
public void test_filtration_noJoinAndInterval() public void test_filtration_noJoinAndInterval()
{ {
DataSource dataSource = new TableDataSource("test"); DataSource dataSource = new TableDataSource("test");
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration( Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource, dataSource,
selectorFilter, selectorFilter,
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
CalciteTests.createJoinableFactoryWrapper()
); );
verify(pair, dataSource, selectorFilter, Intervals.ETERNITY); verify(pair, dataSource, selectorFilter, Intervals.ETERNITY);
} }
@ -74,7 +77,8 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration( Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource, dataSource,
filterWithInterval, 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)); verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200));
} }
@ -86,7 +90,8 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration( Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource, dataSource,
filterWithInterval, 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)); verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200));
} }
@ -99,7 +104,8 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration( Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource, dataSource,
otherFilter, 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)); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
} }
@ -112,7 +118,8 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration( Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource, dataSource,
otherFilter, 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)); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
} }
@ -125,7 +132,8 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration( Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource, dataSource,
otherFilter, 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)); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
} }
@ -138,7 +146,8 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration( Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource, dataSource,
otherFilter, 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)); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
} }
@ -156,7 +165,8 @@ public class DruidQueryTest
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration( Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
dataSource, dataSource,
queryFilter, 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)); verify(pair, expectedDataSource, otherFilter, Intervals.utc(150, 200));
} }
@ -170,7 +180,8 @@ public class DruidQueryTest
"c == \"r.c\"", "c == \"r.c\"",
joinType, joinType,
filter, filter,
ExprMacroTable.nil() ExprMacroTable.nil(),
CalciteTests.createJoinableFactoryWrapper()
); );
} }

View File

@ -283,6 +283,11 @@ public class CalciteTests
return QueryFrameworkUtils.createExprMacroTable(INJECTOR); return QueryFrameworkUtils.createExprMacroTable(INJECTOR);
} }
public static JoinableFactoryWrapper createJoinableFactoryWrapper()
{
return new JoinableFactoryWrapper(QueryFrameworkUtils.createDefaultJoinableFactory(INJECTOR));
}
public static DruidOperatorTable createOperatorTable() public static DruidOperatorTable createOperatorTable()
{ {
return QueryFrameworkUtils.createOperatorTable(INJECTOR); return QueryFrameworkUtils.createOperatorTable(INJECTOR);

View File

@ -373,7 +373,8 @@ public class SqlTestFramework
authorizerMapper, authorizerMapper,
queryJsonMapper(), queryJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()) new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
); );
final SqlStatementFactory sqlStatementFactory = QueryFrameworkUtils.createSqlStatementFactory( final SqlStatementFactory sqlStatementFactory = QueryFrameworkUtils.createSqlStatementFactory(
engine, engine,

View File

@ -222,6 +222,7 @@ public class SqlResourceTest extends CalciteTestBase
); );
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
req = request(true); req = request(true);
testRequestLogger = new TestRequestLogger(); testRequestLogger = new TestRequestLogger();
@ -234,7 +235,8 @@ public class SqlResourceTest extends CalciteTestBase
CalciteTests.TEST_AUTHORIZER_MAPPER, CalciteTests.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper(), CalciteTests.getJsonMapper(),
CalciteTests.DRUID_SCHEMA_NAME, CalciteTests.DRUID_SCHEMA_NAME,
new CalciteRulesManager(ImmutableSet.of()) new CalciteRulesManager(ImmutableSet.of()),
CalciteTests.createJoinableFactoryWrapper()
); );
lifecycleManager = new SqlLifecycleManager() lifecycleManager = new SqlLifecycleManager()