mirror of https://github.com/apache/druid.git
Temp changes refactoring
This commit is contained in:
parent
1b75daf811
commit
9da42a9ef0
|
@ -103,6 +103,12 @@ public class InputNumberDataSource implements DataSource
|
|||
return analysis.getDataSource().createSegmentMapFunction(query, new AtomicLong());
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis)
|
||||
{
|
||||
return new byte[]{};
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getInputNumber()
|
||||
{
|
||||
|
|
|
@ -21,6 +21,7 @@ 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.segment.SegmentReference;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -99,4 +100,12 @@ public interface DataSource
|
|||
* @return the segment function
|
||||
*/
|
||||
Function<SegmentReference, SegmentReference> createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc);
|
||||
|
||||
/**
|
||||
*
|
||||
* @return a non-empty byte array - If there is join datasource involved and caching is possible.
|
||||
* 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.
|
||||
*/
|
||||
byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis);
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.RowAdapter;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -243,6 +244,12 @@ public class InlineDataSource implements DataSource
|
|||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis)
|
||||
{
|
||||
return new byte[]{};
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
|
|
|
@ -33,7 +33,9 @@ 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;
|
||||
|
@ -59,6 +61,7 @@ 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;
|
||||
|
@ -86,6 +89,8 @@ public class JoinDataSource implements DataSource
|
|||
private final String rightPrefix;
|
||||
private final JoinConditionAnalysis conditionAnalysis;
|
||||
private final JoinType joinType;
|
||||
private static final byte JOIN_OPERATION = 0x1;
|
||||
private static final Logger log = new Logger(JoinDataSource.class);
|
||||
// An optional filter on the left side if left is direct table access
|
||||
@Nullable
|
||||
private final DimFilter leftFilter;
|
||||
|
@ -425,4 +430,32 @@ public class JoinDataSource implements DataSource
|
|||
);
|
||||
return segmentMapFn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey(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 = 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,6 +23,7 @@ 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.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.Collections;
|
||||
|
@ -108,6 +109,12 @@ public class LookupDataSource implements DataSource
|
|||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis)
|
||||
{
|
||||
return new byte[]{};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -25,6 +25,7 @@ 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.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.Collections;
|
||||
|
@ -100,6 +101,12 @@ public class QueryDataSource implements DataSource
|
|||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis)
|
||||
{
|
||||
return new byte[]{};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -24,6 +24,7 @@ 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.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.Collections;
|
||||
|
@ -105,6 +106,12 @@ public class TableDataSource implements DataSource
|
|||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis)
|
||||
{
|
||||
return new byte[]{};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -26,6 +26,7 @@ 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.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -117,6 +118,12 @@ public class UnionDataSource implements DataSource
|
|||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis)
|
||||
{
|
||||
return new byte[]{};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -22,12 +22,22 @@ 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.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
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.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.JoinableFactoryWrapperTest;
|
||||
import org.apache.druid.segment.join.NoopDataSource;
|
||||
import org.apache.druid.segment.join.NoopJoinableFactory;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.Mock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
|
@ -35,7 +45,9 @@ 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
|
||||
|
@ -180,6 +192,33 @@ public class JoinDataSourceTest
|
|||
Assert.assertEquals(joinDataSource, deserialized);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test_computeJoinDataSourceCacheKey_noClauses()
|
||||
{
|
||||
DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class);
|
||||
JoinDataSource joinDataSource = JoinDataSource.create(
|
||||
new TableDataSource("table1"),
|
||||
new TableDataSource("table2"),
|
||||
"j.",
|
||||
"x == \"j.x\"",
|
||||
JoinType.LEFT,
|
||||
TrueDimFilter.instance(),
|
||||
ExprMacroTable.nil(),
|
||||
joinableFactoryWrapper
|
||||
);
|
||||
EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.emptyList());
|
||||
EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty());
|
||||
EasyMock.expect(analysis.getDataSource()).andReturn(joinDataSource);
|
||||
EasyMock.replay(analysis);
|
||||
|
||||
expectedException.expect(IAE.class);
|
||||
expectedException.expectMessage(StringUtils.format(
|
||||
"No join clauses to build the cache key for data source [%s]",
|
||||
joinDataSource
|
||||
));
|
||||
joinDataSource.getCacheKey(analysis);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testException_leftFilterOnNonTableSource()
|
||||
{
|
||||
|
|
|
@ -124,228 +124,6 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest
|
|||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
@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()
|
||||
|
@ -824,53 +602,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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.segment.join;
|
|||
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
|
||||
import java.util.List;
|
||||
|
@ -77,4 +78,10 @@ public class NoopDataSource implements DataSource
|
|||
{
|
||||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis)
|
||||
{
|
||||
return new byte[]{};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -837,8 +837,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
|
|||
{
|
||||
Preconditions.checkNotNull(strategy, "strategy cannot be null");
|
||||
if (dataSourceAnalysis.isJoin()) {
|
||||
byte[] joinDataSourceCacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)
|
||||
.orElse(null);
|
||||
byte[] joinDataSourceCacheKey = dataSourceAnalysis.getDataSource().getCacheKey(dataSourceAnalysis);
|
||||
if (null == joinDataSourceCacheKey) {
|
||||
return null; // A join operation that does not support caching
|
||||
}
|
||||
|
|
|
@ -178,7 +178,7 @@ public class SinkQuerySegmentWalker implements QuerySegmentWalker
|
|||
|
||||
// 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(analysis.getDataSource().getCacheKey(analysis))
|
||||
: Optional.of(StringUtils.EMPTY_BYTES);
|
||||
|
||||
Iterable<QueryRunner<T>> perSegmentRunners = Iterables.transform(
|
||||
|
|
|
@ -198,7 +198,7 @@ public class ServerManager implements QuerySegmentWalker
|
|||
|
||||
// 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(analysis.getDataSource().getCacheKey(analysis))
|
||||
: Optional.of(StringUtils.EMPTY_BYTES);
|
||||
|
||||
final FunctionalIterable<QueryRunner<T>> queryRunners = FunctionalIterable
|
||||
|
|
|
@ -28,6 +28,7 @@ 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.query.planning.DataSourceAnalysis;
|
||||
import org.apache.druid.segment.SegmentReference;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
|
@ -132,6 +133,12 @@ public class ExternalDataSource implements DataSource
|
|||
return Function.identity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey(final DataSourceAnalysis dataSourceAnalysis)
|
||||
{
|
||||
return new byte[]{};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue