mirror of https://github.com/apache/druid.git
Refactoring the data source before unnest (#13085)
* First set of changes for framework
* Second set of changes to move segment map function to data source
* Minot change to server manager
* Removing the createSegmentMapFunction from JoinableFactoryWrapper and moving to JoinDataSource
* Checkstyle fixes
* Patching Eric's fix for injection
* Checkstyle and fixing some CI issues
* Fixing code inspections and some failed tests and one injector for test in avatica
* Another set of changes for CI...almost there
* Equals and hashcode part update
* Fixing injector from Eric + refactoring for broadcastJoinHelper
* Updating second injector. Might revert later if better way found
* Fixing guice issue in JoinableFactory
* Addressing review comments part 1
* Temp changes refactoring
* Revert "Temp changes refactoring"
This reverts commit 9da42a9ef0
.
* temp
* Temp discussions
* Refactoring temp
* Refatoring the query rewrite to refer to a datasource
* Refactoring getCacheKey by moving it inside data source
* Nullable annotation check in injector
* Addressing some comments, removing 2 analysis.isJoin() checks and correcting the benchmark files
* Minor changes for refactoring
* Addressing reviews part 1
* Refactoring part 2 with new test cases for broadcast join
* Set for nullables
* removing instance of checks
* Storing nullables in guice to avoid checking on reruns
* Fixing a test case and removing an irrelevant line
* Addressing the atomic reference review comments
This commit is contained in:
parent
72c16097ac
commit
affc522b9f
|
@ -484,7 +484,8 @@ public class SqlBenchmark
|
|||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -321,7 +321,8 @@ public class SqlExpressionBenchmark
|
|||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
|
||||
try {
|
||||
|
|
|
@ -310,7 +310,8 @@ public class SqlNestedDataBenchmark
|
|||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
|
||||
try {
|
||||
|
|
|
@ -127,7 +127,8 @@ public class SqlVsNativeBenchmark
|
|||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
groupByQuery = GroupByQuery
|
||||
.builder()
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.guice;
|
|||
import com.fasterxml.jackson.databind.AnnotationIntrospector;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Inject;
|
||||
import com.google.inject.Injector;
|
||||
|
@ -96,14 +95,12 @@ public class DruidSecondaryModule implements Module
|
|||
return smileMapper;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void setupJackson(Injector injector, final ObjectMapper mapper)
|
||||
{
|
||||
mapper.setInjectableValues(new GuiceInjectableValues(injector));
|
||||
setupAnnotationIntrospector(mapper, new GuiceAnnotationIntrospector());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void setupAnnotationIntrospector(
|
||||
final ObjectMapper mapper,
|
||||
final AnnotationIntrospector annotationIntrospector
|
||||
|
|
|
@ -22,19 +22,27 @@ package org.apache.druid.guice;
|
|||
import com.fasterxml.jackson.databind.BeanProperty;
|
||||
import com.fasterxml.jackson.databind.DeserializationContext;
|
||||
import com.fasterxml.jackson.databind.InjectableValues;
|
||||
import com.google.inject.ConfigurationException;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.HashSet;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class GuiceInjectableValues extends InjectableValues
|
||||
{
|
||||
private final Injector injector;
|
||||
private final AtomicReference<HashSet<Key>> nullables;
|
||||
|
||||
public GuiceInjectableValues(Injector injector)
|
||||
{
|
||||
this.injector = injector;
|
||||
this.nullables = new AtomicReference<>(new HashSet<>());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -49,8 +57,22 @@ public class GuiceInjectableValues extends InjectableValues
|
|||
// whatever provider needs"
|
||||
// Currently we should only be dealing with `Key` instances, and anything more advanced should be handled with
|
||||
// great care
|
||||
if (valueId instanceof Key) {
|
||||
return injector.getInstance((Key) valueId);
|
||||
if (nullables.get().contains((Key) valueId)) {
|
||||
return null;
|
||||
} else if (valueId instanceof Key) {
|
||||
try {
|
||||
return injector.getInstance((Key) valueId);
|
||||
}
|
||||
catch (ConfigurationException ce) {
|
||||
// check if nullable annotation is present for this
|
||||
if (forProperty.getAnnotation(Nullable.class) != null) {
|
||||
HashSet<Key> encounteredNullables = new HashSet<>(nullables.get());
|
||||
encounteredNullables.add((Key) valueId);
|
||||
nullables.set(encounteredNullables);
|
||||
return null;
|
||||
}
|
||||
throw ce;
|
||||
}
|
||||
}
|
||||
throw new IAE(
|
||||
"Unknown class type [%s] for valueId [%s]",
|
||||
|
|
|
@ -47,6 +47,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
|
||||
public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
|
||||
|
@ -88,6 +89,58 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
|
|||
this.broadcastJoinHelper = inputChannelsAndBroadcastJoinHelper.rhs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up their primary and side channels.
|
||||
*/
|
||||
private static Pair<List<ReadableFrameChannel>, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper(
|
||||
final DataSource dataSource,
|
||||
final ReadableInput baseInput,
|
||||
final Int2ObjectMap<ReadableInput> sideChannels,
|
||||
final JoinableFactoryWrapper joinableFactory,
|
||||
final long memoryReservedForBroadcastJoin
|
||||
)
|
||||
{
|
||||
if (!(dataSource instanceof JoinDataSource) && !sideChannels.isEmpty()) {
|
||||
throw new ISE("Did not expect side channels for dataSource [%s]", dataSource);
|
||||
}
|
||||
|
||||
final List<ReadableFrameChannel> inputChannels = new ArrayList<>();
|
||||
final BroadcastJoinHelper broadcastJoinHelper;
|
||||
|
||||
if (baseInput.hasChannel()) {
|
||||
inputChannels.add(baseInput.getChannel());
|
||||
}
|
||||
|
||||
if (dataSource instanceof JoinDataSource) {
|
||||
final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap();
|
||||
final List<FrameReader> channelReaders = new ArrayList<>();
|
||||
|
||||
if (baseInput.hasChannel()) {
|
||||
// BroadcastJoinHelper doesn't need to read the base channel, so stub in a null reader.
|
||||
channelReaders.add(null);
|
||||
}
|
||||
|
||||
for (Int2ObjectMap.Entry<ReadableInput> sideChannelEntry : sideChannels.int2ObjectEntrySet()) {
|
||||
final int inputNumber = sideChannelEntry.getIntKey();
|
||||
inputNumberToProcessorChannelMap.put(inputNumber, inputChannels.size());
|
||||
inputChannels.add(sideChannelEntry.getValue().getChannel());
|
||||
channelReaders.add(sideChannelEntry.getValue().getChannelFrameReader());
|
||||
}
|
||||
|
||||
broadcastJoinHelper = new BroadcastJoinHelper(
|
||||
inputNumberToProcessorChannelMap,
|
||||
inputChannels,
|
||||
channelReaders,
|
||||
joinableFactory,
|
||||
memoryReservedForBroadcastJoin
|
||||
);
|
||||
} else {
|
||||
broadcastJoinHelper = null;
|
||||
}
|
||||
|
||||
return Pair.of(inputChannels, broadcastJoinHelper);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ReadableFrameChannel> inputChannels()
|
||||
{
|
||||
|
@ -146,6 +199,7 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
|
|||
|
||||
private boolean initializeSegmentMapFn(final IntSet readableInputs)
|
||||
{
|
||||
final AtomicLong cpuAccumulator = new AtomicLong();
|
||||
if (segmentMapFn != null) {
|
||||
return true;
|
||||
} else if (broadcastJoinHelper == null) {
|
||||
|
@ -153,64 +207,11 @@ public abstract class BaseLeafFrameProcessor implements FrameProcessor<Long>
|
|||
return true;
|
||||
} else {
|
||||
final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs);
|
||||
|
||||
DataSource inlineChannelDataSource = broadcastJoinHelper.inlineChannelData(query.getDataSource());
|
||||
if (retVal) {
|
||||
segmentMapFn = broadcastJoinHelper.makeSegmentMapFn(query);
|
||||
segmentMapFn = inlineChannelDataSource.createSegmentMapFunction(query, cpuAccumulator);
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up their primary and side channels.
|
||||
*/
|
||||
private static Pair<List<ReadableFrameChannel>, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper(
|
||||
final DataSource dataSource,
|
||||
final ReadableInput baseInput,
|
||||
final Int2ObjectMap<ReadableInput> sideChannels,
|
||||
final JoinableFactoryWrapper joinableFactory,
|
||||
final long memoryReservedForBroadcastJoin
|
||||
)
|
||||
{
|
||||
if (!(dataSource instanceof JoinDataSource) && !sideChannels.isEmpty()) {
|
||||
throw new ISE("Did not expect side channels for dataSource [%s]", dataSource);
|
||||
}
|
||||
|
||||
final List<ReadableFrameChannel> inputChannels = new ArrayList<>();
|
||||
final BroadcastJoinHelper broadcastJoinHelper;
|
||||
|
||||
if (baseInput.hasChannel()) {
|
||||
inputChannels.add(baseInput.getChannel());
|
||||
}
|
||||
|
||||
if (dataSource instanceof JoinDataSource) {
|
||||
final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap();
|
||||
final List<FrameReader> channelReaders = new ArrayList<>();
|
||||
|
||||
if (baseInput.hasChannel()) {
|
||||
// BroadcastJoinHelper doesn't need to read the base channel, so stub in a null reader.
|
||||
channelReaders.add(null);
|
||||
}
|
||||
|
||||
for (Int2ObjectMap.Entry<ReadableInput> sideChannelEntry : sideChannels.int2ObjectEntrySet()) {
|
||||
final int inputNumber = sideChannelEntry.getIntKey();
|
||||
inputNumberToProcessorChannelMap.put(inputNumber, inputChannels.size());
|
||||
inputChannels.add(sideChannelEntry.getValue().getChannel());
|
||||
channelReaders.add(sideChannelEntry.getValue().getChannelFrameReader());
|
||||
}
|
||||
|
||||
broadcastJoinHelper = new BroadcastJoinHelper(
|
||||
inputNumberToProcessorChannelMap,
|
||||
inputChannels,
|
||||
channelReaders,
|
||||
joinableFactory,
|
||||
memoryReservedForBroadcastJoin
|
||||
);
|
||||
} else {
|
||||
broadcastJoinHelper = null;
|
||||
}
|
||||
|
||||
return Pair.of(inputChannels, broadcastJoinHelper);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.msq.querykit;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import it.unimi.dsi.fastutil.ints.Int2IntMap;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
|
||||
|
@ -32,18 +31,12 @@ import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault;
|
|||
import org.apache.druid.msq.indexing.error.MSQException;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.Cursor;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class BroadcastJoinHelper
|
||||
|
@ -138,21 +131,9 @@ public class BroadcastJoinHelper
|
|||
return sideChannelNumbers;
|
||||
}
|
||||
|
||||
public Function<SegmentReference, SegmentReference> makeSegmentMapFn(final Query<?> query)
|
||||
{
|
||||
final DataSource dataSourceWithInlinedChannelData = inlineChannelData(query.getDataSource());
|
||||
final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(dataSourceWithInlinedChannelData);
|
||||
|
||||
return joinableFactory.createSegmentMapFn(
|
||||
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null),
|
||||
analysis.getPreJoinableClauses(),
|
||||
new AtomicLong(),
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
DataSource inlineChannelData(final DataSource originalDataSource)
|
||||
public DataSource inlineChannelData(final DataSource originalDataSource)
|
||||
{
|
||||
if (originalDataSource instanceof InputNumberDataSource) {
|
||||
final int inputNumber = ((InputNumberDataSource) originalDataSource).getInputNumber();
|
||||
|
|
|
@ -315,9 +315,9 @@ public class DataSourcePlan
|
|||
clause.getPrefix(),
|
||||
clause.getCondition(),
|
||||
clause.getJoinType(),
|
||||
|
||||
// First JoinDataSource (i == 0) involves the base table, so we need to propagate the base table filter.
|
||||
i == 0 ? analysis.getJoinBaseTableFilter().orElse(null) : null
|
||||
i == 0 ? analysis.getJoinBaseTableFilter().orElse(null) : null,
|
||||
dataSource.getJoinableFactoryWrapper()
|
||||
);
|
||||
inputSpecs.addAll(clausePlan.getInputSpecs());
|
||||
clausePlan.getBroadcastInputs().intStream().forEach(n -> broadcastInputs.add(n + shift));
|
||||
|
|
|
@ -24,11 +24,15 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
|
||||
@JsonTypeName("inputNumber")
|
||||
public class InputNumberDataSource implements DataSource
|
||||
|
@ -81,6 +85,27 @@ public class InputNumberDataSource implements DataSource
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(
|
||||
Query query,
|
||||
AtomicLong cpuTimeAcc
|
||||
)
|
||||
{
|
||||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
return newSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getInputNumber()
|
||||
{
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.druid.msq.indexing.MSQTuningConfig;
|
|||
import org.apache.druid.msq.test.MSQTestBase;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
|
||||
|
@ -438,6 +439,122 @@ public class MSQSelectTest extends MSQTestBase
|
|||
.verifyResults();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBroadcastJoin()
|
||||
{
|
||||
final RowSignature resultSignature = RowSignature.builder()
|
||||
.add("dim2", ColumnType.STRING)
|
||||
.add("EXPR$1", ColumnType.DOUBLE)
|
||||
.build();
|
||||
|
||||
final ImmutableList<Object[]> expectedResults;
|
||||
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
expectedResults = ImmutableList.of(
|
||||
new Object[]{null, 4.0},
|
||||
new Object[]{"", 3.0},
|
||||
new Object[]{"a", 2.5},
|
||||
new Object[]{"abc", 5.0}
|
||||
);
|
||||
} else {
|
||||
expectedResults = ImmutableList.of(
|
||||
new Object[]{null, 3.6666666666666665},
|
||||
new Object[]{"a", 2.5},
|
||||
new Object[]{"abc", 5.0}
|
||||
);
|
||||
}
|
||||
|
||||
final GroupByQuery query =
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(
|
||||
join(
|
||||
new TableDataSource(CalciteTests.DATASOURCE1),
|
||||
new QueryDataSource(
|
||||
newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("dim2", "m1", "m2")
|
||||
.context(
|
||||
defaultScanQueryContext(
|
||||
RowSignature.builder()
|
||||
.add("dim2", ColumnType.STRING)
|
||||
.add("m1", ColumnType.FLOAT)
|
||||
.add("m2", ColumnType.DOUBLE)
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.limit(10)
|
||||
.build()
|
||||
),
|
||||
"j0.",
|
||||
equalsCondition(
|
||||
DruidExpression.ofColumn(ColumnType.FLOAT, "m1"),
|
||||
DruidExpression.ofColumn(ColumnType.FLOAT, "j0.m1")
|
||||
),
|
||||
JoinType.INNER
|
||||
)
|
||||
)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setDimensions(new DefaultDimensionSpec("j0.dim2", "d0", ColumnType.STRING))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setAggregatorSpecs(
|
||||
useDefault
|
||||
? aggregators(
|
||||
new DoubleSumAggregatorFactory("a0:sum", "j0.m2"),
|
||||
new CountAggregatorFactory("a0:count")
|
||||
)
|
||||
: aggregators(
|
||||
new DoubleSumAggregatorFactory("a0:sum", "j0.m2"),
|
||||
new FilteredAggregatorFactory(
|
||||
new CountAggregatorFactory("a0:count"),
|
||||
not(selector("j0.m2", null, null)),
|
||||
|
||||
// Not sure why the name is only set in SQL-compatible null mode. Seems strange.
|
||||
// May be due to JSON serialization: name is set on the serialized aggregator even
|
||||
// if it was originally created with no name.
|
||||
NullHandling.sqlCompatible() ? "a0:count" : null
|
||||
)
|
||||
)
|
||||
)
|
||||
.setPostAggregatorSpecs(
|
||||
ImmutableList.of(
|
||||
new ArithmeticPostAggregator(
|
||||
"a0",
|
||||
"quotient",
|
||||
ImmutableList.of(
|
||||
new FieldAccessPostAggregator(null, "a0:sum"),
|
||||
new FieldAccessPostAggregator(null, "a0:count")
|
||||
)
|
||||
)
|
||||
|
||||
)
|
||||
)
|
||||
.setContext(DEFAULT_MSQ_CONTEXT)
|
||||
.build();
|
||||
|
||||
testSelectQuery()
|
||||
.setSql(
|
||||
"SELECT t1.dim2, AVG(t1.m2) FROM "
|
||||
+ "foo "
|
||||
+ "INNER JOIN (SELECT * FROM foo LIMIT 10) AS t1 "
|
||||
+ "ON t1.m1 = foo.m1 "
|
||||
+ "GROUP BY t1.dim2"
|
||||
)
|
||||
.setExpectedMSQSpec(
|
||||
MSQSpec.builder()
|
||||
.query(query)
|
||||
.columnMappings(new ColumnMappings(ImmutableList.of(
|
||||
new ColumnMapping("d0", "dim2"),
|
||||
new ColumnMapping("a0", "EXPR$1")
|
||||
)))
|
||||
.tuningConfig(MSQTuningConfig.defaultConfig())
|
||||
.build()
|
||||
)
|
||||
.setExpectedRowSignature(resultSignature)
|
||||
.setExpectedResultRows(expectedResults)
|
||||
.verifyResults();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByOrderByAggregation()
|
||||
{
|
||||
|
|
|
@ -189,6 +189,7 @@ public class BroadcastJoinHelperTest extends InitializedNullHandlingTest
|
|||
"j.",
|
||||
JoinConditionAnalysis.forExpression("x == \"j.x\"", "j.", ExprMacroTable.nil()),
|
||||
JoinType.INNER,
|
||||
null,
|
||||
null
|
||||
)
|
||||
);
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.msq.test;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.InjectableValues;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
|
@ -30,10 +29,8 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.io.ByteStreams;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.TypeLiteral;
|
||||
import com.google.inject.util.Modules;
|
||||
import com.google.inject.util.Providers;
|
||||
|
@ -43,6 +40,7 @@ import org.apache.druid.data.input.impl.LongDimensionSchema;
|
|||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||
import org.apache.druid.discovery.NodeRole;
|
||||
import org.apache.druid.frame.testutil.FrameTestUtil;
|
||||
import org.apache.druid.guice.DruidSecondaryModule;
|
||||
import org.apache.druid.guice.GuiceInjectors;
|
||||
import org.apache.druid.guice.IndexingServiceTuningConfigModule;
|
||||
import org.apache.druid.guice.JoinableFactoryModule;
|
||||
|
@ -113,7 +111,6 @@ import org.apache.druid.segment.StorageAdapter;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
|
||||
import org.apache.druid.segment.loading.LocalDataSegmentPusher;
|
||||
import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig;
|
||||
import org.apache.druid.segment.loading.LocalLoadSpec;
|
||||
|
@ -162,7 +159,6 @@ import org.mockito.Mockito;
|
|||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
@ -253,8 +249,14 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
groupByBuffers = TestGroupByBuffers.createDefault();
|
||||
|
||||
SqlTestFramework qf = queryFramework();
|
||||
Injector secondInjector = GuiceInjectors.makeStartupInjector();
|
||||
|
||||
Injector secondInjector = GuiceInjectors.makeStartupInjectorWithModules(
|
||||
ImmutableList.of(
|
||||
binder -> {
|
||||
binder.bind(ExprMacroTable.class).toInstance(CalciteTests.createExprMacroTable());
|
||||
binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT);
|
||||
}
|
||||
)
|
||||
);
|
||||
|
||||
ObjectMapper secondMapper = setupObjectMapper(secondInjector);
|
||||
indexIO = new IndexIO(secondMapper, () -> 0);
|
||||
|
@ -271,70 +273,69 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
segmentManager = new MSQTestSegmentManager(segmentCacheManager, indexIO);
|
||||
|
||||
Injector injector = GuiceInjectors.makeStartupInjectorWithModules(ImmutableList.of(
|
||||
new Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
binder -> {
|
||||
DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
|
||||
{
|
||||
DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
|
||||
@Override
|
||||
public String getFormatString()
|
||||
{
|
||||
@Override
|
||||
public String getFormatString()
|
||||
{
|
||||
return "test";
|
||||
}
|
||||
};
|
||||
|
||||
GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig();
|
||||
|
||||
binder.bind(DruidProcessingConfig.class).toInstance(druidProcessingConfig);
|
||||
binder.bind(new TypeLiteral<Set<NodeRole>>()
|
||||
{
|
||||
}).annotatedWith(Self.class).toInstance(ImmutableSet.of(NodeRole.PEON));
|
||||
binder.bind(QueryProcessingPool.class)
|
||||
.toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool")));
|
||||
binder.bind(DataSegmentProvider.class)
|
||||
.toInstance((dataSegment, channelCounters) ->
|
||||
new LazyResourceHolder<>(getSupplierForSegment(dataSegment)));
|
||||
binder.bind(IndexIO.class).toInstance(indexIO);
|
||||
binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker());
|
||||
|
||||
binder.bind(GroupByStrategySelector.class)
|
||||
.toInstance(GroupByQueryRunnerTest.makeQueryRunnerFactory(groupByQueryConfig, groupByBuffers)
|
||||
.getStrategySelector());
|
||||
|
||||
LocalDataSegmentPusherConfig config = new LocalDataSegmentPusherConfig();
|
||||
try {
|
||||
config.storageDirectory = tmpFolder.newFolder("localsegments");
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "Unable to create folder");
|
||||
}
|
||||
binder.bind(DataSegmentPusher.class).toInstance(new MSQTestDelegateDataSegmentPusher(
|
||||
new LocalDataSegmentPusher(config),
|
||||
segmentManager
|
||||
));
|
||||
binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer());
|
||||
binder.bindConstant().annotatedWith(PruneLoadSpec.class).to(false);
|
||||
// Client is not used in tests
|
||||
binder.bind(Key.get(ServiceClientFactory.class, EscalatedGlobal.class))
|
||||
.toProvider(Providers.of(null));
|
||||
// fault tolerance module
|
||||
try {
|
||||
JsonConfigProvider.bind(
|
||||
binder,
|
||||
MSQDurableStorageModule.MSQ_INTERMEDIATE_STORAGE_PREFIX,
|
||||
StorageConnectorProvider.class,
|
||||
MultiStageQuery.class
|
||||
);
|
||||
localFileStorageDir = tmpFolder.newFolder("fault");
|
||||
binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class))
|
||||
.toProvider(new LocalFileStorageConnectorProvider(localFileStorageDir));
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "Unable to create setup storage connector");
|
||||
return "test";
|
||||
}
|
||||
};
|
||||
|
||||
GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig();
|
||||
|
||||
|
||||
binder.bind(DruidProcessingConfig.class).toInstance(druidProcessingConfig);
|
||||
binder.bind(new TypeLiteral<Set<NodeRole>>()
|
||||
{
|
||||
}).annotatedWith(Self.class).toInstance(ImmutableSet.of(NodeRole.PEON));
|
||||
binder.bind(QueryProcessingPool.class)
|
||||
.toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool")));
|
||||
binder.bind(DataSegmentProvider.class)
|
||||
.toInstance((dataSegment, channelCounters) ->
|
||||
new LazyResourceHolder<>(getSupplierForSegment(dataSegment)));
|
||||
binder.bind(IndexIO.class).toInstance(indexIO);
|
||||
binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(qf.walker());
|
||||
|
||||
binder.bind(GroupByStrategySelector.class)
|
||||
.toInstance(GroupByQueryRunnerTest.makeQueryRunnerFactory(groupByQueryConfig, groupByBuffers)
|
||||
.getStrategySelector());
|
||||
|
||||
LocalDataSegmentPusherConfig config = new LocalDataSegmentPusherConfig();
|
||||
try {
|
||||
config.storageDirectory = tmpFolder.newFolder("localsegments");
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "Unable to create folder");
|
||||
}
|
||||
binder.bind(DataSegmentPusher.class).toInstance(new MSQTestDelegateDataSegmentPusher(
|
||||
new LocalDataSegmentPusher(config),
|
||||
segmentManager
|
||||
));
|
||||
binder.bind(DataSegmentAnnouncer.class).toInstance(new NoopDataSegmentAnnouncer());
|
||||
binder.bindConstant().annotatedWith(PruneLoadSpec.class).to(false);
|
||||
// Client is not used in tests
|
||||
binder.bind(Key.get(ServiceClientFactory.class, EscalatedGlobal.class))
|
||||
.toProvider(Providers.of(null));
|
||||
// fault tolerance module
|
||||
try {
|
||||
JsonConfigProvider.bind(
|
||||
binder,
|
||||
MSQDurableStorageModule.MSQ_INTERMEDIATE_STORAGE_PREFIX,
|
||||
StorageConnectorProvider.class,
|
||||
MultiStageQuery.class
|
||||
);
|
||||
localFileStorageDir = tmpFolder.newFolder("fault");
|
||||
binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class))
|
||||
.toProvider(new LocalFileStorageConnectorProvider(localFileStorageDir));
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "Unable to create setup storage connector");
|
||||
}
|
||||
|
||||
binder.bind(ExprMacroTable.class).toInstance(CalciteTests.createExprMacroTable());
|
||||
binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT);
|
||||
},
|
||||
binder -> {
|
||||
// Requirements of JoinableFactoryModule
|
||||
|
@ -393,7 +394,8 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
objectMapper,
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
|
||||
sqlStatementFactory = CalciteTests.createSqlStatementFactory(engine, plannerFactory);
|
||||
|
@ -568,14 +570,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
"compaction"
|
||||
)
|
||||
).registerSubtypes(ExternalDataSource.class));
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std()
|
||||
.addValue(ObjectMapper.class, mapper)
|
||||
.addValue(Injector.class, injector)
|
||||
.addValue(DataSegment.PruneSpecsHolder.class, DataSegment.PruneSpecsHolder.DEFAULT)
|
||||
.addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller())
|
||||
.addValue(ExprMacroTable.class, CalciteTests.createExprMacroTable())
|
||||
);
|
||||
DruidSecondaryModule.setupJackson(injector, mapper);
|
||||
|
||||
mapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local"));
|
||||
|
||||
|
|
|
@ -21,9 +21,14 @@ package org.apache.druid.query;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.query.planning.PreJoinableClause;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Represents a source... of data... for a query. Analogous to the "FROM" clause in SQL.
|
||||
|
@ -83,8 +88,39 @@ public interface DataSource
|
|||
* Returns true if this datasource represents concrete data that can be scanned via a
|
||||
* {@link org.apache.druid.segment.Segment} adapter of some kind. True for e.g. 'table' but not for 'query' or 'join'.
|
||||
*
|
||||
* @see org.apache.druid.query.planning.DataSourceAnalysis#isConcreteBased() which uses this
|
||||
* @see org.apache.druid.query.planning.DataSourceAnalysis#isConcreteTableBased() which uses this
|
||||
* @see DataSourceAnalysis#isConcreteBased() which uses this
|
||||
* @see DataSourceAnalysis#isConcreteTableBased() which uses this
|
||||
*/
|
||||
boolean isConcrete();
|
||||
|
||||
/**
|
||||
* Returns a segment function on to how to segment should be modified.
|
||||
*
|
||||
* @param query the input query
|
||||
* @param cpuTimeAcc the cpu time accumulator
|
||||
* @return the segment function
|
||||
*/
|
||||
Function<SegmentReference, SegmentReference> createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc);
|
||||
|
||||
/**
|
||||
* Returns an updated datasource based on the specified new source.
|
||||
*
|
||||
* @param newSource the new datasource to be used to update an existing query
|
||||
* @return the updated datasource to be used
|
||||
*/
|
||||
DataSource withUpdatedDataSource(DataSource newSource);
|
||||
|
||||
/**
|
||||
* Compute a cache key prefix for a data source. This includes the data sources that participate in the RHS of a
|
||||
* join as well as any query specific constructs associated with join data source such as base table filter. This key prefix
|
||||
* can be used in segment level cache or result level cache. The function can return following
|
||||
* - Non-empty byte array - If there is join datasource involved and caching is possible. The result includes
|
||||
* join condition expression, join type and cache key returned by joinable factory for each {@link PreJoinableClause}
|
||||
* - NULL - There is a join but caching is not possible. It may happen if one of the participating datasource
|
||||
* in the JOIN is not cacheable.
|
||||
*
|
||||
* @return the cache key to be used as part of query cache key
|
||||
*/
|
||||
byte[] getCacheKey();
|
||||
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.segment.RowAdapter;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
|
@ -36,12 +37,14 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
/**
|
||||
* Represents an inline datasource, where the rows are embedded within the DataSource object itself.
|
||||
*
|
||||
* <p>
|
||||
* The rows are backed by an Iterable, which can be lazy or not. Lazy datasources will only be iterated if someone calls
|
||||
* {@link #getRows()} and iterates the result, or until someone calls {@link #getRowsAsList()}.
|
||||
*/
|
||||
|
@ -102,6 +105,55 @@ public class InlineDataSource implements DataSource
|
|||
return new InlineDataSource(rows, signature);
|
||||
}
|
||||
|
||||
/**
|
||||
* A very zealous equality checker for "rows" that respects deep equality of arrays, but nevertheless refrains
|
||||
* from materializing things needlessly. Useful for unit tests that want to compare equality of different
|
||||
* InlineDataSource instances.
|
||||
*/
|
||||
private static boolean rowsEqual(final Iterable<Object[]> rowsA, final Iterable<Object[]> rowsB)
|
||||
{
|
||||
if (rowsA instanceof List && rowsB instanceof List) {
|
||||
final List<Object[]> listA = (List<Object[]>) rowsA;
|
||||
final List<Object[]> listB = (List<Object[]>) rowsB;
|
||||
|
||||
if (listA.size() != listB.size()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
for (int i = 0; i < listA.size(); i++) {
|
||||
final Object[] rowA = listA.get(i);
|
||||
final Object[] rowB = listB.get(i);
|
||||
|
||||
if (!Arrays.equals(rowA, rowB)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
} else {
|
||||
return Objects.equals(rowsA, rowsB);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A very zealous hash code computer for "rows" that is compatible with {@link #rowsEqual}.
|
||||
*/
|
||||
private static int rowsHashCode(final Iterable<Object[]> rows)
|
||||
{
|
||||
if (rows instanceof List) {
|
||||
final List<Object[]> list = (List<Object[]>) rows;
|
||||
|
||||
int code = 1;
|
||||
for (final Object[] row : list) {
|
||||
code = 31 * code + Arrays.hashCode(row);
|
||||
}
|
||||
|
||||
return code;
|
||||
} else {
|
||||
return Objects.hash(rows);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getTableNames()
|
||||
{
|
||||
|
@ -182,6 +234,27 @@ public class InlineDataSource implements DataSource
|
|||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(
|
||||
Query query,
|
||||
AtomicLong cpuTimeAcc
|
||||
)
|
||||
{
|
||||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
return newSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the row signature (map of column name to type) for this inline datasource. Note that types may
|
||||
* be null, meaning we know we have a column with a certain name, but we don't know what its type is.
|
||||
|
@ -232,53 +305,4 @@ public class InlineDataSource implements DataSource
|
|||
"signature=" + signature +
|
||||
'}';
|
||||
}
|
||||
|
||||
/**
|
||||
* A very zealous equality checker for "rows" that respects deep equality of arrays, but nevertheless refrains
|
||||
* from materializing things needlessly. Useful for unit tests that want to compare equality of different
|
||||
* InlineDataSource instances.
|
||||
*/
|
||||
private static boolean rowsEqual(final Iterable<Object[]> rowsA, final Iterable<Object[]> rowsB)
|
||||
{
|
||||
if (rowsA instanceof List && rowsB instanceof List) {
|
||||
final List<Object[]> listA = (List<Object[]>) rowsA;
|
||||
final List<Object[]> listB = (List<Object[]>) rowsB;
|
||||
|
||||
if (listA.size() != listB.size()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
for (int i = 0; i < listA.size(); i++) {
|
||||
final Object[] rowA = listA.get(i);
|
||||
final Object[] rowB = listB.get(i);
|
||||
|
||||
if (!Arrays.equals(rowA, rowB)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
} else {
|
||||
return Objects.equals(rowsA, rowsB);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A very zealous hash code computer for "rows" that is compatible with {@link #rowsEqual}.
|
||||
*/
|
||||
private static int rowsHashCode(final Iterable<Object[]> rows)
|
||||
{
|
||||
if (rows instanceof List) {
|
||||
final List<Object[]> list = (List<Object[]>) rows;
|
||||
|
||||
int code = 1;
|
||||
for (final Object[] row : list) {
|
||||
code = 31 * code + Arrays.hashCode(row);
|
||||
}
|
||||
|
||||
return code;
|
||||
} else {
|
||||
return Objects.hash(rows);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,36 +26,61 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.primitives.Ints;
|
||||
import org.apache.druid.common.guava.GuavaUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.query.planning.PreJoinableClause;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.join.HashJoinSegment;
|
||||
import org.apache.druid.segment.join.JoinConditionAnalysis;
|
||||
import org.apache.druid.segment.join.JoinPrefixUtils;
|
||||
import org.apache.druid.segment.join.JoinType;
|
||||
import org.apache.druid.segment.join.JoinableClause;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterAnalyzer;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey;
|
||||
import org.apache.druid.segment.join.filter.JoinableClauses;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.utils.JvmUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Represents a join of two datasources.
|
||||
*
|
||||
* <p>
|
||||
* Logically, this datasource contains the result of:
|
||||
*
|
||||
* <p>
|
||||
* (1) prefixing all right-side columns with "rightPrefix"
|
||||
* (2) then, joining the left and (prefixed) right sides using the provided type and condition
|
||||
*
|
||||
* <p>
|
||||
* Any columns from the left-hand side that start with "rightPrefix", and are at least one character longer than
|
||||
* the prefix, will be shadowed. It is up to the caller to ensure that no important columns are shadowed by the
|
||||
* chosen prefix.
|
||||
*
|
||||
* When analyzed by {@link org.apache.druid.query.planning.DataSourceAnalysis}, the right-hand side of this datasource
|
||||
* will become a {@link org.apache.druid.query.planning.PreJoinableClause} object.
|
||||
* <p>
|
||||
* When analyzed by {@link DataSourceAnalysis}, the right-hand side of this datasource
|
||||
* will become a {@link PreJoinableClause} object.
|
||||
*/
|
||||
public class JoinDataSource implements DataSource
|
||||
{
|
||||
|
@ -67,6 +92,12 @@ public class JoinDataSource implements DataSource
|
|||
// An optional filter on the left side if left is direct table access
|
||||
@Nullable
|
||||
private final DimFilter leftFilter;
|
||||
@Nullable
|
||||
private final JoinableFactoryWrapper joinableFactoryWrapper;
|
||||
private static final Logger log = new Logger(JoinDataSource.class);
|
||||
private final DataSourceAnalysis analysis;
|
||||
|
||||
|
||||
|
||||
private JoinDataSource(
|
||||
DataSource left,
|
||||
|
@ -74,7 +105,8 @@ public class JoinDataSource implements DataSource
|
|||
String rightPrefix,
|
||||
JoinConditionAnalysis conditionAnalysis,
|
||||
JoinType joinType,
|
||||
@Nullable DimFilter leftFilter
|
||||
@Nullable DimFilter leftFilter,
|
||||
@Nullable JoinableFactoryWrapper joinableFactoryWrapper
|
||||
)
|
||||
{
|
||||
this.left = Preconditions.checkNotNull(left, "left");
|
||||
|
@ -88,6 +120,8 @@ public class JoinDataSource implements DataSource
|
|||
"left filter is only supported if left data source is direct table access"
|
||||
);
|
||||
this.leftFilter = leftFilter;
|
||||
this.joinableFactoryWrapper = joinableFactoryWrapper;
|
||||
this.analysis = DataSourceAnalysis.forDataSource(this);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -101,7 +135,8 @@ public class JoinDataSource implements DataSource
|
|||
@JsonProperty("condition") String condition,
|
||||
@JsonProperty("joinType") JoinType joinType,
|
||||
@Nullable @JsonProperty("leftFilter") DimFilter leftFilter,
|
||||
@JacksonInject ExprMacroTable macroTable
|
||||
@JacksonInject ExprMacroTable macroTable,
|
||||
@Nullable @JacksonInject JoinableFactoryWrapper joinableFactoryWrapper
|
||||
)
|
||||
{
|
||||
return new JoinDataSource(
|
||||
|
@ -114,7 +149,8 @@ public class JoinDataSource implements DataSource
|
|||
macroTable
|
||||
),
|
||||
joinType,
|
||||
leftFilter
|
||||
leftFilter,
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -127,12 +163,22 @@ public class JoinDataSource implements DataSource
|
|||
final String rightPrefix,
|
||||
final JoinConditionAnalysis conditionAnalysis,
|
||||
final JoinType joinType,
|
||||
final DimFilter leftFilter
|
||||
final DimFilter leftFilter,
|
||||
@Nullable final JoinableFactoryWrapper joinableFactoryWrapper
|
||||
)
|
||||
{
|
||||
return new JoinDataSource(left, right, rightPrefix, conditionAnalysis, joinType, leftFilter);
|
||||
return new JoinDataSource(
|
||||
left,
|
||||
right,
|
||||
rightPrefix,
|
||||
conditionAnalysis,
|
||||
joinType,
|
||||
leftFilter,
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Set<String> getTableNames()
|
||||
{
|
||||
|
@ -185,6 +231,12 @@ public class JoinDataSource implements DataSource
|
|||
return leftFilter;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public JoinableFactoryWrapper getJoinableFactoryWrapper()
|
||||
{
|
||||
return joinableFactoryWrapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<DataSource> getChildren()
|
||||
{
|
||||
|
@ -204,7 +256,8 @@ public class JoinDataSource implements DataSource
|
|||
rightPrefix,
|
||||
conditionAnalysis,
|
||||
joinType,
|
||||
leftFilter
|
||||
leftFilter,
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -230,6 +283,7 @@ public class JoinDataSource implements DataSource
|
|||
* Computes a set of column names for left table expressions in join condition which may already have been defined as
|
||||
* a virtual column in the virtual column registry. It helps to remove any extraenous virtual columns created and only
|
||||
* use the relevant ones.
|
||||
*
|
||||
* @return a set of column names which might be virtual columns on left table in join condition
|
||||
*/
|
||||
public Set<String> getVirtualColumnCandidates()
|
||||
|
@ -278,4 +332,153 @@ public class JoinDataSource implements DataSource
|
|||
", leftFilter=" + leftFilter +
|
||||
'}';
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join
|
||||
* clauses is > 0). If mapping is not needed, this method will return {@link Function#identity()}.
|
||||
*
|
||||
* @param baseFilter Filter to apply before the join takes place
|
||||
* @param clauses Pre-joinable clauses
|
||||
* @param cpuTimeAccumulator An accumulator that we will add CPU nanos to; this is part of the function to encourage
|
||||
* callers to remember to track metrics on CPU time required for creation of Joinables
|
||||
* @param query The query that will be run on the mapped segments. Usually this should be
|
||||
* {@code analysis.getBaseQuery().orElse(query)}, where "analysis" is a
|
||||
* {@link DataSourceAnalysis} and "query" is the original
|
||||
* query from the end user.
|
||||
*/
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFn(
|
||||
@Nullable final Filter baseFilter,
|
||||
final List<PreJoinableClause> clauses,
|
||||
final AtomicLong cpuTimeAccumulator,
|
||||
final Query<?> query
|
||||
)
|
||||
{
|
||||
// compute column correlations here and RHS correlated values
|
||||
return JvmUtils.safeAccumulateThreadCpuTime(
|
||||
cpuTimeAccumulator,
|
||||
() -> {
|
||||
if (clauses.isEmpty()) {
|
||||
return Function.identity();
|
||||
} else {
|
||||
final JoinableClauses joinableClauses = JoinableClauses.createClauses(
|
||||
clauses,
|
||||
joinableFactoryWrapper.getJoinableFactory()
|
||||
);
|
||||
final JoinFilterRewriteConfig filterRewriteConfig = JoinFilterRewriteConfig.forQuery(query);
|
||||
|
||||
// Pick off any join clauses that can be converted into filters.
|
||||
final Set<String> requiredColumns = query.getRequiredColumns();
|
||||
final Filter baseFilterToUse;
|
||||
final List<JoinableClause> clausesToUse;
|
||||
|
||||
if (requiredColumns != null && filterRewriteConfig.isEnableRewriteJoinToFilter()) {
|
||||
final Pair<List<Filter>, List<JoinableClause>> conversionResult = JoinableFactoryWrapper.convertJoinsToFilters(
|
||||
joinableClauses.getJoinableClauses(),
|
||||
requiredColumns,
|
||||
Ints.checkedCast(Math.min(filterRewriteConfig.getFilterRewriteMaxSize(), Integer.MAX_VALUE))
|
||||
);
|
||||
|
||||
baseFilterToUse =
|
||||
Filters.maybeAnd(
|
||||
Lists.newArrayList(
|
||||
Iterables.concat(
|
||||
Collections.singleton(baseFilter),
|
||||
conversionResult.lhs
|
||||
)
|
||||
)
|
||||
).orElse(null);
|
||||
clausesToUse = conversionResult.rhs;
|
||||
} else {
|
||||
baseFilterToUse = baseFilter;
|
||||
clausesToUse = joinableClauses.getJoinableClauses();
|
||||
}
|
||||
|
||||
// Analyze remaining join clauses to see if filters on them can be pushed down.
|
||||
final JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
filterRewriteConfig,
|
||||
clausesToUse,
|
||||
query.getVirtualColumns(),
|
||||
Filters.maybeAnd(Arrays.asList(baseFilterToUse, Filters.toFilter(query.getFilter())))
|
||||
.orElse(null)
|
||||
)
|
||||
);
|
||||
|
||||
return baseSegment ->
|
||||
new HashJoinSegment(
|
||||
baseSegment,
|
||||
baseFilterToUse,
|
||||
GuavaUtils.firstNonNull(clausesToUse, ImmutableList.of()),
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(
|
||||
Query query,
|
||||
AtomicLong cpuTimeAccumulator
|
||||
)
|
||||
{
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = createSegmentMapFn(
|
||||
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null),
|
||||
analysis.getPreJoinableClauses(),
|
||||
cpuTimeAccumulator,
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
);
|
||||
return segmentMapFn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
DataSource current = newSource;
|
||||
DimFilter joinBaseFilter = analysis.getJoinBaseTableFilter().orElse(null);
|
||||
|
||||
for (final PreJoinableClause clause : analysis.getPreJoinableClauses()) {
|
||||
current = JoinDataSource.create(
|
||||
current,
|
||||
clause.getDataSource(),
|
||||
clause.getPrefix(),
|
||||
clause.getCondition(),
|
||||
clause.getJoinType(),
|
||||
joinBaseFilter,
|
||||
this.joinableFactoryWrapper
|
||||
);
|
||||
joinBaseFilter = null;
|
||||
}
|
||||
return current;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
final List<PreJoinableClause> clauses = analysis.getPreJoinableClauses();
|
||||
if (clauses.isEmpty()) {
|
||||
throw new IAE("No join clauses to build the cache key for data source [%s]", analysis.getDataSource());
|
||||
}
|
||||
|
||||
final CacheKeyBuilder keyBuilder;
|
||||
keyBuilder = new CacheKeyBuilder(JoinableFactoryWrapper.JOIN_OPERATION);
|
||||
if (analysis.getJoinBaseTableFilter().isPresent()) {
|
||||
keyBuilder.appendCacheable(analysis.getJoinBaseTableFilter().get());
|
||||
}
|
||||
for (PreJoinableClause clause : clauses) {
|
||||
Optional<byte[]> bytes = joinableFactoryWrapper.getJoinableFactory().computeJoinCacheKey(clause.getDataSource(), clause.getCondition());
|
||||
if (!bytes.isPresent()) {
|
||||
// Encountered a data source which didn't support cache yet
|
||||
log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource());
|
||||
return new byte[]{};
|
||||
}
|
||||
keyBuilder.appendByteArray(bytes.get());
|
||||
keyBuilder.appendString(clause.getCondition().getOriginalExpression());
|
||||
keyBuilder.appendString(clause.getPrefix());
|
||||
keyBuilder.appendString(clause.getJoinType().name());
|
||||
}
|
||||
return keyBuilder.build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,18 +23,21 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Represents a lookup.
|
||||
*
|
||||
* <p>
|
||||
* Currently, this datasource is not actually queryable, and attempts to do so will lead to errors. It is here as a
|
||||
* placeholder for a future time in which it will become queryable.
|
||||
*
|
||||
* <p>
|
||||
* The "lookupName" referred to here should be provided by a
|
||||
* {@link org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider}.
|
||||
*/
|
||||
|
@ -96,6 +99,27 @@ public class LookupDataSource implements DataSource
|
|||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(
|
||||
Query query,
|
||||
AtomicLong cpuTime
|
||||
)
|
||||
{
|
||||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
return newSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.druid.query.aggregation.PostAggregator;
|
|||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.query.planning.PreJoinableClause;
|
||||
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
|
@ -191,40 +190,7 @@ public class Queries
|
|||
*/
|
||||
public static <T> Query<T> withBaseDataSource(final Query<T> query, final DataSource newBaseDataSource)
|
||||
{
|
||||
final Query<T> retVal;
|
||||
|
||||
if (query.getDataSource() instanceof QueryDataSource) {
|
||||
final Query<?> subQuery = ((QueryDataSource) query.getDataSource()).getQuery();
|
||||
retVal = query.withDataSource(new QueryDataSource(withBaseDataSource(subQuery, newBaseDataSource)));
|
||||
} else {
|
||||
final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource());
|
||||
|
||||
DataSource current = newBaseDataSource;
|
||||
DimFilter joinBaseFilter = analysis.getJoinBaseTableFilter().orElse(null);
|
||||
|
||||
for (final PreJoinableClause clause : analysis.getPreJoinableClauses()) {
|
||||
current = JoinDataSource.create(
|
||||
current,
|
||||
clause.getDataSource(),
|
||||
clause.getPrefix(),
|
||||
clause.getCondition(),
|
||||
clause.getJoinType(),
|
||||
joinBaseFilter
|
||||
);
|
||||
joinBaseFilter = null;
|
||||
}
|
||||
|
||||
retVal = query.withDataSource(current);
|
||||
}
|
||||
|
||||
// Verify postconditions, just in case.
|
||||
final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(retVal.getDataSource());
|
||||
|
||||
if (!newBaseDataSource.equals(analysis.getBaseDataSource())) {
|
||||
throw new ISE("Unable to replace base dataSource");
|
||||
}
|
||||
|
||||
return retVal;
|
||||
return query.withDataSource(query.getDataSource().withUpdatedDataSource(newBaseDataSource));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -25,10 +25,13 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
|
||||
@JsonTypeName("query")
|
||||
public class QueryDataSource implements DataSource
|
||||
|
@ -88,6 +91,28 @@ public class QueryDataSource implements DataSource
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(
|
||||
Query query,
|
||||
AtomicLong cpuTime
|
||||
)
|
||||
{
|
||||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
return new QueryDataSource(query.withDataSource(query.getDataSource().withUpdatedDataSource(newSource)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -24,11 +24,14 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
|
||||
@JsonTypeName("table")
|
||||
public class TableDataSource implements DataSource
|
||||
|
@ -93,6 +96,27 @@ public class TableDataSource implements DataSource
|
|||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(
|
||||
Query query,
|
||||
AtomicLong cpuTime
|
||||
)
|
||||
{
|
||||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
return newSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new byte[0];
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -26,9 +26,12 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Iterables;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class UnionDataSource implements DataSource
|
||||
|
@ -105,6 +108,27 @@ public class UnionDataSource implements DataSource
|
|||
return dataSources.stream().allMatch(DataSource::isConcrete);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(
|
||||
Query query,
|
||||
AtomicLong cpuTime
|
||||
)
|
||||
{
|
||||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
return newSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -39,7 +39,7 @@ import java.util.Optional;
|
|||
|
||||
/**
|
||||
* Represents a deep, left-heavy join of a left-hand side baseSegment onto a series of right-hand side clauses.
|
||||
*
|
||||
* <p>
|
||||
* In other words, logically the operation is: join(join(join(baseSegment, clauses[0]), clauses[1]), clauses[2]) etc.
|
||||
*/
|
||||
public class HashJoinSegment implements SegmentReference
|
||||
|
@ -148,15 +148,4 @@ public class HashJoinSegment implements SegmentReference
|
|||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public Filter getBaseFilter()
|
||||
{
|
||||
return baseFilter;
|
||||
}
|
||||
|
||||
public List<JoinableClause> getClauses()
|
||||
{
|
||||
return clauses;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,7 +34,7 @@ import java.util.stream.Collectors;
|
|||
* clause is "t1 JOIN t2 ON t1.x = t2.x" then this class represents "JOIN t2 ON x = t2.x" -- it does not include
|
||||
* references to the left-hand "t1".
|
||||
* <p>
|
||||
* Created from {@link org.apache.druid.query.planning.PreJoinableClause} by {@link JoinableFactoryWrapper#createSegmentMapFn}.
|
||||
* Created from {@link org.apache.druid.query.planning.PreJoinableClause}.
|
||||
*/
|
||||
public class JoinableClause implements ReferenceCountedObject
|
||||
{
|
||||
|
|
|
@ -22,42 +22,19 @@ package org.apache.druid.segment.join;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.HashMultiset;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Multiset;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.common.guava.GuavaUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.query.planning.PreJoinableClause;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.filter.FalseFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterAnalyzer;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis;
|
||||
import org.apache.druid.segment.join.filter.JoinFilterPreAnalysisKey;
|
||||
import org.apache.druid.segment.join.filter.JoinableClauses;
|
||||
import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig;
|
||||
import org.apache.druid.utils.JvmUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
|
@ -66,8 +43,7 @@ import java.util.stream.Collectors;
|
|||
public class JoinableFactoryWrapper
|
||||
{
|
||||
|
||||
private static final byte JOIN_OPERATION = 0x1;
|
||||
private static final Logger log = new Logger(JoinableFactoryWrapper.class);
|
||||
public static final byte JOIN_OPERATION = 0x1;
|
||||
|
||||
private final JoinableFactory joinableFactory;
|
||||
|
||||
|
@ -77,144 +53,12 @@ public class JoinableFactoryWrapper
|
|||
this.joinableFactory = Preconditions.checkNotNull(joinableFactory, "joinableFactory");
|
||||
}
|
||||
|
||||
public JoinableFactory getJoinableFactory()
|
||||
{
|
||||
return joinableFactory;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join
|
||||
* clauses is > 0). If mapping is not needed, this method will return {@link Function#identity()}.
|
||||
*
|
||||
* @param baseFilter Filter to apply before the join takes place
|
||||
* @param clauses Pre-joinable clauses
|
||||
* @param cpuTimeAccumulator An accumulator that we will add CPU nanos to; this is part of the function to encourage
|
||||
* callers to remember to track metrics on CPU time required for creation of Joinables
|
||||
* @param query The query that will be run on the mapped segments. Usually this should be
|
||||
* {@code analysis.getBaseQuery().orElse(query)}, where "analysis" is a
|
||||
* {@link DataSourceAnalysis} and "query" is the original
|
||||
* query from the end user.
|
||||
*/
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFn(
|
||||
@Nullable final Filter baseFilter,
|
||||
final List<PreJoinableClause> clauses,
|
||||
final AtomicLong cpuTimeAccumulator,
|
||||
final Query<?> query
|
||||
)
|
||||
{
|
||||
// compute column correlations here and RHS correlated values
|
||||
return JvmUtils.safeAccumulateThreadCpuTime(
|
||||
cpuTimeAccumulator,
|
||||
() -> {
|
||||
if (clauses.isEmpty()) {
|
||||
return Function.identity();
|
||||
} else {
|
||||
final JoinableClauses joinableClauses = JoinableClauses.createClauses(clauses, joinableFactory);
|
||||
final JoinFilterRewriteConfig filterRewriteConfig = JoinFilterRewriteConfig.forQuery(query);
|
||||
|
||||
// Pick off any join clauses that can be converted into filters.
|
||||
final Set<String> requiredColumns = query.getRequiredColumns();
|
||||
final Filter baseFilterToUse;
|
||||
final List<JoinableClause> clausesToUse;
|
||||
|
||||
if (requiredColumns != null && filterRewriteConfig.isEnableRewriteJoinToFilter()) {
|
||||
final Pair<List<Filter>, List<JoinableClause>> conversionResult = convertJoinsToFilters(
|
||||
joinableClauses.getJoinableClauses(),
|
||||
requiredColumns,
|
||||
Ints.checkedCast(Math.min(filterRewriteConfig.getFilterRewriteMaxSize(), Integer.MAX_VALUE))
|
||||
);
|
||||
|
||||
baseFilterToUse =
|
||||
Filters.maybeAnd(
|
||||
Lists.newArrayList(
|
||||
Iterables.concat(
|
||||
Collections.singleton(baseFilter),
|
||||
conversionResult.lhs
|
||||
)
|
||||
)
|
||||
).orElse(null);
|
||||
clausesToUse = conversionResult.rhs;
|
||||
} else {
|
||||
baseFilterToUse = baseFilter;
|
||||
clausesToUse = joinableClauses.getJoinableClauses();
|
||||
}
|
||||
|
||||
// Analyze remaining join clauses to see if filters on them can be pushed down.
|
||||
final JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis(
|
||||
new JoinFilterPreAnalysisKey(
|
||||
filterRewriteConfig,
|
||||
clausesToUse,
|
||||
query.getVirtualColumns(),
|
||||
Filters.maybeAnd(Arrays.asList(baseFilterToUse, Filters.toFilter(query.getFilter())))
|
||||
.orElse(null)
|
||||
)
|
||||
);
|
||||
|
||||
return baseSegment ->
|
||||
new HashJoinSegment(
|
||||
baseSegment,
|
||||
baseFilterToUse,
|
||||
GuavaUtils.firstNonNull(clausesToUse, ImmutableList.of()),
|
||||
joinFilterPreAnalysis
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compute a cache key prefix for a join data source. This includes the data sources that participate in the RHS of a
|
||||
* join as well as any query specific constructs associated with join data source such as base table filter. This key prefix
|
||||
* can be used in segment level cache or result level cache. The function can return following wrapped in an
|
||||
* Optional
|
||||
* - Non-empty byte array - If there is join datasource involved and caching is possible. The result includes
|
||||
* join condition expression, join type and cache key returned by joinable factory for each {@link PreJoinableClause}
|
||||
* - NULL - There is a join but caching is not possible. It may happen if one of the participating datasource
|
||||
* in the JOIN is not cacheable.
|
||||
*
|
||||
* @param dataSourceAnalysis for the join datasource
|
||||
*
|
||||
* @return the optional cache key to be used as part of query cache key
|
||||
*
|
||||
* @throws {@link IAE} if this operation is called on a non-join data source
|
||||
*/
|
||||
public Optional<byte[]> computeJoinDataSourceCacheKey(
|
||||
final DataSourceAnalysis dataSourceAnalysis
|
||||
)
|
||||
{
|
||||
final List<PreJoinableClause> clauses = dataSourceAnalysis.getPreJoinableClauses();
|
||||
if (clauses.isEmpty()) {
|
||||
throw new IAE("No join clauses to build the cache key for data source [%s]", dataSourceAnalysis.getDataSource());
|
||||
}
|
||||
|
||||
final CacheKeyBuilder keyBuilder;
|
||||
keyBuilder = new CacheKeyBuilder(JOIN_OPERATION);
|
||||
if (dataSourceAnalysis.getJoinBaseTableFilter().isPresent()) {
|
||||
keyBuilder.appendCacheable(dataSourceAnalysis.getJoinBaseTableFilter().get());
|
||||
}
|
||||
for (PreJoinableClause clause : clauses) {
|
||||
Optional<byte[]> bytes = joinableFactory.computeJoinCacheKey(clause.getDataSource(), clause.getCondition());
|
||||
if (!bytes.isPresent()) {
|
||||
// Encountered a data source which didn't support cache yet
|
||||
log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource());
|
||||
return Optional.empty();
|
||||
}
|
||||
keyBuilder.appendByteArray(bytes.get());
|
||||
keyBuilder.appendString(clause.getCondition().getOriginalExpression());
|
||||
keyBuilder.appendString(clause.getPrefix());
|
||||
keyBuilder.appendString(clause.getJoinType().name());
|
||||
}
|
||||
return Optional.of(keyBuilder.build());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Converts any join clauses to filters that can be converted, and returns the rest as-is.
|
||||
*
|
||||
* <p>
|
||||
* See {@link #convertJoinToFilter} for details on the logic.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static Pair<List<Filter>, List<JoinableClause>> convertJoinsToFilters(
|
||||
public static Pair<List<Filter>, List<JoinableClause>> convertJoinsToFilters(
|
||||
final List<JoinableClause> clauses,
|
||||
final Set<String> requiredColumns,
|
||||
final int maxNumFilterValues
|
||||
|
@ -271,16 +115,16 @@ public class JoinableFactoryWrapper
|
|||
|
||||
/**
|
||||
* Converts a join clause into an "in" filter if possible.
|
||||
*
|
||||
* <p>
|
||||
* The requirements are:
|
||||
*
|
||||
* <p>
|
||||
* - it must be an INNER equi-join
|
||||
* - the right-hand columns referenced by the condition must not have any duplicate values. If there are duplicates
|
||||
* values in the column, then the join is tried to be converted to a filter while maintaining the join clause on top
|
||||
* as well for correct results.
|
||||
* values in the column, then the join is tried to be converted to a filter while maintaining the join clause on top
|
||||
* as well for correct results.
|
||||
* - no columns from the right-hand side can appear in "requiredColumns". If the columns from right side are required
|
||||
* (ie they are directly or indirectly projected in the join output), then the join is tried to be converted to a
|
||||
* filter while maintaining the join clause on top as well for correct results.
|
||||
* (ie they are directly or indirectly projected in the join output), then the join is tried to be converted to a
|
||||
* filter while maintaining the join clause on top as well for correct results.
|
||||
*
|
||||
* @return {@link JoinClauseToFilterConversion} object which contains the converted filter for the clause and a boolean
|
||||
* to represent whether the converted filter encapsulates the whole clause or not. More semantics of the object are
|
||||
|
@ -340,12 +184,17 @@ public class JoinableFactoryWrapper
|
|||
return new JoinClauseToFilterConversion(null, false);
|
||||
}
|
||||
|
||||
public JoinableFactory getJoinableFactory()
|
||||
{
|
||||
return joinableFactory;
|
||||
}
|
||||
|
||||
/**
|
||||
* Encapsulates the conversion which happened for a joinable clause.
|
||||
* convertedFilter represents the filter which got generated from the conversion.
|
||||
* joinClauseFullyConverted represents whether convertedFilter fully encapsulated the joinable clause or not.
|
||||
* Encapsulation of the clause means that the filter can replace the whole joinable clause.
|
||||
*
|
||||
* <p>
|
||||
* If convertedFilter is null and joinClauseFullyConverted is true, it means that all parts of the joinable clause can
|
||||
* be broken into filters. Further, all the clause conditions are on columns where the right side is only null values.
|
||||
* In that case, we replace joinable with a FalseFilter.
|
||||
|
|
|
@ -22,28 +22,39 @@ package org.apache.druid.query;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterators;
|
||||
import nl.jqno.equalsverifier.EqualsVerifier;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.query.filter.TrueDimFilter;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.join.JoinConditionAnalysis;
|
||||
import org.apache.druid.segment.join.JoinType;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.segment.join.NoopJoinableFactory;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Optional;
|
||||
|
||||
|
||||
public class JoinDataSourceTest
|
||||
{
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
public static final JoinableFactoryWrapper NOOP_JOINABLE_FACTORY_WRAPPER = new JoinableFactoryWrapper(
|
||||
NoopJoinableFactory.INSTANCE
|
||||
);
|
||||
private final TableDataSource fooTable = new TableDataSource("foo");
|
||||
private final TableDataSource barTable = new TableDataSource("bar");
|
||||
private final LookupDataSource lookylooLookup = new LookupDataSource("lookyloo");
|
||||
|
||||
private final JoinDataSource joinTableToLookup = JoinDataSource.create(
|
||||
fooTable,
|
||||
lookylooLookup,
|
||||
|
@ -51,9 +62,9 @@ public class JoinDataSourceTest
|
|||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
);
|
||||
|
||||
private final JoinDataSource joinTableToTable = JoinDataSource.create(
|
||||
fooTable,
|
||||
barTable,
|
||||
|
@ -61,8 +72,13 @@ public class JoinDataSourceTest
|
|||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
);
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
@Mock
|
||||
private JoinableFactoryWrapper joinableFactoryWrapper;
|
||||
|
||||
@Test
|
||||
public void test_getTableNames_tableToTable()
|
||||
|
@ -149,13 +165,14 @@ public class JoinDataSourceTest
|
|||
EqualsVerifier.forClass(JoinDataSource.class)
|
||||
.usingGetClass()
|
||||
.withNonnullFields("left", "right", "rightPrefix", "conditionAnalysis", "joinType")
|
||||
.withIgnoredFields("joinableFactoryWrapper", "analysis")
|
||||
.verify();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_serde() throws Exception
|
||||
{
|
||||
final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
|
||||
final ObjectMapper jsonMapper = TestHelper.makeJsonMapperForJoinable(joinableFactoryWrapper);
|
||||
JoinDataSource joinDataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
|
@ -163,7 +180,8 @@ public class JoinDataSourceTest
|
|||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
TrueDimFilter.instance(),
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
final JoinDataSource deserialized = (JoinDataSource) jsonMapper.readValue(
|
||||
|
@ -186,7 +204,8 @@ public class JoinDataSourceTest
|
|||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
TrueDimFilter.instance(),
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -200,7 +219,8 @@ public class JoinDataSourceTest
|
|||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
TrueDimFilter.instance(),
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(TrueDimFilter.instance(), dataSource.getLeftFilter());
|
||||
}
|
||||
|
@ -215,8 +235,275 @@ public class JoinDataSourceTest
|
|||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
);
|
||||
Assert.assertEquals(dataSource.getVirtualColumnCandidates(), ImmutableSet.of("x"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_noHashJoin()
|
||||
{
|
||||
JoinDataSource dataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
NOOP_JOINABLE_FACTORY_WRAPPER
|
||||
);
|
||||
|
||||
Optional<byte[]> cacheKey = Optional.ofNullable(dataSource.getCacheKey());
|
||||
|
||||
Assert.assertTrue(cacheKey.isPresent());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_sameKeyForSameJoin()
|
||||
{
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
JoinDataSource joinDataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
JoinDataSource joinDataSource1 = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
byte[] cacheKey1 = joinDataSource.getCacheKey();
|
||||
byte[] cacheKey2 = joinDataSource1.getCacheKey();
|
||||
|
||||
Assert.assertNotEquals(cacheKey1.length, 0);
|
||||
Assert.assertNotEquals(cacheKey2.length, 0);
|
||||
Assert.assertTrue(Arrays.equals(cacheKey1, cacheKey2));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithTables()
|
||||
{
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
JoinDataSource joinDataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
JoinDataSource joinDataSource1 = JoinDataSource.create(
|
||||
new TableDataSource("table11"),
|
||||
new TableDataSource("table12"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
byte[] cacheKey1 = joinDataSource.getCacheKey();
|
||||
byte[] cacheKey2 = joinDataSource1.getCacheKey();
|
||||
|
||||
Assert.assertNotEquals(cacheKey1.length, 0);
|
||||
Assert.assertNotEquals(cacheKey2.length, 0);
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithExpressions()
|
||||
{
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
JoinDataSource joinDataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
JoinDataSource joinDataSource1 = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"y == \"j.y\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
byte[] cacheKey1 = joinDataSource.getCacheKey();
|
||||
byte[] cacheKey2 = joinDataSource1.getCacheKey();
|
||||
|
||||
Assert.assertNotEquals(cacheKey1.length, 0);
|
||||
Assert.assertNotEquals(cacheKey2.length, 0);
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinType()
|
||||
{
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
JoinDataSource joinDataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
JoinDataSource joinDataSource1 = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
byte[] cacheKey1 = joinDataSource.getCacheKey();
|
||||
byte[] cacheKey2 = joinDataSource1.getCacheKey();
|
||||
|
||||
Assert.assertNotEquals(cacheKey1.length, 0);
|
||||
Assert.assertNotEquals(cacheKey2.length, 0);
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithPrefix()
|
||||
{
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
JoinDataSource joinDataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
JoinDataSource joinDataSource1 = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"ab.",
|
||||
"x == \"ab.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
byte[] cacheKey1 = joinDataSource.getCacheKey();
|
||||
byte[] cacheKey2 = joinDataSource1.getCacheKey();
|
||||
|
||||
Assert.assertNotEquals(cacheKey1.length, 0);
|
||||
Assert.assertNotEquals(cacheKey2.length, 0);
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithBaseFilter()
|
||||
{
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
NullHandling.initializeForTests();
|
||||
final InDimFilter expectedInDimFilter = new InDimFilter("dimTest", Arrays.asList("good", "bad"), null);
|
||||
|
||||
JoinDataSource joinDataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
expectedInDimFilter,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
JoinDataSource joinDataSource1 = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
byte[] cacheKey1 = joinDataSource.getCacheKey();
|
||||
byte[] cacheKey2 = joinDataSource1.getCacheKey();
|
||||
|
||||
Assert.assertNotEquals(cacheKey1.length, 0);
|
||||
Assert.assertNotEquals(cacheKey2.length, 0);
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_cachingUnsupported()
|
||||
{
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
JoinDataSource joinDataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new LookupDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
byte[] cacheKey1 = joinDataSource.getCacheKey();
|
||||
Assert.assertEquals(cacheKey1.length, 0);
|
||||
}
|
||||
|
||||
|
||||
private static class JoinableFactoryWithCacheKey extends NoopJoinableFactory
|
||||
{
|
||||
@Override
|
||||
public Optional<byte[]> computeJoinCacheKey(DataSource dataSource, JoinConditionAnalysis condition)
|
||||
{
|
||||
if (dataSource.isCacheable(false) && condition.canHashJoin()) {
|
||||
String tableName = Iterators.getOnlyElement(dataSource.getTableNames().iterator());
|
||||
return Optional.of(StringUtils.toUtf8(tableName));
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -425,7 +425,8 @@ public class QueriesTest
|
|||
"\"foo.x\" == \"bar.x\"",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
)
|
||||
)
|
||||
.intervals("2000/3000")
|
||||
|
@ -462,7 +463,8 @@ public class QueriesTest
|
|||
"\"foo.x\" == \"bar.x\"",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
)
|
||||
)
|
||||
.intervals("2000/3000")
|
||||
|
@ -500,14 +502,16 @@ public class QueriesTest
|
|||
"\"foo.x\" == \"bar.x\"",
|
||||
JoinType.INNER,
|
||||
TrueDimFilter.instance(),
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
),
|
||||
new TableDataSource("foo_outer"),
|
||||
"j0.",
|
||||
"\"foo_outer.x\" == \"bar.x\"",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
)
|
||||
|
||||
)
|
||||
|
@ -533,14 +537,16 @@ public class QueriesTest
|
|||
"\"foo.x\" == \"bar.x\"",
|
||||
JoinType.INNER,
|
||||
TrueDimFilter.instance(),
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
),
|
||||
new TableDataSource("foo_outer"),
|
||||
"j0.",
|
||||
"\"foo_outer.x\" == \"bar.x\"",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
)
|
||||
|
||||
)
|
||||
|
|
|
@ -612,7 +612,8 @@ public class DataSourceAnalysisTest
|
|||
joinClause(rightPrefix).getOriginalExpression(),
|
||||
joinType,
|
||||
dimFilter,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.druid.query.topn.TopNResultValue;
|
|||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.data.ComparableList;
|
||||
import org.apache.druid.segment.data.ComparableStringArray;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.timeline.DataSegment.PruneSpecsHolder;
|
||||
import org.junit.Assert;
|
||||
|
@ -95,9 +96,10 @@ public class TestHelper
|
|||
public static ObjectMapper makeJsonMapper()
|
||||
{
|
||||
final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
AnnotationIntrospector introspector = makeAnnotationIntrospector();
|
||||
final AnnotationIntrospector introspector = makeAnnotationIntrospector();
|
||||
DruidSecondaryModule.setupAnnotationIntrospector(mapper, introspector);
|
||||
|
||||
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std()
|
||||
.addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
|
||||
|
@ -107,6 +109,23 @@ public class TestHelper
|
|||
return mapper;
|
||||
}
|
||||
|
||||
public static ObjectMapper makeJsonMapperForJoinable(JoinableFactoryWrapper joinableFactoryWrapper)
|
||||
{
|
||||
final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
AnnotationIntrospector introspector = makeAnnotationIntrospector();
|
||||
DruidSecondaryModule.setupAnnotationIntrospector(mapper, introspector);
|
||||
|
||||
|
||||
mapper.setInjectableValues(
|
||||
new InjectableValues.Std()
|
||||
.addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE)
|
||||
.addValue(ObjectMapper.class.getName(), mapper)
|
||||
.addValue(PruneSpecsHolder.class, PruneSpecsHolder.DEFAULT)
|
||||
.addValue(JoinableFactoryWrapper.class, joinableFactoryWrapper)
|
||||
);
|
||||
return mapper;
|
||||
}
|
||||
|
||||
public static ObjectMapper makeSmileMapper()
|
||||
{
|
||||
final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
|
|
@ -22,34 +22,17 @@ package org.apache.druid.segment.join;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.common.config.NullHandlingTest;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.GlobalTableDataSource;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.LookupDataSource;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.query.TestQuery;
|
||||
import org.apache.druid.query.extraction.MapLookupExtractor;
|
||||
import org.apache.druid.query.filter.FalseDimFilter;
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.query.filter.TrueDimFilter;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.query.planning.PreJoinableClause;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.segment.QueryableIndexSegment;
|
||||
import org.apache.druid.segment.ReferenceCountingSegment;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.filter.FalseFilter;
|
||||
|
@ -57,24 +40,17 @@ import org.apache.druid.segment.join.lookup.LookupJoinable;
|
|||
import org.apache.druid.segment.join.table.IndexedTable;
|
||||
import org.apache.druid.segment.join.table.IndexedTableJoinable;
|
||||
import org.apache.druid.segment.join.table.RowBasedIndexedTable;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class JoinableFactoryWrapperTest extends NullHandlingTest
|
||||
|
@ -85,12 +61,12 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest
|
|||
|
||||
private static final Map<String, String> TEST_LOOKUP =
|
||||
ImmutableMap.<String, String>builder()
|
||||
.put("MX", "Mexico")
|
||||
.put("NO", "Norway")
|
||||
.put("SV", "El Salvador")
|
||||
.put("US", "United States")
|
||||
.put("", "Empty key")
|
||||
.build();
|
||||
.put("MX", "Mexico")
|
||||
.put("NO", "Norway")
|
||||
.put("SV", "El Salvador")
|
||||
.put("US", "United States")
|
||||
.put("", "Empty key")
|
||||
.build();
|
||||
|
||||
private static final Set<String> TEST_LOOKUP_KEYS =
|
||||
NullHandling.sqlCompatible()
|
||||
|
@ -137,340 +113,6 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest
|
|||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
@Test
|
||||
public void test_createSegmentMapFn_noClauses()
|
||||
{
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = NOOP_JOINABLE_FACTORY_WRAPPER.createSegmentMapFn(
|
||||
null,
|
||||
ImmutableList.of(),
|
||||
new AtomicLong(),
|
||||
null
|
||||
);
|
||||
|
||||
Assert.assertSame(Function.identity(), segmentMapFn);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_createSegmentMapFn_unusableClause()
|
||||
{
|
||||
final PreJoinableClause clause = makePreJoinableClause(
|
||||
INDEXED_TABLE_DS,
|
||||
"country == \"j.country\"",
|
||||
"j.",
|
||||
JoinType.LEFT
|
||||
);
|
||||
|
||||
expectedException.expect(IllegalStateException.class);
|
||||
expectedException.expectMessage("dataSource is not joinable");
|
||||
|
||||
final Function<SegmentReference, SegmentReference> ignored = NOOP_JOINABLE_FACTORY_WRAPPER.createSegmentMapFn(
|
||||
null,
|
||||
ImmutableList.of(clause),
|
||||
new AtomicLong(),
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_createSegmentMapFn_usableClause()
|
||||
{
|
||||
final PreJoinableClause clause = makePreJoinableClause(
|
||||
INDEXED_TABLE_DS,
|
||||
"country == \"j.country\"",
|
||||
"j.",
|
||||
JoinType.LEFT
|
||||
);
|
||||
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new InlineJoinableFactory());
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn(
|
||||
null,
|
||||
ImmutableList.of(clause),
|
||||
new AtomicLong(),
|
||||
new TestQuery(
|
||||
new TableDataSource("test"),
|
||||
new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))),
|
||||
false,
|
||||
new HashMap<>()
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertNotSame(Function.identity(), segmentMapFn);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_createSegmentMapFn_usableClause_joinToFilterEnabled() throws IOException
|
||||
{
|
||||
final PreJoinableClause clause = makePreJoinableClause(
|
||||
INDEXED_TABLE_DS,
|
||||
"country == \"j.country\"",
|
||||
"j.",
|
||||
JoinType.INNER
|
||||
);
|
||||
// required columns are necessary for the rewrite
|
||||
final TestQuery queryWithRequiredColumnsAndJoinFilterRewrite = (TestQuery) new TestQuery(
|
||||
new TableDataSource("test"),
|
||||
new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))),
|
||||
false,
|
||||
new HashMap<>()
|
||||
).withOverriddenContext(ImmutableMap.of(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, "true"));
|
||||
queryWithRequiredColumnsAndJoinFilterRewrite.setRequiredColumns(ImmutableSet.of("country"));
|
||||
|
||||
final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new InlineJoinableFactory());
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn(
|
||||
null,
|
||||
ImmutableList.of(clause),
|
||||
new AtomicLong(),
|
||||
queryWithRequiredColumnsAndJoinFilterRewrite
|
||||
);
|
||||
|
||||
// dummy segment
|
||||
final SegmentReference baseSegmentReference = ReferenceCountingSegment.wrapRootGenerationSegment(
|
||||
new QueryableIndexSegment(
|
||||
JoinTestHelper.createFactIndexBuilder(temporaryFolder.newFolder()).buildMMappedIndex(),
|
||||
SegmentId.dummy("facts")
|
||||
)
|
||||
);
|
||||
|
||||
// check the output contains the conversion filter
|
||||
Assert.assertNotSame(Function.identity(), segmentMapFn);
|
||||
final SegmentReference joinSegmentReference = segmentMapFn.apply(baseSegmentReference);
|
||||
Assert.assertTrue(joinSegmentReference instanceof HashJoinSegment);
|
||||
HashJoinSegment hashJoinSegment = (HashJoinSegment) joinSegmentReference;
|
||||
Assert.assertEquals(
|
||||
hashJoinSegment.getBaseFilter(),
|
||||
new InDimFilter(
|
||||
"country",
|
||||
INDEXED_TABLE_DS.getRowsAsList().stream().map(row -> row[0].toString()).collect(Collectors.toSet())
|
||||
)
|
||||
);
|
||||
// the returned clause list is not comparable with an expected clause list since the Joinable
|
||||
// class member in JoinableClause doesn't implement equals method in its implementations
|
||||
Assert.assertEquals(hashJoinSegment.getClauses().size(), 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_noClauses()
|
||||
{
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
DataSource dataSource = new NoopDataSource();
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.emptyList());
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty());
|
||||
EasyMock.expect(analysis.getDataSource()).andReturn(dataSource);
|
||||
EasyMock.replay(analysis);
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
expectedException.expect(IAE.class);
|
||||
expectedException.expectMessage(StringUtils.format(
|
||||
"No join clauses to build the cache key for data source [%s]",
|
||||
dataSource
|
||||
));
|
||||
joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_noHashJoin()
|
||||
{
|
||||
PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.");
|
||||
PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x != \"h.x\"", "h.");
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
Optional<byte[]> cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
|
||||
Assert.assertFalse(cacheKey.isPresent());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_cachingUnsupported()
|
||||
{
|
||||
PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.");
|
||||
DataSource dataSource = new LookupDataSource("lookup");
|
||||
PreJoinableClause clause2 = makePreJoinableClause(dataSource, "x == \"h.x\"", "h.", JoinType.LEFT);
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
Optional<byte[]> cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
|
||||
Assert.assertFalse(cacheKey.isPresent());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_usableClauses()
|
||||
{
|
||||
|
||||
PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.");
|
||||
PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"h.x\"", "h.");
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
Optional<byte[]> cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
|
||||
Assert.assertTrue(cacheKey.isPresent());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithExpression()
|
||||
{
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "y == \"j.y\"", "j.");
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
|
||||
Optional<byte[]> cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey1.isPresent());
|
||||
Assert.assertNotEquals(0, cacheKey1.get().length);
|
||||
|
||||
PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.");
|
||||
EasyMock.reset(analysis);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
Optional<byte[]> cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey2.isPresent());
|
||||
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinType()
|
||||
{
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.LEFT);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
|
||||
Optional<byte[]> cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey1.isPresent());
|
||||
Assert.assertNotEquals(0, cacheKey1.get().length);
|
||||
|
||||
PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.INNER);
|
||||
EasyMock.reset(analysis);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
Optional<byte[]> cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey2.isPresent());
|
||||
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithPrefix()
|
||||
{
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab");
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
|
||||
Optional<byte[]> cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey1.isPresent());
|
||||
Assert.assertNotEquals(0, cacheKey1.get().length);
|
||||
|
||||
PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "xy");
|
||||
EasyMock.reset(analysis);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
Optional<byte[]> cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey2.isPresent());
|
||||
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithBaseFilter()
|
||||
{
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes();
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab");
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
|
||||
Optional<byte[]> cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey1.isPresent());
|
||||
Assert.assertNotEquals(0, cacheKey1.get().length);
|
||||
|
||||
PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab");
|
||||
EasyMock.reset(analysis);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(FalseDimFilter.instance())).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
Optional<byte[]> cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey2.isPresent());
|
||||
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinable()
|
||||
{
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.");
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
|
||||
Optional<byte[]> cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey1.isPresent());
|
||||
Assert.assertNotEquals(0, cacheKey1.get().length);
|
||||
|
||||
PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"j.x\"", "j.");
|
||||
EasyMock.reset(analysis);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
|
||||
EasyMock.replay(analysis);
|
||||
Optional<byte[]> cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey2.isPresent());
|
||||
|
||||
Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_sameKeyForSameJoin()
|
||||
{
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey());
|
||||
|
||||
PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.");
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
|
||||
Optional<byte[]> cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey1.isPresent());
|
||||
Assert.assertNotEquals(0, cacheKey1.get().length);
|
||||
|
||||
PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.");
|
||||
EasyMock.reset(analysis);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes();
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes();
|
||||
EasyMock.replay(analysis);
|
||||
Optional<byte[]> cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis);
|
||||
Assert.assertTrue(cacheKey2.isPresent());
|
||||
|
||||
Assert.assertArrayEquals(cacheKey1.get(), cacheKey2.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_checkClausePrefixesForDuplicatesAndShadowing_noConflicts()
|
||||
{
|
||||
|
@ -563,10 +205,12 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest
|
|||
Assert.assertEquals(
|
||||
Pair.of(
|
||||
ImmutableList.of(new InDimFilter(
|
||||
"x",
|
||||
INDEXED_TABLE_DS.getRowsAsList().stream().map(row -> row[0].toString()).collect(Collectors.toSet()))
|
||||
"x",
|
||||
INDEXED_TABLE_DS.getRowsAsList().stream().map(row -> row[0].toString()).collect(Collectors.toSet())
|
||||
)
|
||||
),
|
||||
ImmutableList.of(joinableClause) // the joinable clause remains intact since we've duplicates in country column
|
||||
ImmutableList.of(joinableClause)
|
||||
// the joinable clause remains intact since we've duplicates in country column
|
||||
),
|
||||
conversion
|
||||
);
|
||||
|
@ -946,53 +590,4 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest
|
|||
conversion
|
||||
);
|
||||
}
|
||||
|
||||
private PreJoinableClause makeGlobalPreJoinableClause(String tableName, String expression, String prefix)
|
||||
{
|
||||
return makeGlobalPreJoinableClause(tableName, expression, prefix, JoinType.LEFT);
|
||||
}
|
||||
|
||||
private PreJoinableClause makeGlobalPreJoinableClause(
|
||||
String tableName,
|
||||
String expression,
|
||||
String prefix,
|
||||
JoinType joinType
|
||||
)
|
||||
{
|
||||
GlobalTableDataSource dataSource = new GlobalTableDataSource(tableName);
|
||||
return makePreJoinableClause(dataSource, expression, prefix, joinType);
|
||||
}
|
||||
|
||||
private PreJoinableClause makePreJoinableClause(
|
||||
DataSource dataSource,
|
||||
String expression,
|
||||
String prefix,
|
||||
JoinType joinType
|
||||
)
|
||||
{
|
||||
JoinConditionAnalysis conditionAnalysis = JoinConditionAnalysis.forExpression(
|
||||
expression,
|
||||
prefix,
|
||||
ExprMacroTable.nil()
|
||||
);
|
||||
return new PreJoinableClause(
|
||||
prefix,
|
||||
dataSource,
|
||||
joinType,
|
||||
conditionAnalysis
|
||||
);
|
||||
}
|
||||
|
||||
private static class JoinableFactoryWithCacheKey extends NoopJoinableFactory
|
||||
{
|
||||
@Override
|
||||
public Optional<byte[]> computeJoinCacheKey(DataSource dataSource, JoinConditionAnalysis condition)
|
||||
{
|
||||
if (dataSource.isCacheable(false) && condition.canHashJoin()) {
|
||||
String tableName = Iterators.getOnlyElement(dataSource.getTableNames().iterator());
|
||||
return Optional.of(StringUtils.toUtf8(tableName));
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,9 +20,13 @@
|
|||
package org.apache.druid.segment.join;
|
||||
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* A datasource that returns nothing. Only used to test un-registered datasources.
|
||||
|
@ -64,4 +68,25 @@ public class NoopDataSource implements DataSource
|
|||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(
|
||||
Query query,
|
||||
AtomicLong cpuTime
|
||||
)
|
||||
{
|
||||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
return newSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new byte[]{};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -108,9 +108,9 @@ public class CacheUtil
|
|||
ServerType serverType
|
||||
)
|
||||
{
|
||||
return isQueryCacheable(query, cacheStrategy, cacheConfig, serverType)
|
||||
return cacheConfig.isUseCache()
|
||||
&& query.context().isUseCache()
|
||||
&& cacheConfig.isUseCache();
|
||||
&& isQueryCacheable(query, cacheStrategy, cacheConfig, serverType);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -296,9 +296,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
|
|||
query,
|
||||
strategy,
|
||||
useCache,
|
||||
populateCache,
|
||||
dataSourceAnalysis,
|
||||
joinableFactoryWrapper
|
||||
populateCache
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -761,24 +759,18 @@ public class CachingClusteredClient implements QuerySegmentWalker
|
|||
{
|
||||
private final Query<T> query;
|
||||
private final CacheStrategy<T, Object, Query<T>> strategy;
|
||||
private final DataSourceAnalysis dataSourceAnalysis;
|
||||
private final JoinableFactoryWrapper joinableFactoryWrapper;
|
||||
private final boolean isSegmentLevelCachingEnable;
|
||||
|
||||
CacheKeyManager(
|
||||
final Query<T> query,
|
||||
final CacheStrategy<T, Object, Query<T>> strategy,
|
||||
final boolean useCache,
|
||||
final boolean populateCache,
|
||||
final DataSourceAnalysis dataSourceAnalysis,
|
||||
final JoinableFactoryWrapper joinableFactoryWrapper
|
||||
final boolean populateCache
|
||||
)
|
||||
{
|
||||
|
||||
this.query = query;
|
||||
this.strategy = strategy;
|
||||
this.dataSourceAnalysis = dataSourceAnalysis;
|
||||
this.joinableFactoryWrapper = joinableFactoryWrapper;
|
||||
this.isSegmentLevelCachingEnable = ((populateCache || useCache)
|
||||
&& !query.context().isBySegment()); // explicit bySegment queries are never cached
|
||||
|
||||
|
@ -840,15 +832,14 @@ public class CachingClusteredClient implements QuerySegmentWalker
|
|||
private byte[] computeQueryCacheKeyWithJoin()
|
||||
{
|
||||
Preconditions.checkNotNull(strategy, "strategy cannot be null");
|
||||
if (dataSourceAnalysis.isJoin()) {
|
||||
byte[] joinDataSourceCacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)
|
||||
.orElse(null);
|
||||
if (null == joinDataSourceCacheKey) {
|
||||
return null; // A join operation that does not support caching
|
||||
}
|
||||
return Bytes.concat(joinDataSourceCacheKey, strategy.computeCacheKey(query));
|
||||
byte[] dataSourceCacheKey = query.getDataSource().getCacheKey();
|
||||
if (null == dataSourceCacheKey) {
|
||||
return null;
|
||||
} else if (dataSourceCacheKey.length > 0) {
|
||||
return Bytes.concat(dataSourceCacheKey, strategy.computeCacheKey(query));
|
||||
} else {
|
||||
return strategy.computeCacheKey(query);
|
||||
}
|
||||
return strategy.computeCacheKey(query);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.druid.client.cache.ForegroundCachePopulator;
|
|||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.guava.FunctionalIterable;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
|
@ -59,7 +58,6 @@ import org.apache.druid.query.spec.SpecificSegmentQueryRunner;
|
|||
import org.apache.druid.query.spec.SpecificSegmentSpec;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.segment.realtime.FireHydrant;
|
||||
import org.apache.druid.segment.realtime.plumber.Sink;
|
||||
|
@ -170,17 +168,15 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker
|
|||
}
|
||||
|
||||
// segmentMapFn maps each base Segment into a joined Segment if necessary.
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn(
|
||||
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null),
|
||||
analysis.getPreJoinableClauses(),
|
||||
cpuTimeAccumulator,
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
);
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = analysis.getDataSource()
|
||||
.createSegmentMapFunction(
|
||||
query,
|
||||
cpuTimeAccumulator
|
||||
);
|
||||
|
||||
|
||||
// We compute the join cache key here itself so it doesn't need to be re-computed for every segment
|
||||
final Optional<byte[]> cacheKeyPrefix = analysis.isJoin()
|
||||
? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis)
|
||||
: Optional.of(StringUtils.EMPTY_BYTES);
|
||||
final Optional<byte[]> cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey());
|
||||
|
||||
Iterable<QueryRunner<T>> perSegmentRunners = Iterables.transform(
|
||||
specs,
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.druid.query.planning.DataSourceAnalysis;
|
|||
import org.apache.druid.segment.ReferenceCountingSegment;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.SegmentWrangler;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
|
@ -94,12 +93,13 @@ public class LocalQuerySegmentWalker implements QuerySegmentWalker
|
|||
|
||||
final AtomicLong cpuAccumulator = new AtomicLong(0L);
|
||||
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn(
|
||||
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null),
|
||||
analysis.getPreJoinableClauses(),
|
||||
cpuAccumulator,
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
);
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = analysis
|
||||
.getDataSource()
|
||||
.createSegmentMapFunction(
|
||||
query,
|
||||
cpuAccumulator
|
||||
);
|
||||
|
||||
|
||||
final QueryRunnerFactory<T, Query<T>> queryRunnerFactory = conglomerate.findFactory(query);
|
||||
final QueryRunner<T> baseRunner = queryRunnerFactory.mergeRunners(
|
||||
|
|
|
@ -59,7 +59,6 @@ import org.apache.druid.query.spec.SpecificSegmentSpec;
|
|||
import org.apache.druid.segment.ReferenceCountingSegment;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.StorageAdapter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.server.SegmentManager;
|
||||
import org.apache.druid.server.SetAndVerifyContextQueryRunner;
|
||||
|
@ -195,18 +194,10 @@ public class ServerManager implements QuerySegmentWalker
|
|||
} else {
|
||||
return new ReportTimelineMissingSegmentQueryRunner<>(Lists.newArrayList(specs));
|
||||
}
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = query.getDataSource().createSegmentMapFunction(query, cpuTimeAccumulator);
|
||||
|
||||
// segmentMapFn maps each base Segment into a joined Segment if necessary.
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn(
|
||||
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null),
|
||||
analysis.getPreJoinableClauses(),
|
||||
cpuTimeAccumulator,
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
);
|
||||
// We compute the join cache key here itself so it doesn't need to be re-computed for every segment
|
||||
final Optional<byte[]> cacheKeyPrefix = analysis.isJoin()
|
||||
? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis)
|
||||
: Optional.of(StringUtils.EMPTY_BYTES);
|
||||
final Optional<byte[]> cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey());
|
||||
|
||||
final FunctionalIterable<QueryRunner<T>> queryRunners = FunctionalIterable
|
||||
.create(specs)
|
||||
|
|
|
@ -25,11 +25,13 @@ import com.google.common.primitives.Bytes;
|
|||
import org.apache.druid.client.selector.QueryableDruidServer;
|
||||
import org.apache.druid.client.selector.ServerSelector;
|
||||
import org.apache.druid.query.CacheStrategy;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContext;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.segment.join.NoopDataSource;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||
|
@ -42,7 +44,6 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.easymock.EasyMock.expect;
|
||||
|
@ -60,6 +61,8 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
|
|||
private JoinableFactoryWrapper joinableFactoryWrapper;
|
||||
@Mock
|
||||
private DataSourceAnalysis dataSourceAnalysis;
|
||||
@Mock
|
||||
private DataSource dataSource;
|
||||
|
||||
private static final byte[] QUERY_CACHE_KEY = new byte[]{1, 2, 3};
|
||||
private static final byte[] JOIN_KEY = new byte[]{4, 5};
|
||||
|
@ -139,7 +142,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
|
|||
@Test
|
||||
public void testComputeEtag_nonJoinDataSource()
|
||||
{
|
||||
expect(dataSourceAnalysis.isJoin()).andReturn(false);
|
||||
expect(query.getDataSource()).andReturn(new NoopDataSource());
|
||||
replayAll();
|
||||
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
|
||||
Set<SegmentServerSelector> selectors = ImmutableSet.of(
|
||||
|
@ -161,8 +164,8 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
|
|||
@Test
|
||||
public void testComputeEtag_joinWithUnsupportedCaching()
|
||||
{
|
||||
expect(dataSourceAnalysis.isJoin()).andReturn(true);
|
||||
expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.empty());
|
||||
expect(query.getDataSource()).andReturn(dataSource);
|
||||
expect(dataSource.getCacheKey()).andReturn(null);
|
||||
replayAll();
|
||||
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
|
||||
Set<SegmentServerSelector> selectors = ImmutableSet.of(
|
||||
|
@ -173,38 +176,15 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
|
|||
Assert.assertNull(actual);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComputeEtag_joinWithSupportedCaching()
|
||||
{
|
||||
expect(dataSourceAnalysis.isJoin()).andReturn(true).anyTimes();
|
||||
expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(JOIN_KEY));
|
||||
replayAll();
|
||||
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
|
||||
Set<SegmentServerSelector> selectors = ImmutableSet.of(
|
||||
makeHistoricalServerSelector(1),
|
||||
makeHistoricalServerSelector(1)
|
||||
);
|
||||
String actual1 = keyManager.computeResultLevelCachingEtag(selectors, null);
|
||||
Assert.assertNotNull(actual1);
|
||||
|
||||
reset(joinableFactoryWrapper);
|
||||
expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(new byte[]{9}));
|
||||
replay(joinableFactoryWrapper);
|
||||
selectors = ImmutableSet.of(
|
||||
makeHistoricalServerSelector(1),
|
||||
makeHistoricalServerSelector(1)
|
||||
);
|
||||
String actual2 = keyManager.computeResultLevelCachingEtag(selectors, null);
|
||||
Assert.assertNotNull(actual2);
|
||||
Assert.assertNotEquals(actual1, actual2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComputeEtag_noEffectifBySegment()
|
||||
{
|
||||
expect(dataSourceAnalysis.isJoin()).andReturn(false);
|
||||
reset(query);
|
||||
|
||||
expect(query.getDataSource()).andReturn(new NoopDataSource());
|
||||
expect(query.context()).andReturn(QueryContext.of(ImmutableMap.of(QueryContexts.BY_SEGMENT_KEY, true))).anyTimes();
|
||||
|
||||
replayAll();
|
||||
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
|
||||
Set<SegmentServerSelector> selectors = ImmutableSet.of(
|
||||
|
@ -219,14 +199,13 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
|
|||
public void testComputeEtag_noEffectIfUseAndPopulateFalse()
|
||||
{
|
||||
expect(dataSourceAnalysis.isJoin()).andReturn(false);
|
||||
expect(query.getDataSource()).andReturn(new NoopDataSource());
|
||||
replayAll();
|
||||
CachingClusteredClient.CacheKeyManager<Object> keyManager = new CachingClusteredClient.CacheKeyManager<>(
|
||||
query,
|
||||
strategy,
|
||||
false,
|
||||
false,
|
||||
dataSourceAnalysis,
|
||||
joinableFactoryWrapper
|
||||
false
|
||||
);
|
||||
Set<SegmentServerSelector> selectors = ImmutableSet.of(
|
||||
makeHistoricalServerSelector(1),
|
||||
|
@ -239,7 +218,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
|
|||
@Test
|
||||
public void testSegmentQueryCacheKey_nonJoinDataSource()
|
||||
{
|
||||
expect(dataSourceAnalysis.isJoin()).andReturn(false);
|
||||
expect(query.getDataSource()).andReturn(new NoopDataSource());
|
||||
replayAll();
|
||||
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
|
||||
byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey();
|
||||
|
@ -249,20 +228,18 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
|
|||
@Test
|
||||
public void testSegmentQueryCacheKey_joinWithUnsupportedCaching()
|
||||
{
|
||||
expect(dataSourceAnalysis.isJoin()).andReturn(true);
|
||||
expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.empty());
|
||||
expect(query.getDataSource()).andReturn(new NoopDataSource());
|
||||
replayAll();
|
||||
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
|
||||
byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey();
|
||||
Assert.assertNull(cacheKey);
|
||||
Assert.assertNotNull(cacheKey);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSegmentQueryCacheKey_joinWithSupportedCaching()
|
||||
{
|
||||
|
||||
expect(dataSourceAnalysis.isJoin()).andReturn(true);
|
||||
expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(JOIN_KEY));
|
||||
expect(query.getDataSource()).andReturn(dataSource);
|
||||
expect(dataSource.getCacheKey()).andReturn(JOIN_KEY);
|
||||
replayAll();
|
||||
CachingClusteredClient.CacheKeyManager<Object> keyManager = makeKeyManager();
|
||||
byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey();
|
||||
|
@ -287,9 +264,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
|
|||
query,
|
||||
strategy,
|
||||
false,
|
||||
false,
|
||||
dataSourceAnalysis,
|
||||
joinableFactoryWrapper
|
||||
false
|
||||
).computeSegmentLevelQueryCacheKey());
|
||||
}
|
||||
|
||||
|
@ -299,9 +274,7 @@ public class CachingClusteredClientCacheKeyManagerTest extends EasyMockSupport
|
|||
query,
|
||||
strategy,
|
||||
true,
|
||||
true,
|
||||
dataSourceAnalysis,
|
||||
joinableFactoryWrapper
|
||||
true
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -127,6 +127,7 @@ public class ClientQuerySegmentWalkerTest
|
|||
private static final String VERSION = "A";
|
||||
private static final ShardSpec SHARD_SPEC = new NumberedShardSpec(0, 1);
|
||||
|
||||
|
||||
private static final InlineDataSource FOO_INLINE = InlineDataSource.fromIterable(
|
||||
ImmutableList.<Object[]>builder()
|
||||
.add(new Object[]{INTERVAL.getStartMillis(), "x", 1})
|
||||
|
@ -502,7 +503,11 @@ public class ClientQuerySegmentWalkerTest
|
|||
"\"j.s\" == \"s\"",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
new JoinableFactoryWrapper(QueryStackTests.makeJoinableFactoryFromDefault(
|
||||
null,
|
||||
null,
|
||||
null))
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularities.ALL)
|
||||
|
@ -570,7 +575,11 @@ public class ClientQuerySegmentWalkerTest
|
|||
"\"j.s\" == \"s\"",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
new JoinableFactoryWrapper(QueryStackTests.makeJoinableFactoryFromDefault(
|
||||
null,
|
||||
null,
|
||||
null))
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularities.ALL)
|
||||
|
@ -749,7 +758,8 @@ public class ClientQuerySegmentWalkerTest
|
|||
"\"j.s\" == \"s\"",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
null
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularities.ALL)
|
||||
|
|
|
@ -45,7 +45,6 @@ import org.apache.druid.query.spec.SpecificSegmentQueryRunner;
|
|||
import org.apache.druid.query.spec.SpecificSegmentSpec;
|
||||
import org.apache.druid.segment.ReferenceCountingSegment;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.timeline.TimelineObjectHolder;
|
||||
import org.apache.druid.timeline.VersionedIntervalTimeline;
|
||||
|
@ -138,11 +137,9 @@ public class TestClusterQuerySegmentWalker implements QuerySegmentWalker
|
|||
throw new ISE("Cannot handle subquery: %s", analysis.getDataSource());
|
||||
}
|
||||
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = joinableFactoryWrapper.createSegmentMapFn(
|
||||
analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null),
|
||||
analysis.getPreJoinableClauses(),
|
||||
new AtomicLong(),
|
||||
analysis.getBaseQuery().orElse(query)
|
||||
final Function<SegmentReference, SegmentReference> segmentMapFn = analysis.getDataSource().createSegmentMapFunction(
|
||||
query,
|
||||
new AtomicLong()
|
||||
);
|
||||
|
||||
final QueryRunner<T> baseRunner = new FinalizeResultsQueryRunner<>(
|
||||
|
|
|
@ -27,12 +27,16 @@ import org.apache.druid.data.input.InputFormat;
|
|||
import org.apache.druid.data.input.InputSource;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Represents external data for INSERT queries. Only used by the SQL layer, not by the query stack.
|
||||
|
@ -119,6 +123,27 @@ public class ExternalDataSource implements DataSource
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Function<SegmentReference, SegmentReference> createSegmentMapFunction(
|
||||
Query query,
|
||||
AtomicLong cpuTime
|
||||
)
|
||||
{
|
||||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSource withUpdatedDataSource(DataSource newSource)
|
||||
{
|
||||
return newSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.QueryContext;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.server.security.Access;
|
||||
import org.apache.druid.server.security.AuthenticationResult;
|
||||
import org.apache.druid.server.security.ResourceAction;
|
||||
|
@ -48,7 +49,6 @@ import org.joda.time.DateTimeZone;
|
|||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -78,6 +78,7 @@ public class PlannerContext
|
|||
private final String sql;
|
||||
private final DruidOperatorTable operatorTable;
|
||||
private final ExprMacroTable macroTable;
|
||||
private final JoinableFactoryWrapper joinableFactoryWrapper;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final PlannerConfig plannerConfig;
|
||||
private final DateTime localNow;
|
||||
|
@ -112,7 +113,8 @@ public class PlannerContext
|
|||
final DruidSchemaCatalog rootSchema,
|
||||
final SqlEngine engine,
|
||||
final Map<String, Object> queryContext,
|
||||
final Set<String> contextKeys
|
||||
final Set<String> contextKeys,
|
||||
final JoinableFactoryWrapper joinableFactoryWrapper
|
||||
)
|
||||
{
|
||||
this.sql = sql;
|
||||
|
@ -125,6 +127,7 @@ public class PlannerContext
|
|||
this.queryContext = queryContext;
|
||||
this.localNow = Preconditions.checkNotNull(localNow, "localNow");
|
||||
this.stringifyArrays = stringifyArrays;
|
||||
this.joinableFactoryWrapper = joinableFactoryWrapper;
|
||||
|
||||
String sqlQueryId = (String) this.queryContext.get(QueryContexts.CTX_SQL_QUERY_ID);
|
||||
// special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle
|
||||
|
@ -143,7 +146,8 @@ public class PlannerContext
|
|||
final DruidSchemaCatalog rootSchema,
|
||||
final SqlEngine engine,
|
||||
final Map<String, Object> queryContext,
|
||||
final Set<String> contextKeys
|
||||
final Set<String> contextKeys,
|
||||
final JoinableFactoryWrapper joinableFactoryWrapper
|
||||
)
|
||||
{
|
||||
final DateTime utcNow;
|
||||
|
@ -183,7 +187,8 @@ public class PlannerContext
|
|||
rootSchema,
|
||||
engine,
|
||||
queryContext,
|
||||
contextKeys
|
||||
contextKeys,
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -217,6 +222,11 @@ public class PlannerContext
|
|||
return localNow.getZone();
|
||||
}
|
||||
|
||||
public JoinableFactoryWrapper getJoinableFactoryWrapper()
|
||||
{
|
||||
return joinableFactoryWrapper;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public String getSchemaResourceType(String schema, String resourceName)
|
||||
{
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.calcite.tools.Frameworks;
|
|||
import org.apache.calcite.tools.ValidationException;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.server.security.Access;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.server.security.NoopEscalator;
|
||||
|
@ -72,6 +73,7 @@ public class PlannerFactory
|
|||
private final AuthorizerMapper authorizerMapper;
|
||||
private final String druidSchemaName;
|
||||
private final CalciteRulesManager calciteRuleManager;
|
||||
private final JoinableFactoryWrapper joinableFactoryWrapper;
|
||||
|
||||
@Inject
|
||||
public PlannerFactory(
|
||||
|
@ -82,7 +84,8 @@ public class PlannerFactory
|
|||
final AuthorizerMapper authorizerMapper,
|
||||
final @Json ObjectMapper jsonMapper,
|
||||
final @DruidSchemaName String druidSchemaName,
|
||||
final CalciteRulesManager calciteRuleManager
|
||||
final CalciteRulesManager calciteRuleManager,
|
||||
final JoinableFactoryWrapper joinableFactoryWrapper
|
||||
)
|
||||
{
|
||||
this.rootSchema = rootSchema;
|
||||
|
@ -93,6 +96,7 @@ public class PlannerFactory
|
|||
this.jsonMapper = jsonMapper;
|
||||
this.druidSchemaName = druidSchemaName;
|
||||
this.calciteRuleManager = calciteRuleManager;
|
||||
this.joinableFactoryWrapper = joinableFactoryWrapper;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -114,7 +118,8 @@ public class PlannerFactory
|
|||
rootSchema,
|
||||
engine,
|
||||
queryContext,
|
||||
contextKeys
|
||||
contextKeys,
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
return new DruidPlanner(buildFrameworkConfig(context), context, engine);
|
||||
|
|
|
@ -195,7 +195,8 @@ public class DruidJoinQueryRel extends DruidRel<DruidJoinQueryRel>
|
|||
condition.getExpression(),
|
||||
toDruidJoinType(joinRel.getJoinType()),
|
||||
getDimFilter(getPlannerContext(), leftSignature, leftFilter),
|
||||
getPlannerContext().getExprMacroTable()
|
||||
getPlannerContext().getExprMacroTable(),
|
||||
getPlannerContext().getJoinableFactoryWrapper()
|
||||
),
|
||||
prefixSignaturePair.rhs,
|
||||
getPlannerContext(),
|
||||
|
|
|
@ -81,6 +81,7 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.Types;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.sql.calcite.aggregation.Aggregation;
|
||||
import org.apache.druid.sql.calcite.aggregation.DimensionExpression;
|
||||
import org.apache.druid.sql.calcite.expression.DruidExpression;
|
||||
|
@ -748,7 +749,8 @@ public class DruidQuery
|
|||
static Pair<DataSource, Filtration> getFiltration(
|
||||
DataSource dataSource,
|
||||
DimFilter filter,
|
||||
VirtualColumnRegistry virtualColumnRegistry
|
||||
VirtualColumnRegistry virtualColumnRegistry,
|
||||
JoinableFactoryWrapper joinableFactoryWrapper
|
||||
)
|
||||
{
|
||||
if (!(dataSource instanceof JoinDataSource)) {
|
||||
|
@ -768,13 +770,16 @@ public class DruidQuery
|
|||
// Adds the intervals from the join left filter to query filtration
|
||||
Filtration queryFiltration = Filtration.create(filter, leftFiltration.getIntervals())
|
||||
.optimize(virtualColumnRegistry.getFullRowSignature());
|
||||
|
||||
|
||||
JoinDataSource newDataSource = JoinDataSource.create(
|
||||
joinDataSource.getLeft(),
|
||||
joinDataSource.getRight(),
|
||||
joinDataSource.getRightPrefix(),
|
||||
joinDataSource.getConditionAnalysis(),
|
||||
joinDataSource.getJoinType(),
|
||||
leftFiltration.getDimFilter()
|
||||
leftFiltration.getDimFilter(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
return Pair.of(newDataSource, queryFiltration);
|
||||
}
|
||||
|
@ -946,7 +951,8 @@ public class DruidQuery
|
|||
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
|
||||
dataSource,
|
||||
filter,
|
||||
virtualColumnRegistry
|
||||
virtualColumnRegistry,
|
||||
plannerContext.getJoinableFactoryWrapper()
|
||||
);
|
||||
final DataSource newDataSource = dataSourceFiltrationPair.lhs;
|
||||
final Filtration filtration = dataSourceFiltrationPair.rhs;
|
||||
|
@ -1056,7 +1062,8 @@ public class DruidQuery
|
|||
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
|
||||
dataSource,
|
||||
filter,
|
||||
virtualColumnRegistry
|
||||
virtualColumnRegistry,
|
||||
plannerContext.getJoinableFactoryWrapper()
|
||||
);
|
||||
final DataSource newDataSource = dataSourceFiltrationPair.lhs;
|
||||
final Filtration filtration = dataSourceFiltrationPair.rhs;
|
||||
|
@ -1154,7 +1161,8 @@ public class DruidQuery
|
|||
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
|
||||
dataSource,
|
||||
filter,
|
||||
virtualColumnRegistry
|
||||
virtualColumnRegistry,
|
||||
plannerContext.getJoinableFactoryWrapper()
|
||||
);
|
||||
final DataSource newDataSource = dataSourceFiltrationPair.lhs;
|
||||
final Filtration filtration = dataSourceFiltrationPair.rhs;
|
||||
|
@ -1199,7 +1207,8 @@ public class DruidQuery
|
|||
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
|
||||
dataSource,
|
||||
filter,
|
||||
virtualColumnRegistry
|
||||
virtualColumnRegistry,
|
||||
plannerContext.getJoinableFactoryWrapper()
|
||||
);
|
||||
final DataSource newDataSource = dataSourceFiltrationPair.lhs;
|
||||
final Filtration filtration = dataSourceFiltrationPair.rhs;
|
||||
|
@ -1324,7 +1333,8 @@ public class DruidQuery
|
|||
final Pair<DataSource, Filtration> dataSourceFiltrationPair = getFiltration(
|
||||
dataSource,
|
||||
filter,
|
||||
virtualColumnRegistry
|
||||
virtualColumnRegistry,
|
||||
plannerContext.getJoinableFactoryWrapper()
|
||||
);
|
||||
final DataSource newDataSource = dataSourceFiltrationPair.lhs;
|
||||
final Filtration filtration = dataSourceFiltrationPair.rhs;
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.druid.query.DefaultQueryConfig;
|
|||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.server.QueryScheduler;
|
||||
import org.apache.druid.server.QueryStackTests;
|
||||
import org.apache.druid.server.initialization.ServerConfig;
|
||||
|
@ -144,6 +145,7 @@ public class SqlStatementTest
|
|||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||
|
||||
testRequestLogger = new TestRequestLogger();
|
||||
final JoinableFactoryWrapper joinableFactoryWrapper = CalciteTests.createJoinableFactoryWrapper();
|
||||
|
||||
final PlannerFactory plannerFactory = new PlannerFactory(
|
||||
rootSchema,
|
||||
|
@ -153,7 +155,8 @@ public class SqlStatementTest
|
|||
CalciteTests.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
|
||||
this.sqlStatementFactory = new SqlStatementFactory(
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.druid.math.expr.ExprMacroTable;
|
|||
import org.apache.druid.query.BaseQuery;
|
||||
import org.apache.druid.query.DefaultQueryConfig;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.server.DruidNode;
|
||||
import org.apache.druid.server.QueryLifecycleFactory;
|
||||
import org.apache.druid.server.QueryScheduler;
|
||||
|
@ -289,6 +290,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
binder.install(new SqlModule.SqlStatementFactoryModule());
|
||||
binder.bind(new TypeLiteral<Supplier<DefaultQueryConfig>>(){}).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of())));
|
||||
binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of()));
|
||||
binder.bind(JoinableFactoryWrapper.class).toInstance(CalciteTests.createJoinableFactoryWrapper());
|
||||
}
|
||||
)
|
||||
.build();
|
||||
|
@ -984,7 +986,8 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -999,6 +1002,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
|
||||
final List<Meta.Frame> frames = new ArrayList<>();
|
||||
final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame");
|
||||
|
||||
DruidMeta smallFrameDruidMeta = new DruidMeta(
|
||||
makeStatementFactory(),
|
||||
config,
|
||||
|
@ -1058,6 +1062,7 @@ public class DruidAvaticaHandlerTest extends CalciteTestBase
|
|||
|
||||
final List<Meta.Frame> frames = new ArrayList<>();
|
||||
final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame");
|
||||
|
||||
DruidMeta smallFrameDruidMeta = new DruidMeta(
|
||||
makeStatementFactory(),
|
||||
config,
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.DateTimes;
|
|||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.server.QueryStackTests;
|
||||
import org.apache.druid.server.security.AllowAllAuthenticator;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
|
@ -103,6 +104,7 @@ public class DruidStatementTest extends CalciteTestBase
|
|||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||
DruidSchemaCatalog rootSchema =
|
||||
CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
final JoinableFactoryWrapper joinableFactoryWrapper = CalciteTests.createJoinableFactoryWrapper();
|
||||
final PlannerFactory plannerFactory = new PlannerFactory(
|
||||
rootSchema,
|
||||
operatorTable,
|
||||
|
@ -111,7 +113,8 @@ public class DruidStatementTest extends CalciteTestBase
|
|||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
this.sqlStatementFactory = CalciteTests.createSqlStatementFactory(
|
||||
CalciteTests.createMockSqlEngine(walker, conglomerate),
|
||||
|
|
|
@ -106,7 +106,6 @@ import org.junit.rules.ExpectedException;
|
|||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -420,7 +419,8 @@ public class BaseCalciteQueryTest extends CalciteTestBase implements QueryCompon
|
|||
condition,
|
||||
joinType,
|
||||
filter,
|
||||
CalciteTests.createExprMacroTable()
|
||||
CalciteTests.createExprMacroTable(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -142,7 +142,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
|
|||
"(\"_d0\" == \"j0.d0\")",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
)
|
||||
)
|
||||
.columns("_d0")
|
||||
|
@ -199,7 +200,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
|
|||
"(\"dim4\" == \"j0._d0\")",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
)
|
||||
)
|
||||
.context(context)
|
||||
|
@ -3248,7 +3250,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
|
|||
"(\"dim2\" == \"j0.dim2\")",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
)
|
||||
)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
|
@ -5088,7 +5091,8 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
|
|||
"(\"l1\" == \"j0.ROW_VALUE\")",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
)
|
||||
)
|
||||
.columns("l1")
|
||||
|
|
|
@ -3944,7 +3944,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"(\"dim1\" == \"j0.ROW_VALUE\")",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
|
@ -13090,7 +13091,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"(\"__time\" == \"j0.__time\")",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
))
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.druid.segment.QueryableIndex;
|
|||
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
|
||||
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
|
||||
import org.apache.druid.segment.generator.SegmentGenerator;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.server.QueryStackTests;
|
||||
import org.apache.druid.server.security.AuthTestUtils;
|
||||
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
|
||||
|
@ -140,6 +141,7 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe
|
|||
final PlannerConfig plannerConfig = new PlannerConfig();
|
||||
final DruidSchemaCatalog rootSchema =
|
||||
CalciteTests.createMockRootSchema(CONGLOMERATE, WALKER, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER);
|
||||
final JoinableFactoryWrapper joinableFactoryWrapper = CalciteTests.createJoinableFactoryWrapper();
|
||||
ENGINE = CalciteTests.createMockSqlEngine(WALKER, CONGLOMERATE);
|
||||
PLANNER_FACTORY = new PlannerFactory(
|
||||
rootSchema,
|
||||
|
@ -149,7 +151,8 @@ public class SqlVectorizedExpressionSanityTest extends InitializedNullHandlingTe
|
|||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.druid.segment.RowBasedColumnSelectorFactory;
|
|||
import org.apache.druid.segment.VirtualColumn;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.segment.virtual.VirtualizedColumnSelectorFactory;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.DruidTypeSystem;
|
||||
|
@ -62,7 +63,6 @@ import org.joda.time.DateTimeZone;
|
|||
import org.junit.Assert;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.math.BigDecimal;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -74,6 +74,7 @@ import java.util.stream.Collectors;
|
|||
|
||||
class ExpressionTestHelper
|
||||
{
|
||||
private static final JoinableFactoryWrapper JOINABLE_FACTORY_WRAPPER = CalciteTests.createJoinableFactoryWrapper();
|
||||
private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create(
|
||||
"SELECT 1", // The actual query isn't important for this test
|
||||
CalciteTests.createOperatorTable(),
|
||||
|
@ -89,7 +90,8 @@ class ExpressionTestHelper
|
|||
),
|
||||
null /* Don't need engine */,
|
||||
Collections.emptyMap(),
|
||||
Collections.emptySet()
|
||||
Collections.emptySet(),
|
||||
JOINABLE_FACTORY_WRAPPER
|
||||
);
|
||||
|
||||
private final RowSignature rowSignature;
|
||||
|
|
|
@ -64,7 +64,8 @@ public class ExternalTableScanRuleTest
|
|||
),
|
||||
engine,
|
||||
Collections.emptyMap(),
|
||||
Collections.emptySet()
|
||||
Collections.emptySet(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
plannerContext.setQueryMaker(
|
||||
engine.buildQueryMakerForSelect(EasyMock.createMock(RelRoot.class), plannerContext)
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.druid.guice.LazySingleton;
|
|||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.jackson.JacksonModule;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.segment.join.JoinableFactoryWrapper;
|
||||
import org.apache.druid.server.QueryLifecycleFactory;
|
||||
import org.apache.druid.server.security.AuthorizerMapper;
|
||||
import org.apache.druid.server.security.ResourceType;
|
||||
|
@ -54,7 +55,6 @@ import org.junit.runner.RunWith;
|
|||
|
||||
import javax.validation.Validation;
|
||||
import javax.validation.Validator;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -81,6 +81,8 @@ public class CalcitePlannerModuleTest extends CalciteTestBase
|
|||
@Mock
|
||||
private ExprMacroTable macroTable;
|
||||
@Mock
|
||||
private JoinableFactoryWrapper joinableFactoryWrapper;
|
||||
@Mock
|
||||
private AuthorizerMapper authorizerMapper;
|
||||
@Mock
|
||||
private DruidSchemaCatalog rootSchema;
|
||||
|
@ -125,6 +127,7 @@ public class CalcitePlannerModuleTest extends CalciteTestBase
|
|||
binder.bind(Key.get(new TypeLiteral<Set<SqlAggregator>>() {})).toInstance(aggregators);
|
||||
binder.bind(Key.get(new TypeLiteral<Set<SqlOperatorConversion>>() {})).toInstance(operatorConversions);
|
||||
binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema);
|
||||
binder.bind(JoinableFactoryWrapper.class).toInstance(joinableFactoryWrapper);
|
||||
},
|
||||
target,
|
||||
binder -> {
|
||||
|
@ -176,7 +179,8 @@ public class CalcitePlannerModuleTest extends CalciteTestBase
|
|||
rootSchema,
|
||||
null,
|
||||
Collections.emptyMap(),
|
||||
Collections.emptySet()
|
||||
Collections.emptySet(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
boolean containsCustomRule = injector.getInstance(CalciteRulesManager.class)
|
||||
.druidConventionRuleSet(context)
|
||||
|
|
|
@ -97,7 +97,8 @@ public class DruidRexExecutorTest extends InitializedNullHandlingTest
|
|||
),
|
||||
null /* Don't need an engine */,
|
||||
Collections.emptyMap(),
|
||||
Collections.emptySet()
|
||||
Collections.emptySet(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
|
||||
private final RexBuilder rexBuilder = new RexBuilder(new JavaTypeFactoryImpl());
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.druid.query.ordering.StringComparators;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.join.JoinType;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -59,10 +60,12 @@ public class DruidQueryTest
|
|||
public void test_filtration_noJoinAndInterval()
|
||||
{
|
||||
DataSource dataSource = new TableDataSource("test");
|
||||
|
||||
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
|
||||
dataSource,
|
||||
selectorFilter,
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false)
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
verify(pair, dataSource, selectorFilter, Intervals.ETERNITY);
|
||||
}
|
||||
|
@ -74,7 +77,8 @@ public class DruidQueryTest
|
|||
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
|
||||
dataSource,
|
||||
filterWithInterval,
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false)
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200));
|
||||
}
|
||||
|
@ -86,7 +90,8 @@ public class DruidQueryTest
|
|||
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
|
||||
dataSource,
|
||||
filterWithInterval,
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false)
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200));
|
||||
}
|
||||
|
@ -99,7 +104,8 @@ public class DruidQueryTest
|
|||
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
|
||||
dataSource,
|
||||
otherFilter,
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false)
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
|
||||
}
|
||||
|
@ -112,7 +118,8 @@ public class DruidQueryTest
|
|||
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
|
||||
dataSource,
|
||||
otherFilter,
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false)
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
|
||||
}
|
||||
|
@ -125,7 +132,8 @@ public class DruidQueryTest
|
|||
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
|
||||
dataSource,
|
||||
otherFilter,
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false)
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
|
||||
}
|
||||
|
@ -138,7 +146,8 @@ public class DruidQueryTest
|
|||
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
|
||||
dataSource,
|
||||
otherFilter,
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false)
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200));
|
||||
}
|
||||
|
@ -156,7 +165,8 @@ public class DruidQueryTest
|
|||
Pair<DataSource, Filtration> pair = DruidQuery.getFiltration(
|
||||
dataSource,
|
||||
queryFilter,
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false)
|
||||
VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
verify(pair, expectedDataSource, otherFilter, Intervals.utc(150, 200));
|
||||
}
|
||||
|
@ -170,7 +180,8 @@ public class DruidQueryTest
|
|||
"c == \"r.c\"",
|
||||
joinType,
|
||||
filter,
|
||||
ExprMacroTable.nil()
|
||||
ExprMacroTable.nil(),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -283,6 +283,11 @@ public class CalciteTests
|
|||
return QueryFrameworkUtils.createExprMacroTable(INJECTOR);
|
||||
}
|
||||
|
||||
public static JoinableFactoryWrapper createJoinableFactoryWrapper()
|
||||
{
|
||||
return new JoinableFactoryWrapper(QueryFrameworkUtils.createDefaultJoinableFactory(INJECTOR));
|
||||
}
|
||||
|
||||
public static DruidOperatorTable createOperatorTable()
|
||||
{
|
||||
return QueryFrameworkUtils.createOperatorTable(INJECTOR);
|
||||
|
|
|
@ -373,7 +373,8 @@ public class SqlTestFramework
|
|||
authorizerMapper,
|
||||
queryJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
final SqlStatementFactory sqlStatementFactory = QueryFrameworkUtils.createSqlStatementFactory(
|
||||
engine,
|
||||
|
|
|
@ -222,6 +222,7 @@ public class SqlResourceTest extends CalciteTestBase
|
|||
);
|
||||
final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable();
|
||||
final ExprMacroTable macroTable = CalciteTests.createExprMacroTable();
|
||||
|
||||
req = request(true);
|
||||
|
||||
testRequestLogger = new TestRequestLogger();
|
||||
|
@ -234,7 +235,8 @@ public class SqlResourceTest extends CalciteTestBase
|
|||
CalciteTests.TEST_AUTHORIZER_MAPPER,
|
||||
CalciteTests.getJsonMapper(),
|
||||
CalciteTests.DRUID_SCHEMA_NAME,
|
||||
new CalciteRulesManager(ImmutableSet.of())
|
||||
new CalciteRulesManager(ImmutableSet.of()),
|
||||
CalciteTests.createJoinableFactoryWrapper()
|
||||
);
|
||||
|
||||
lifecycleManager = new SqlLifecycleManager()
|
||||
|
|
Loading…
Reference in New Issue