From ad5441e913504220fae3affcfc4d47e92c905e0f Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Tue, 13 Sep 2022 12:00:17 -0700 Subject: [PATCH 01/31] First set of changes for framework --- .../msq/querykit/InputNumberDataSource.java | 11 +++++ .../org/apache/druid/query/DataSource.java | 10 +++++ .../apache/druid/query/InlineDataSource.java | 10 +++++ .../apache/druid/query/JoinDataSource.java | 45 ++++++++++++++++++- .../apache/druid/query/LookupDataSource.java | 10 +++++ .../apache/druid/query/QueryDataSource.java | 10 +++++ .../apache/druid/query/TableDataSource.java | 10 +++++ .../apache/druid/query/UnionDataSource.java | 10 +++++ .../druid/segment/join/NoopDataSource.java | 11 +++++ .../calcite/external/ExternalDataSource.java | 11 +++++ .../sql/calcite/planner/PlannerContext.java | 14 ++++-- .../sql/calcite/planner/PlannerFactory.java | 9 +++- .../apache/druid/sql/SqlStatementTest.java | 5 ++- .../sql/avatica/DruidAvaticaHandlerTest.java | 9 +++- .../druid/sql/avatica/DruidStatementTest.java | 5 ++- .../sql/calcite/BaseCalciteQueryTest.java | 6 ++- .../SqlVectorizedExpressionSanityTest.java | 5 ++- .../expression/ExpressionTestHelper.java | 5 ++- .../external/ExternalTableScanRuleTest.java | 3 +- .../planner/CalcitePlannerModuleTest.java | 6 ++- .../calcite/planner/DruidRexExecutorTest.java | 3 +- .../druid/sql/calcite/util/CalciteTests.java | 5 +++ .../druid/sql/http/SqlResourceTest.java | 5 ++- 23 files changed, 200 insertions(+), 18 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index df3d9fc3b06d..439eee70f4ce 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -24,11 +24,14 @@ 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.function.Function; @JsonTypeName("inputNumber") public class InputNumberDataSource implements DataSource @@ -81,6 +84,14 @@ public boolean isConcrete() return false; } + @Override + public Function createSegmentMapFunction( + Query query + ) + { + return Function.identity(); + } + @JsonProperty public int getInputNumber() { diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 10d8e9806c4a..71d1c06cc1dd 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -21,9 +21,11 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.segment.SegmentReference; import java.util.List; import java.util.Set; +import java.util.function.Function; /** * Represents a source... of data... for a query. Analogous to the "FROM" clause in SQL. @@ -87,4 +89,12 @@ public interface DataSource * @see org.apache.druid.query.planning.DataSourceAnalysis#isConcreteTableBased() which uses this */ boolean isConcrete(); + + /** + * Returns a segment function on to how to segment should be modified. + * + * @param query the input query + * @return the segment function + */ + Function createSegmentMapFunction(Query query); } diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index 163bb986f91c..fecb29be400a 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -27,6 +27,7 @@ import com.google.common.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,6 +37,7 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -182,6 +184,14 @@ public boolean isConcrete() return true; } + @Override + public Function createSegmentMapFunction( + Query query + ) + { + return Function.identity(); + } + /** * 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. diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index a6206bc46198..7e4e49566c73 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -30,16 +30,25 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.segment.SegmentReference; +import org.apache.druid.segment.filter.Filters; 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.Joinable; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import javax.annotation.Nullable; 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; /** @@ -54,7 +63,7 @@ * 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 + * When analyzed by {@link DataSourceAnalysis}, the right-hand side of this datasource * will become a {@link org.apache.druid.query.planning.PreJoinableClause} object. */ public class JoinDataSource implements DataSource @@ -278,4 +287,38 @@ public String toString() ", leftFilter=" + leftFilter + '}'; } + + @Override + public Function createSegmentMapFunction( + Query query + ) + { + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); + final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); + // segmentMapFn maps each base Segment into a joined Segment if necessary. + JoinableFactory jf = new JoinableFactory() + { + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return false; + } + + @Override + public Optional build( + DataSource dataSource, JoinConditionAnalysis condition + ) + { + return Optional.empty(); + } + }; + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(jf); + final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( + analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), + analysis.getPreJoinableClauses(), + cpuTimeAccumulator, + analysis.getBaseQuery().orElse(query) + ); + return segmentMapFn; + } } diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index 4b2b394554f9..36707a9b66ce 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -23,11 +23,13 @@ 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.function.Function; /** * Represents a lookup. @@ -96,6 +98,14 @@ public boolean isConcrete() return true; } + @Override + public Function createSegmentMapFunction( + Query query + ) + { + return Function.identity(); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 475368112c0b..0575eacf0f4d 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -25,10 +25,12 @@ 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.function.Function; @JsonTypeName("query") public class QueryDataSource implements DataSource @@ -88,6 +90,14 @@ public boolean isConcrete() return false; } + @Override + public Function createSegmentMapFunction( + Query query + ) + { + return Function.identity(); + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index dfe785a62659..c147e39874a1 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -24,11 +24,13 @@ 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.function.Function; @JsonTypeName("table") public class TableDataSource implements DataSource @@ -93,6 +95,14 @@ public boolean isConcrete() return true; } + @Override + public Function createSegmentMapFunction( + Query query + ) + { + return Function.identity(); + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index 902c8c8f6e6d..9136558fc669 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -26,9 +26,11 @@ 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.function.Function; import java.util.stream.Collectors; public class UnionDataSource implements DataSource @@ -105,6 +107,14 @@ public boolean isConcrete() return dataSources.stream().allMatch(DataSource::isConcrete); } + @Override + public Function createSegmentMapFunction( + Query query + ) + { + return Function.identity(); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java index 6ad1cb1f8918..b040e7579dfc 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java @@ -20,9 +20,12 @@ 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.function.Function; /** * A datasource that returns nothing. Only used to test un-registered datasources. @@ -64,4 +67,12 @@ public boolean isConcrete() { return false; } + + @Override + public Function createSegmentMapFunction( + Query query + ) + { + return Function.identity(); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index f32c5c1498db..ed88bcdd5f31 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -27,12 +27,15 @@ 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.function.Function; /** * Represents external data for INSERT queries. Only used by the SQL layer, not by the query stack. @@ -119,6 +122,14 @@ public boolean isConcrete() return false; } + @Override + public Function createSegmentMapFunction( + Query query + ) + { + return Function.identity(); + } + @Override public boolean equals(Object o) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 616823b1a66a..02dd10fa5c02 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -35,6 +35,7 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.QueryContext; +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; @@ -77,6 +78,7 @@ public class PlannerContext private final String sql; private final DruidOperatorTable operatorTable; private final ExprMacroTable macroTable; + private final JoinableFactoryWrapper jfw; private final ObjectMapper jsonMapper; private final PlannerConfig plannerConfig; private final DateTime localNow; @@ -110,7 +112,8 @@ private PlannerContext( final boolean stringifyArrays, final DruidSchemaCatalog rootSchema, final SqlEngine engine, - final QueryContext queryContext + final QueryContext queryContext, + JoinableFactoryWrapper jfw ) { this.sql = sql; @@ -123,6 +126,7 @@ private PlannerContext( this.queryContext = queryContext; this.localNow = Preconditions.checkNotNull(localNow, "localNow"); this.stringifyArrays = stringifyArrays; + this.jfw = jfw; String sqlQueryId = (String) this.queryContext.get(CTX_SQL_QUERY_ID); // special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle @@ -140,8 +144,9 @@ public static PlannerContext create( final PlannerConfig plannerConfig, final DruidSchemaCatalog rootSchema, final SqlEngine engine, - final QueryContext queryContext - ) + final QueryContext queryContext, + final JoinableFactoryWrapper jfw + ) { final DateTime utcNow; final DateTimeZone timeZone; @@ -179,7 +184,8 @@ public static PlannerContext create( stringifyArrays, rootSchema, engine, - queryContext + queryContext, + jfw ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java index d8852a87a439..d8b2157c9a8e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java @@ -40,6 +40,7 @@ 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.AuthorizerMapper; import org.apache.druid.server.security.NoopEscalator; @@ -71,6 +72,7 @@ public class PlannerFactory private final AuthorizerMapper authorizerMapper; private final String druidSchemaName; private final CalciteRulesManager calciteRuleManager; + private final JoinableFactoryWrapper jfw; @Inject public PlannerFactory( @@ -81,7 +83,8 @@ public PlannerFactory( final AuthorizerMapper authorizerMapper, final @Json ObjectMapper jsonMapper, final @DruidSchemaName String druidSchemaName, - final CalciteRulesManager calciteRuleManager + final CalciteRulesManager calciteRuleManager, + final JoinableFactoryWrapper jfw ) { this.rootSchema = rootSchema; @@ -92,6 +95,7 @@ public PlannerFactory( this.jsonMapper = jsonMapper; this.druidSchemaName = druidSchemaName; this.calciteRuleManager = calciteRuleManager; + this.jfw = jfw; } /** @@ -107,7 +111,8 @@ public DruidPlanner createPlanner(final SqlEngine engine, final String sql, fina plannerConfig, rootSchema, engine, - queryContext + queryContext, + jfw ); return new DruidPlanner(buildFrameworkConfig(context), context, engine); diff --git a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java index 0653c533a14e..a61bc665c861 100644 --- a/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/SqlStatementTest.java @@ -35,6 +35,7 @@ import org.apache.druid.query.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; @@ -145,6 +146,7 @@ public Sequence run(Query query, Sequence resultSequence) final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); testRequestLogger = new TestRequestLogger(); + final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); final PlannerFactory plannerFactory = new PlannerFactory( rootSchema, @@ -154,7 +156,8 @@ public Sequence run(Query query, Sequence resultSequence) CalciteTests.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + joinableFactoryWrapper ); this.sqlStatementFactory = new SqlStatementFactory( diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java index 7d989cae2300..134de75f4dd9 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -53,6 +53,7 @@ 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; @@ -919,6 +920,7 @@ public int getMaxRowsPerFrame() final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); final List frames = new ArrayList<>(); final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame"); + final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); DruidMeta smallFrameDruidMeta = new DruidMeta( @@ -932,7 +934,8 @@ public int getMaxRowsPerFrame() AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + joinableFactoryWrapper ) ), smallFrameConfig, @@ -1013,6 +1016,7 @@ public int getMinRowsPerFrame() final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); final List frames = new ArrayList<>(); final ScheduledExecutorService exec = Execs.scheduledSingleThreaded("testMaxRowsPerFrame"); + final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); DruidMeta smallFrameDruidMeta = new DruidMeta( @@ -1026,7 +1030,8 @@ public int getMinRowsPerFrame() AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + joinableFactoryWrapper ) ), smallFrameConfig, diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java index 2483dd663eca..9d58d3b0299c 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidStatementTest.java @@ -31,6 +31,7 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContext; 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 void setUp() throws Exception final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); + final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); final PlannerFactory plannerFactory = new PlannerFactory( rootSchema, operatorTable, @@ -111,7 +113,8 @@ public void setUp() throws Exception AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + joinableFactoryWrapper ); this.sqlStatementFactory = CalciteTests.createSqlStatementFactory( CalciteTests.createMockSqlEngine(walker, conglomerate), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 917190596097..fdc4965313bc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -77,6 +77,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.join.JoinType; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AuthConfig; @@ -1078,6 +1079,8 @@ public SqlStatementFactory getSqlStatementFactory( authorizerMapper ); + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); + final PlannerFactory plannerFactory = new PlannerFactory( rootSchema, operatorTable, @@ -1086,7 +1089,8 @@ public SqlStatementFactory getSqlStatementFactory( authorizerMapper, objectMapper, CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + joinableFactoryWrapper ); final SqlStatementFactory sqlStatementFactory = CalciteTests.createSqlStatementFactory( engine, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java index f49bb0c40ea9..aebfd08a34ee 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/SqlVectorizedExpressionSanityTest.java @@ -38,6 +38,7 @@ 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; @@ -139,6 +140,7 @@ public static void setupClass() final PlannerConfig plannerConfig = new PlannerConfig(); final DruidSchemaCatalog rootSchema = CalciteTests.createMockRootSchema(CONGLOMERATE, WALKER, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); + final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); ENGINE = CalciteTests.createMockSqlEngine(WALKER, CONGLOMERATE); PLANNER_FACTORY = new PlannerFactory( rootSchema, @@ -148,7 +150,8 @@ public static void setupClass() AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + joinableFactoryWrapper ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java index 539e585e5fd8..f094c1b18026 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java @@ -43,6 +43,7 @@ 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; @@ -74,6 +75,7 @@ class ExpressionTestHelper { + private static final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create( "SELECT 1", // The actual query isn't important for this test CalciteTests.createOperatorTable(), @@ -88,7 +90,8 @@ NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class)) ) ), null /* Don't need engine */, - new QueryContext() + new QueryContext(), + joinableFactoryWrapper ); private final RowSignature rowSignature; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java index d536cbcbbb14..8d041aea4dfa 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/external/ExternalTableScanRuleTest.java @@ -62,7 +62,8 @@ NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class)) ) ), engine, - new QueryContext() + new QueryContext(), + CalciteTests.createJoinableFactoryWrapper() ); plannerContext.setQueryMaker( engine.buildQueryMakerForSelect(EasyMock.createMock(RelRoot.class), plannerContext) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java index 115b7c0237c2..995b31159d23 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java @@ -35,6 +35,7 @@ import org.apache.druid.jackson.JacksonModule; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContext; +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; @@ -80,6 +81,8 @@ public class CalcitePlannerModuleTest extends CalciteTestBase @Mock private ExprMacroTable macroTable; @Mock + private JoinableFactoryWrapper joinableFactoryWrapper; + @Mock private AuthorizerMapper authorizerMapper; @Mock private DruidSchemaCatalog rootSchema; @@ -174,7 +177,8 @@ public void testExtensionCalciteRule() injector.getInstance(PlannerConfig.class), rootSchema, null, - new QueryContext() + new QueryContext(), + joinableFactoryWrapper ); boolean containsCustomRule = injector.getInstance(CalciteRulesManager.class) .druidConventionRuleSet(context) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java index 6d910384e040..0d25e7688ef7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/DruidRexExecutorTest.java @@ -97,7 +97,8 @@ NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class)) ) ), null /* Don't need an engine */, - new QueryContext() + new QueryContext(), + CalciteTests.createJoinableFactoryWrapper() ); private final RexBuilder rexBuilder = new RexBuilder(new JavaTypeFactoryImpl()); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index adc132d736fa..15f916aec3be 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -1072,6 +1072,11 @@ public static ExprMacroTable createExprMacroTable() return new ExprMacroTable(exprMacros); } + public static JoinableFactoryWrapper createJoinableFactoryWrapper() + { + return new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); + } + public static DruidOperatorTable createOperatorTable() { try { diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 90fadf92802c..110ba8d69c05 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -60,6 +60,7 @@ import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryResponse; import org.apache.druid.server.QueryScheduler; @@ -222,6 +223,7 @@ public Sequence run(Query query, Sequence resultSequence) ); final DruidOperatorTable operatorTable = CalciteTests.createOperatorTable(); final ExprMacroTable macroTable = CalciteTests.createExprMacroTable(); + req = request(true); testRequestLogger = new TestRequestLogger(); @@ -234,7 +236,8 @@ public Sequence run(Query query, Sequence resultSequence) CalciteTests.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); lifecycleManager = new SqlLifecycleManager() From 2c4ba0cd2b46c8dcedc20e60caf38780d0ed1f39 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Tue, 13 Sep 2022 15:42:09 -0700 Subject: [PATCH 02/31] Second set of changes to move segment map function to data source --- .../druid/math/expr/ExprMacroTable.java | 1 + .../apache/druid/query/JoinDataSource.java | 39 ++++++++----------- .../java/org/apache/druid/query/Queries.java | 3 +- .../druid/query/JoinDataSourceTest.java | 18 ++++++--- .../org/apache/druid/query/QueriesTest.java | 18 ++++++--- .../planning/DataSourceAnalysisTest.java | 3 +- .../server/coordination/ServerManager.java | 6 +-- .../server/ClientQuerySegmentWalkerTest.java | 10 +++-- .../sql/calcite/planner/PlannerContext.java | 5 +++ .../sql/calcite/rel/DruidJoinQueryRel.java | 3 +- .../druid/sql/calcite/rel/DruidQuery.java | 24 ++++++++---- .../sql/calcite/BaseCalciteQueryTest.java | 3 +- .../sql/calcite/CalciteJoinQueryTest.java | 12 ++++-- .../druid/sql/calcite/CalciteQueryTest.java | 6 ++- .../druid/sql/calcite/rel/DruidQueryTest.java | 30 +++++++++----- 15 files changed, 114 insertions(+), 67 deletions(-) diff --git a/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java b/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java index ec7c2036c903..718e10dade13 100644 --- a/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java +++ b/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java @@ -44,6 +44,7 @@ public class ExprMacroTable { private static final ExprMacroTable NIL = new ExprMacroTable(Collections.emptyList()); + private final Map macroMap; public ExprMacroTable(final List macros) diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 7e4e49566c73..844b221dec5a 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -76,6 +76,8 @@ 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 JoinDataSource( DataSource left, @@ -83,7 +85,8 @@ private JoinDataSource( String rightPrefix, JoinConditionAnalysis conditionAnalysis, JoinType joinType, - @Nullable DimFilter leftFilter + @Nullable DimFilter leftFilter, + @Nullable JoinableFactoryWrapper joinableFactoryWrapper ) { this.left = Preconditions.checkNotNull(left, "left"); @@ -97,6 +100,7 @@ private JoinDataSource( "left filter is only supported if left data source is direct table access" ); this.leftFilter = leftFilter; + this.joinableFactoryWrapper = joinableFactoryWrapper; } /** @@ -110,7 +114,8 @@ public static JoinDataSource create( @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( @@ -123,7 +128,8 @@ public static JoinDataSource create( macroTable ), joinType, - leftFilter + leftFilter, + joinableFactoryWrapper ); } @@ -136,12 +142,15 @@ public static JoinDataSource create( final String rightPrefix, final JoinConditionAnalysis conditionAnalysis, final JoinType joinType, - final DimFilter leftFilter + final DimFilter leftFilter, + final JoinableFactoryWrapper joinableFactoryWrapper ) { - return new JoinDataSource(left, right, rightPrefix, conditionAnalysis, joinType, leftFilter); + return new JoinDataSource(left, right, rightPrefix, conditionAnalysis, joinType, leftFilter, joinableFactoryWrapper); } + + @Override public Set getTableNames() { @@ -213,7 +222,8 @@ public DataSource withChildren(List children) rightPrefix, conditionAnalysis, joinType, - leftFilter + leftFilter, + joinableFactoryWrapper ); } @@ -295,24 +305,7 @@ public Function createSegmentMapFunction( { final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); - // segmentMapFn maps each base Segment into a joined Segment if necessary. - JoinableFactory jf = new JoinableFactory() - { - @Override - public boolean isDirectlyJoinable(DataSource dataSource) - { - return false; - } - @Override - public Optional build( - DataSource dataSource, JoinConditionAnalysis condition - ) - { - return Optional.empty(); - } - }; - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(jf); final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), analysis.getPreJoinableClauses(), diff --git a/processing/src/main/java/org/apache/druid/query/Queries.java b/processing/src/main/java/org/apache/druid/query/Queries.java index 58de4695faf4..f5938d51c5f7 100644 --- a/processing/src/main/java/org/apache/druid/query/Queries.java +++ b/processing/src/main/java/org/apache/druid/query/Queries.java @@ -207,7 +207,8 @@ public static Query withBaseDataSource(final Query query, final DataSo clause.getPrefix(), clause.getCondition(), clause.getJoinType(), - joinBaseFilter + joinBaseFilter, + null ); joinBaseFilter = null; } diff --git a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java index ff374ee25093..4bc801cf70d1 100644 --- a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java @@ -51,7 +51,8 @@ public class JoinDataSourceTest "x == \"j.x\"", JoinType.LEFT, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); private final JoinDataSource joinTableToTable = JoinDataSource.create( @@ -61,7 +62,8 @@ public class JoinDataSourceTest "x == \"j.x\"", JoinType.LEFT, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); @Test @@ -163,7 +165,8 @@ public void test_serde() throws Exception "x == \"j.x\"", JoinType.LEFT, TrueDimFilter.instance(), - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); final JoinDataSource deserialized = (JoinDataSource) jsonMapper.readValue( @@ -186,7 +189,8 @@ public void testException_leftFilterOnNonTableSource() "x == \"j.x\"", JoinType.LEFT, TrueDimFilter.instance(), - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); } @@ -200,7 +204,8 @@ public void testLeftFilter() "x == \"j.x\"", JoinType.LEFT, TrueDimFilter.instance(), - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); Assert.assertEquals(TrueDimFilter.instance(), dataSource.getLeftFilter()); } @@ -215,7 +220,8 @@ public void testVirtualColumnCandidates() "x == \"j.x\"", JoinType.LEFT, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); Assert.assertEquals(dataSource.getVirtualColumnCandidates(), ImmutableSet.of("x")); } diff --git a/processing/src/test/java/org/apache/druid/query/QueriesTest.java b/processing/src/test/java/org/apache/druid/query/QueriesTest.java index 210314b636b3..b83e256d7a80 100644 --- a/processing/src/test/java/org/apache/druid/query/QueriesTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueriesTest.java @@ -425,7 +425,8 @@ public void testWithBaseDataSourceSubQueryStackWithJoinOnUnion() "\"foo.x\" == \"bar.x\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) .intervals("2000/3000") @@ -462,7 +463,8 @@ public void testWithBaseDataSourceSubQueryStackWithJoinOnUnion() "\"foo.x\" == \"bar.x\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) .intervals("2000/3000") @@ -500,14 +502,16 @@ public void testWithBaseDataSourcedBaseFilterWithMultiJoin() "\"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 void testWithBaseDataSourcedBaseFilterWithMultiJoin() "\"foo.x\" == \"bar.x\"", JoinType.INNER, TrueDimFilter.instance(), - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ), new TableDataSource("foo_outer"), "j0.", "\"foo_outer.x\" == \"bar.x\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) diff --git a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java index aab7d0e0baa2..4ea43979353f 100644 --- a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java +++ b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java @@ -612,7 +612,8 @@ private static JoinDataSource join( joinClause(rightPrefix).getOriginalExpression(), joinType, dimFilter, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ); } diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index 9a31000b4689..cab59cd5774b 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -195,14 +195,14 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable(Lists.newArrayList(specs)); } - + Function segmentMapFn = analysis.getDataSource().createSegmentMapFunction(query); // segmentMapFn maps each base Segment into a joined Segment if necessary. - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( + /*final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), analysis.getPreJoinableClauses(), cpuTimeAccumulator, analysis.getBaseQuery().orElse(query) - ); + );*/ // We compute the join cache key here itself so it doesn't need to be re-computed for every segment final Optional cacheKeyPrefix = analysis.isJoin() ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index e19dfcf189af..029d21ad136a 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -127,6 +127,7 @@ public class ClientQuerySegmentWalkerTest private static final String VERSION = "A"; private static final ShardSpec SHARD_SPEC = new NumberedShardSpec(0, 1); + private static final InlineDataSource FOO_INLINE = InlineDataSource.fromIterable( ImmutableList.builder() .add(new Object[]{INTERVAL.getStartMillis(), "x", 1}) @@ -502,7 +503,8 @@ public void testJoinOnGroupByOnTable() "\"j.s\" == \"s\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) .setGranularity(Granularities.ALL) @@ -570,7 +572,8 @@ public void testJoinOnGroupByOnUnionOfTables() "\"j.s\" == \"s\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) .setGranularity(Granularities.ALL) @@ -749,7 +752,8 @@ public void testJoinOnTableErrorCantInlineTable() "\"j.s\" == \"s\"", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + null ) ) .setGranularity(Granularities.ALL) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 02dd10fa5c02..4476c9e500fd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -219,6 +219,11 @@ public DateTimeZone getTimeZone() return localNow.getZone(); } + public JoinableFactoryWrapper getJoinableFactoryWrapper() + { + return jfw; + } + @Nullable public String getSchemaResourceType(String schema, String resourceName) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java index 137803bb46eb..9ff0a3cf1baf 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java @@ -195,7 +195,8 @@ public DruidQuery toDruidQuery(final boolean finalizeAggregations) condition.getExpression(), toDruidJoinType(joinRel.getJoinType()), getDimFilter(getPlannerContext(), leftSignature, leftFilter), - getPlannerContext().getExprMacroTable() + getPlannerContext().getExprMacroTable(), + getPlannerContext().getJoinableFactoryWrapper() ), prefixSignaturePair.rhs, getPlannerContext(), diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 185d96b945f2..b2581120a510 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -80,6 +80,7 @@ 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; @@ -738,7 +739,8 @@ private VirtualColumns getVirtualColumns(final boolean includeDimensions) static Pair getFiltration( DataSource dataSource, DimFilter filter, - VirtualColumnRegistry virtualColumnRegistry + VirtualColumnRegistry virtualColumnRegistry, + JoinableFactoryWrapper jfw ) { if (!(dataSource instanceof JoinDataSource)) { @@ -758,13 +760,16 @@ static Pair getFiltration( // 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(), + jfw ); return Pair.of(newDataSource, queryFiltration); } @@ -889,7 +894,8 @@ private TimeBoundaryQuery toTimeBoundaryQuery() final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; @@ -999,7 +1005,8 @@ private TimeseriesQuery toTimeseriesQuery() final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; @@ -1093,7 +1100,8 @@ private TopNQuery toTopNQuery() final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; @@ -1138,7 +1146,8 @@ private GroupByQuery toGroupByQuery() final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; @@ -1252,7 +1261,8 @@ private ScanQuery toScanQuery() final Pair dataSourceFiltrationPair = getFiltration( dataSource, filter, - virtualColumnRegistry + virtualColumnRegistry, + plannerContext.getJoinableFactoryWrapper() ); final DataSource newDataSource = dataSourceFiltrationPair.lhs; final Filtration filtration = dataSourceFiltrationPair.rhs; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index fdc4965313bc..765e865b8b0f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -440,7 +440,8 @@ public static JoinDataSource join( condition, joinType, filter, - CalciteTests.createExprMacroTable() + CalciteTests.createExprMacroTable(), + CalciteTests.createJoinableFactoryWrapper() ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index d614c23d5d22..f97c551a7eec 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -143,7 +143,8 @@ public void testInnerJoinWithLimitAndAlias() "(\"_d0\" == \"j0.d0\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .columns("_d0") @@ -200,7 +201,8 @@ public void testExactTopNOnInnerJoinWithLimit() "(\"dim4\" == \"j0._d0\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .context(context) @@ -3249,7 +3251,8 @@ public void testLeftJoinRightTableCanBeEmpty() "(\"dim2\" == \"j0.dim2\")", JoinType.LEFT, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .setInterval(querySegmentSpec(Filtration.eternity())) @@ -5090,7 +5093,8 @@ public void testPlanWithInFilterMoreThanInSubQueryThreshold() "(\"l1\" == \"j0.ROW_VALUE\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .columns("l1") diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 7115e3ca0cdd..b62704968660 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -3901,7 +3901,8 @@ public void testGroupingWithNullInFilter() "(\"dim1\" == \"j0.ROW_VALUE\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -12970,7 +12971,8 @@ public void testJoinWithTimeDimension() "(\"__time\" == \"j0.__time\")", JoinType.INNER, null, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() )) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java index 68edf3f1f284..100b700d32bb 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java @@ -19,6 +19,7 @@ package org.apache.druid.sql.calcite.rel; +import org.apache.calcite.rel.core.Calc; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -35,6 +36,7 @@ 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 +61,12 @@ public class DruidQueryTest public void test_filtration_noJoinAndInterval() { DataSource dataSource = new TableDataSource("test"); + Pair pair = DruidQuery.getFiltration( dataSource, selectorFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, dataSource, selectorFilter, Intervals.ETERNITY); } @@ -74,7 +78,8 @@ public void test_filtration_intervalInQueryFilter() Pair pair = DruidQuery.getFiltration( dataSource, filterWithInterval, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200)); } @@ -86,7 +91,8 @@ public void test_filtration_joinDataSource_intervalInQueryFilter() Pair pair = DruidQuery.getFiltration( dataSource, filterWithInterval, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, dataSource, selectorFilter, Intervals.utc(100, 200)); } @@ -99,7 +105,8 @@ public void test_filtration_joinDataSource_intervalInBaseTableFilter_inner() Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); } @@ -112,7 +119,8 @@ public void test_filtration_joinDataSource_intervalInBaseTableFilter_left() Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); } @@ -125,7 +133,8 @@ public void test_filtration_joinDataSource_intervalInBaseTableFilter_right() Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); } @@ -138,7 +147,8 @@ public void test_filtration_joinDataSource_intervalInBaseTableFilter_full() Pair pair = DruidQuery.getFiltration( dataSource, otherFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(100, 200)); } @@ -156,7 +166,8 @@ public void test_filtration_intervalsInBothFilters() Pair pair = DruidQuery.getFiltration( dataSource, queryFilter, - VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false) + VirtualColumnRegistry.create(RowSignature.empty(), TestExprMacroTable.INSTANCE, false), + CalciteTests.createJoinableFactoryWrapper() ); verify(pair, expectedDataSource, otherFilter, Intervals.utc(150, 200)); } @@ -170,7 +181,8 @@ private JoinDataSource join(JoinType joinType, DimFilter filter) "c == \"r.c\"", joinType, filter, - ExprMacroTable.nil() + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() ); } From 311e252704092702bd521da9825a891319687978 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Tue, 13 Sep 2022 19:26:16 -0700 Subject: [PATCH 03/31] Minot change to server manager --- .../apache/druid/server/coordination/ServerManager.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index cab59cd5774b..b5031e1b63f7 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -196,13 +196,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable(Lists.newArrayList(specs)); } Function segmentMapFn = analysis.getDataSource().createSegmentMapFunction(query); - // segmentMapFn maps each base Segment into a joined Segment if necessary. - /*final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - cpuTimeAccumulator, - analysis.getBaseQuery().orElse(query) - );*/ + // We compute the join cache key here itself so it doesn't need to be re-computed for every segment final Optional cacheKeyPrefix = analysis.isJoin() ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) From fdf482961a8f3e445cd24c91e7b92f1c1165e179 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Tue, 13 Sep 2022 22:20:14 -0700 Subject: [PATCH 04/31] Removing the createSegmentMapFunction from JoinableFactoryWrapper and moving to JoinDataSource --- .../msq/querykit/BroadcastJoinHelper.java | 7 +- .../druid/msq/querykit/DataSourcePlan.java | 4 +- .../msq/querykit/InputNumberDataSource.java | 3 +- .../msq/querykit/BroadcastJoinHelperTest.java | 1 + .../apache/druid/msq/test/MSQTestBase.java | 3 +- .../org/apache/druid/query/DataSource.java | 4 +- .../apache/druid/query/InlineDataSource.java | 3 +- .../apache/druid/query/JoinDataSource.java | 106 ++++++++++++++++- .../apache/druid/query/LookupDataSource.java | 4 +- .../apache/druid/query/QueryDataSource.java | 3 +- .../apache/druid/query/TableDataSource.java | 3 +- .../apache/druid/query/UnionDataSource.java | 3 +- .../druid/segment/join/JoinableClause.java | 2 +- .../segment/join/JoinableFactoryWrapper.java | 82 +------------ .../join/JoinableFactoryWrapperTest.java | 111 ------------------ .../druid/segment/join/NoopDataSource.java | 3 +- .../appenderator/SinkQuerySegmentWalker.java | 13 +- .../druid/server/LocalQuerySegmentWalker.java | 13 +- .../server/coordination/ServerManager.java | 4 +- .../server/TestClusterQuerySegmentWalker.java | 9 +- .../calcite/external/ExternalDataSource.java | 3 +- 21 files changed, 147 insertions(+), 237 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java index 07e730124574..d5312d8065c6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java @@ -143,12 +143,7 @@ public Function makeSegmentMapFn(final 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) - ); + return analysis.getDataSource().createSegmentMapFunction(query, new AtomicLong()); } @VisibleForTesting diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java index 2bd59a7f9cdf..d50ae98e3748 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java @@ -315,9 +315,9 @@ private static DataSourcePlan forJoin( 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, + null ); inputSpecs.addAll(clausePlan.getInputSpecs()); clausePlan.getBroadcastInputs().intStream().forEach(n -> broadcastInputs.add(n + shift)); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index 439eee70f4ce..ab3028bb4e2f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -31,6 +31,7 @@ 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") @@ -86,7 +87,7 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query + Query query, AtomicLong cpuTime ) { return Function.identity(); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java index 3246938b16a7..4a32144d2745 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java @@ -189,6 +189,7 @@ public void testBuildTableAndInlineData() throws IOException "j.", JoinConditionAnalysis.forExpression("x == \"j.x\"", "j.", ExprMacroTable.nil()), JoinType.INNER, + null, null ) ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 880c6f4adead..66dcab53803b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -390,7 +390,8 @@ public String getFormatString() AuthTestUtils.TEST_AUTHORIZER_MAPPER, objectMapper, CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); sqlStatementFactory = CalciteTests.createSqlStatementFactory(engine, plannerFactory); diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 71d1c06cc1dd..aad3ac1ef278 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; /** @@ -94,7 +95,8 @@ public interface DataSource * Returns a segment function on to how to segment should be modified. * * @param query the input query + * @param cpuTimeAcc the cpu time accumulator * @return the segment function */ - Function createSegmentMapFunction(Query query); + Function createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc); } diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index fecb29be400a..a214daae0caf 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -37,6 +37,7 @@ 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; @@ -186,7 +187,7 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query + Query query, AtomicLong cpuTimeAcc ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 844b221dec5a..b949734f5eef 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -26,22 +26,39 @@ 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.math.expr.ExprMacroTable; 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.Joinable; +import org.apache.druid.segment.join.JoinableClause; import org.apache.druid.segment.join.JoinableFactory; 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; @@ -64,7 +81,7 @@ * chosen prefix. * * When analyzed by {@link DataSourceAnalysis}, the right-hand side of this datasource - * will become a {@link org.apache.druid.query.planning.PreJoinableClause} object. + * will become a {@link PreJoinableClause} object. */ public class JoinDataSource implements DataSource { @@ -298,15 +315,96 @@ public String toString() '}'; } + + /** + * Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join + * clauses is > 0). If mapping is not needed, this method will return {@link Function#identity()}. + * + * @param baseFilter Filter to apply before the join takes place + * @param clauses Pre-joinable clauses + * @param cpuTimeAccumulator An accumulator that we will add CPU nanos to; this is part of the function to encourage + * callers to remember to track metrics on CPU time required for creation of Joinables + * @param query The query that will be run on the mapped segments. Usually this should be + * {@code analysis.getBaseQuery().orElse(query)}, where "analysis" is a + * {@link DataSourceAnalysis} and "query" is the original + * query from the end user. + */ + public Function createSegmentMapFn( + @Nullable final Filter baseFilter, + final List clauses, + final AtomicLong cpuTimeAccumulator, + final Query query + ) + { + // compute column correlations here and RHS correlated values + return JvmUtils.safeAccumulateThreadCpuTime( + cpuTimeAccumulator, + () -> { + if (clauses.isEmpty()) { + return Function.identity(); + } else { + final JoinableClauses joinableClauses = JoinableClauses.createClauses(clauses, + joinableFactoryWrapper.getJoinableFactory()); + final JoinFilterRewriteConfig filterRewriteConfig = JoinFilterRewriteConfig.forQuery(query); + + // Pick off any join clauses that can be converted into filters. + final Set requiredColumns = query.getRequiredColumns(); + final Filter baseFilterToUse; + final List clausesToUse; + + if (requiredColumns != null && filterRewriteConfig.isEnableRewriteJoinToFilter()) { + final Pair, List> conversionResult = JoinableFactoryWrapper.convertJoinsToFilters( + joinableClauses.getJoinableClauses(), + requiredColumns, + Ints.checkedCast(Math.min(filterRewriteConfig.getFilterRewriteMaxSize(), Integer.MAX_VALUE)) + ); + + baseFilterToUse = + Filters.maybeAnd( + Lists.newArrayList( + Iterables.concat( + Collections.singleton(baseFilter), + conversionResult.lhs + ) + ) + ).orElse(null); + clausesToUse = conversionResult.rhs; + } else { + baseFilterToUse = baseFilter; + clausesToUse = joinableClauses.getJoinableClauses(); + } + + // Analyze remaining join clauses to see if filters on them can be pushed down. + final JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + new JoinFilterPreAnalysisKey( + filterRewriteConfig, + clausesToUse, + query.getVirtualColumns(), + Filters.maybeAnd(Arrays.asList(baseFilterToUse, Filters.toFilter(query.getFilter()))) + .orElse(null) + ) + ); + + return baseSegment -> + new HashJoinSegment( + baseSegment, + baseFilterToUse, + GuavaUtils.firstNonNull(clausesToUse, ImmutableList.of()), + joinFilterPreAnalysis + ); + } + } + ); + } + @Override public Function createSegmentMapFunction( - Query query + Query query, AtomicLong cpuTimeAccumulator ) { final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); - final AtomicLong cpuTimeAccumulator = new AtomicLong(0L); - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( + final Function segmentMapFn = createSegmentMapFn( analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), analysis.getPreJoinableClauses(), cpuTimeAccumulator, diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index 36707a9b66ce..37b49b3ba6e7 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -29,6 +29,8 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; /** @@ -100,7 +102,7 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query + Query query, AtomicLong cpuTime ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 0575eacf0f4d..f3b86e703f51 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -30,6 +30,7 @@ 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") @@ -92,7 +93,7 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query + Query query, AtomicLong cpuTime ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index c147e39874a1..6c39c25613d5 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -30,6 +30,7 @@ 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") @@ -97,7 +98,7 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query + Query query, AtomicLong cpuTime ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index 9136558fc669..1e0e14c4081b 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import java.util.stream.Collectors; @@ -109,7 +110,7 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query + Query query, AtomicLong cpuTime ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java index ca5bb44ac9db..974b2dea1316 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java @@ -34,7 +34,7 @@ * clause is "t1 JOIN t2 ON t1.x = t2.x" then this class represents "JOIN t2 ON x = t2.x" -- it does not include * references to the left-hand "t1". *

- * Created from {@link org.apache.druid.query.planning.PreJoinableClause} by {@link JoinableFactoryWrapper#createSegmentMapFn}. + * Created from {@link org.apache.druid.query.planning.PreJoinableClause}. */ public class JoinableClause implements ReferenceCountedObject { diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java index 5530d7243dbe..931ddab26090 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java @@ -82,86 +82,6 @@ public JoinableFactory getJoinableFactory() return joinableFactory; } - /** - * Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join - * clauses is > 0). If mapping is not needed, this method will return {@link Function#identity()}. - * - * @param baseFilter Filter to apply before the join takes place - * @param clauses Pre-joinable clauses - * @param cpuTimeAccumulator An accumulator that we will add CPU nanos to; this is part of the function to encourage - * callers to remember to track metrics on CPU time required for creation of Joinables - * @param query The query that will be run on the mapped segments. Usually this should be - * {@code analysis.getBaseQuery().orElse(query)}, where "analysis" is a - * {@link DataSourceAnalysis} and "query" is the original - * query from the end user. - */ - public Function createSegmentMapFn( - @Nullable final Filter baseFilter, - final List clauses, - final AtomicLong cpuTimeAccumulator, - final Query query - ) - { - // compute column correlations here and RHS correlated values - return JvmUtils.safeAccumulateThreadCpuTime( - cpuTimeAccumulator, - () -> { - if (clauses.isEmpty()) { - return Function.identity(); - } else { - final JoinableClauses joinableClauses = JoinableClauses.createClauses(clauses, joinableFactory); - final JoinFilterRewriteConfig filterRewriteConfig = JoinFilterRewriteConfig.forQuery(query); - - // Pick off any join clauses that can be converted into filters. - final Set requiredColumns = query.getRequiredColumns(); - final Filter baseFilterToUse; - final List clausesToUse; - - if (requiredColumns != null && filterRewriteConfig.isEnableRewriteJoinToFilter()) { - final Pair, List> conversionResult = convertJoinsToFilters( - joinableClauses.getJoinableClauses(), - requiredColumns, - Ints.checkedCast(Math.min(filterRewriteConfig.getFilterRewriteMaxSize(), Integer.MAX_VALUE)) - ); - - baseFilterToUse = - Filters.maybeAnd( - Lists.newArrayList( - Iterables.concat( - Collections.singleton(baseFilter), - conversionResult.lhs - ) - ) - ).orElse(null); - clausesToUse = conversionResult.rhs; - } else { - baseFilterToUse = baseFilter; - clausesToUse = joinableClauses.getJoinableClauses(); - } - - // Analyze remaining join clauses to see if filters on them can be pushed down. - final JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - new JoinFilterPreAnalysisKey( - filterRewriteConfig, - clausesToUse, - query.getVirtualColumns(), - Filters.maybeAnd(Arrays.asList(baseFilterToUse, Filters.toFilter(query.getFilter()))) - .orElse(null) - ) - ); - - return baseSegment -> - new HashJoinSegment( - baseSegment, - baseFilterToUse, - GuavaUtils.firstNonNull(clausesToUse, ImmutableList.of()), - joinFilterPreAnalysis - ); - } - } - ); - } - /** * Compute a cache key prefix for a join data source. This includes the data sources that participate in the RHS of a * join as well as any query specific constructs associated with join data source such as base table filter. This key prefix @@ -214,7 +134,7 @@ public Optional computeJoinDataSourceCacheKey( * See {@link #convertJoinToFilter} for details on the logic. */ @VisibleForTesting - static Pair, List> convertJoinsToFilters( + public static Pair, List> convertJoinsToFilters( final List clauses, final Set requiredColumns, final int maxNumFilterValues diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java index 8f5b699ee7fb..bf3ad2125ef3 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java @@ -137,117 +137,6 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest @Rule public ExpectedException expectedException = ExpectedException.none(); - @Test - public void test_createSegmentMapFn_noClauses() - { - final Function segmentMapFn = NOOP_JOINABLE_FACTORY_WRAPPER.createSegmentMapFn( - null, - ImmutableList.of(), - new AtomicLong(), - null - ); - - Assert.assertSame(Function.identity(), segmentMapFn); - } - - @Test - public void test_createSegmentMapFn_unusableClause() - { - final PreJoinableClause clause = makePreJoinableClause( - INDEXED_TABLE_DS, - "country == \"j.country\"", - "j.", - JoinType.LEFT - ); - - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("dataSource is not joinable"); - - final Function ignored = NOOP_JOINABLE_FACTORY_WRAPPER.createSegmentMapFn( - null, - ImmutableList.of(clause), - new AtomicLong(), - null - ); - } - - @Test - public void test_createSegmentMapFn_usableClause() - { - final PreJoinableClause clause = makePreJoinableClause( - INDEXED_TABLE_DS, - "country == \"j.country\"", - "j.", - JoinType.LEFT - ); - - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new InlineJoinableFactory()); - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - null, - ImmutableList.of(clause), - new AtomicLong(), - new TestQuery( - new TableDataSource("test"), - new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, - new HashMap<>() - ) - ); - - Assert.assertNotSame(Function.identity(), segmentMapFn); - } - - @Test - public void test_createSegmentMapFn_usableClause_joinToFilterEnabled() throws IOException - { - final PreJoinableClause clause = makePreJoinableClause( - INDEXED_TABLE_DS, - "country == \"j.country\"", - "j.", - JoinType.INNER - ); - // required columns are necessary for the rewrite - final TestQuery queryWithRequiredColumnsAndJoinFilterRewrite = (TestQuery) new TestQuery( - new TableDataSource("test"), - new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), - false, - new HashMap<>() - ).withOverriddenContext(ImmutableMap.of(QueryContexts.REWRITE_JOIN_TO_FILTER_ENABLE_KEY, "true")); - queryWithRequiredColumnsAndJoinFilterRewrite.setRequiredColumns(ImmutableSet.of("country")); - - final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new InlineJoinableFactory()); - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - null, - ImmutableList.of(clause), - new AtomicLong(), - queryWithRequiredColumnsAndJoinFilterRewrite - ); - - // dummy segment - final SegmentReference baseSegmentReference = ReferenceCountingSegment.wrapRootGenerationSegment( - new QueryableIndexSegment( - JoinTestHelper.createFactIndexBuilder(temporaryFolder.newFolder()).buildMMappedIndex(), - SegmentId.dummy("facts") - ) - ); - - // check the output contains the conversion filter - Assert.assertNotSame(Function.identity(), segmentMapFn); - final SegmentReference joinSegmentReference = segmentMapFn.apply(baseSegmentReference); - Assert.assertTrue(joinSegmentReference instanceof HashJoinSegment); - HashJoinSegment hashJoinSegment = (HashJoinSegment) joinSegmentReference; - Assert.assertEquals( - hashJoinSegment.getBaseFilter(), - new InDimFilter( - "country", - INDEXED_TABLE_DS.getRowsAsList().stream().map(row -> row[0].toString()).collect(Collectors.toSet()) - ) - ); - // the returned clause list is not comparable with an expected clause list since the Joinable - // class member in JoinableClause doesn't implement equals method in its implementations - Assert.assertEquals(hashJoinSegment.getClauses().size(), 1); - } - @Test public void test_computeJoinDataSourceCacheKey_noClauses() { diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java index b040e7579dfc..6dad479e4b00 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; /** @@ -70,7 +71,7 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query + Query query, AtomicLong cpuTime ) { return Function.identity(); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 53783fcbdb39..3456a9e6ec92 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -59,7 +59,6 @@ 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,12 +169,12 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final } // segmentMapFn maps each base Segment into a joined Segment if necessary. - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - cpuTimeAccumulator, - analysis.getBaseQuery().orElse(query) - ); + Function segmentMapFn = analysis.getDataSource() + .createSegmentMapFunction( + query, + cpuTimeAccumulator + ); + // We compute the join cache key here itself so it doesn't need to be re-computed for every segment final Optional cacheKeyPrefix = analysis.isJoin() diff --git a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java index 8ed289b8bce9..a330f4352de3 100644 --- a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java @@ -36,7 +36,6 @@ import org.apache.druid.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,12 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final final AtomicLong cpuAccumulator = new AtomicLong(0L); - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - cpuAccumulator, - analysis.getBaseQuery().orElse(query) - ); + Function segmentMapFn = analysis.getDataSource() + .createSegmentMapFunction( + query, + cpuAccumulator + ); + final QueryRunnerFactory> queryRunnerFactory = conglomerate.findFactory(query); final QueryRunner baseRunner = queryRunnerFactory.mergeRunners( diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index b5031e1b63f7..f06e947ea312 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -195,8 +195,8 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable(Lists.newArrayList(specs)); } - Function segmentMapFn = analysis.getDataSource().createSegmentMapFunction(query); - + Function segmentMapFn = analysis.getDataSource().createSegmentMapFunction(query, cpuTimeAccumulator); + // We compute the join cache key here itself so it doesn't need to be re-computed for every segment final Optional cacheKeyPrefix = analysis.isJoin() ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) diff --git a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java index 673a93be12cf..2245b28169a8 100644 --- a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java @@ -45,7 +45,6 @@ import org.apache.druid.query.spec.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 QueryRunner getQueryRunnerForSegments(final Query query, final throw new ISE("Cannot handle subquery: %s", analysis.getDataSource()); } - final Function segmentMapFn = joinableFactoryWrapper.createSegmentMapFn( - analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), - analysis.getPreJoinableClauses(), - new AtomicLong(), - analysis.getBaseQuery().orElse(query) + final Function segmentMapFn = analysis.getDataSource().createSegmentMapFunction( + query, + new AtomicLong() ); final QueryRunner baseRunner = new FinalizeResultsQueryRunner<>( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index ed88bcdd5f31..659b771ae3e8 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -35,6 +35,7 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; /** @@ -124,7 +125,7 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query + Query query, AtomicLong cpuTime ) { return Function.identity(); From 16fb8d9626ee739494d81a73f82abceb17d9c3af Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Thu, 15 Sep 2022 10:16:28 -0700 Subject: [PATCH 05/31] Checkstyle fixes --- .../msq/querykit/BroadcastJoinHelper.java | 1 - .../msq/querykit/InputNumberDataSource.java | 3 +- .../apache/druid/query/InlineDataSource.java | 103 +++++++------- .../apache/druid/query/JoinDataSource.java | 33 +++-- .../apache/druid/query/LookupDataSource.java | 8 +- .../apache/druid/query/QueryDataSource.java | 3 +- .../apache/druid/query/TableDataSource.java | 3 +- .../apache/druid/query/UnionDataSource.java | 3 +- .../segment/join/JoinableFactoryWrapper.java | 132 ++++++++---------- .../join/JoinableFactoryWrapperTest.java | 33 ++--- .../druid/segment/join/NoopDataSource.java | 3 +- .../server/coordination/ServerManager.java | 1 - .../calcite/external/ExternalDataSource.java | 3 +- .../sql/calcite/planner/PlannerContext.java | 2 +- .../expression/ExpressionTestHelper.java | 4 +- .../druid/sql/calcite/rel/DruidQueryTest.java | 1 - .../druid/sql/http/SqlResourceTest.java | 1 - 17 files changed, 158 insertions(+), 179 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java index d5312d8065c6..4d65cb712cdb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java @@ -37,7 +37,6 @@ 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; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index ab3028bb4e2f..fcc53370b1a9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -87,7 +87,8 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query, AtomicLong cpuTime + Query query, + AtomicLong cpuTime ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index a214daae0caf..dc7606e1be27 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -44,7 +44,7 @@ /** * Represents an inline datasource, where the rows are embedded within the DataSource object itself. - * + *

* The rows are backed by an Iterable, which can be lazy or not. Lazy datasources will only be iterated if someone calls * {@link #getRows()} and iterates the result, or until someone calls {@link #getRowsAsList()}. */ @@ -105,6 +105,55 @@ public static InlineDataSource fromIterable( return new InlineDataSource(rows, signature); } + /** + * A very zealous equality checker for "rows" that respects deep equality of arrays, but nevertheless refrains + * from materializing things needlessly. Useful for unit tests that want to compare equality of different + * InlineDataSource instances. + */ + private static boolean rowsEqual(final Iterable rowsA, final Iterable rowsB) + { + if (rowsA instanceof List && rowsB instanceof List) { + final List listA = (List) rowsA; + final List listB = (List) rowsB; + + if (listA.size() != listB.size()) { + return false; + } + + for (int i = 0; i < listA.size(); i++) { + final Object[] rowA = listA.get(i); + final Object[] rowB = listB.get(i); + + if (!Arrays.equals(rowA, rowB)) { + return false; + } + } + + return true; + } else { + return Objects.equals(rowsA, rowsB); + } + } + + /** + * A very zealous hash code computer for "rows" that is compatible with {@link #rowsEqual}. + */ + private static int rowsHashCode(final Iterable rows) + { + if (rows instanceof List) { + final List list = (List) rows; + + int code = 1; + for (final Object[] row : list) { + code = 31 * code + Arrays.hashCode(row); + } + + return code; + } else { + return Objects.hash(rows); + } + } + @Override public Set getTableNames() { @@ -187,7 +236,8 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query, AtomicLong cpuTimeAcc + Query query, + AtomicLong cpuTimeAcc ) { return Function.identity(); @@ -243,53 +293,4 @@ public String toString() "signature=" + signature + '}'; } - - /** - * A very zealous equality checker for "rows" that respects deep equality of arrays, but nevertheless refrains - * from materializing things needlessly. Useful for unit tests that want to compare equality of different - * InlineDataSource instances. - */ - private static boolean rowsEqual(final Iterable rowsA, final Iterable rowsB) - { - if (rowsA instanceof List && rowsB instanceof List) { - final List listA = (List) rowsA; - final List listB = (List) rowsB; - - if (listA.size() != listB.size()) { - return false; - } - - for (int i = 0; i < listA.size(); i++) { - final Object[] rowA = listA.get(i); - final Object[] rowB = listB.get(i); - - if (!Arrays.equals(rowA, rowB)) { - return false; - } - } - - return true; - } else { - return Objects.equals(rowsA, rowsB); - } - } - - /** - * A very zealous hash code computer for "rows" that is compatible with {@link #rowsEqual}. - */ - private static int rowsHashCode(final Iterable rows) - { - if (rows instanceof List) { - final List list = (List) rows; - - int code = 1; - for (final Object[] row : list) { - code = 31 * code + Arrays.hashCode(row); - } - - return code; - } else { - return Objects.hash(rows); - } - } } diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index b949734f5eef..52ca56d6ee35 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -44,9 +44,7 @@ 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.Joinable; import org.apache.druid.segment.join.JoinableClause; -import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; @@ -56,13 +54,11 @@ 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; @@ -70,16 +66,16 @@ /** * Represents a join of two datasources. - * + *

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

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

* Any columns from the left-hand side that start with "rightPrefix", and are at least one character longer than * the prefix, will be shadowed. It is up to the caller to ensure that no important columns are shadowed by the * chosen prefix. - * + *

* When analyzed by {@link DataSourceAnalysis}, the right-hand side of this datasource * will become a {@link PreJoinableClause} object. */ @@ -163,11 +159,18 @@ public static JoinDataSource create( final JoinableFactoryWrapper joinableFactoryWrapper ) { - return new JoinDataSource(left, right, rightPrefix, conditionAnalysis, joinType, leftFilter, joinableFactoryWrapper); + return new JoinDataSource( + left, + right, + rightPrefix, + conditionAnalysis, + joinType, + leftFilter, + joinableFactoryWrapper + ); } - @Override public Set getTableNames() { @@ -266,6 +269,7 @@ public boolean isConcrete() * Computes a set of column names for left table expressions in join condition which may already have been defined as * a virtual column in the virtual column registry. It helps to remove any extraenous virtual columns created and only * use the relevant ones. + * * @return a set of column names which might be virtual columns on left table in join condition */ public Set getVirtualColumnCandidates() @@ -343,8 +347,10 @@ public Function createSegmentMapFn( if (clauses.isEmpty()) { return Function.identity(); } else { - final JoinableClauses joinableClauses = JoinableClauses.createClauses(clauses, - joinableFactoryWrapper.getJoinableFactory()); + 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. @@ -399,7 +405,8 @@ public Function createSegmentMapFn( @Override public Function createSegmentMapFunction( - Query query, AtomicLong cpuTimeAccumulator + Query query, + AtomicLong cpuTimeAccumulator ) { final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index 37b49b3ba6e7..098a50374b1f 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -29,16 +29,15 @@ import java.util.List; import java.util.Objects; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; /** * Represents a lookup. - * + *

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

* The "lookupName" referred to here should be provided by a * {@link org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider}. */ @@ -102,7 +101,8 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query, AtomicLong cpuTime + Query query, + AtomicLong cpuTime ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index f3b86e703f51..5cff91e3ed1c 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -93,7 +93,8 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query, AtomicLong cpuTime + Query query, + AtomicLong cpuTime ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index 6c39c25613d5..f547a1d263d2 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -98,7 +98,8 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query, AtomicLong cpuTime + Query query, + AtomicLong cpuTime ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index 1e0e14c4081b..6fd36f1188fc 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -110,7 +110,8 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query, AtomicLong cpuTime + Query query, + AtomicLong cpuTime ) { return Function.identity(); diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java index 931ddab26090..3c7dd2c06207 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java @@ -22,42 +22,25 @@ 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; /** @@ -77,60 +60,9 @@ public JoinableFactoryWrapper(final JoinableFactory joinableFactory) this.joinableFactory = Preconditions.checkNotNull(joinableFactory, "joinableFactory"); } - public JoinableFactory getJoinableFactory() - { - return joinableFactory; - } - - /** - * Compute a cache key prefix for a join data source. This includes the data sources that participate in the RHS of a - * join as well as any query specific constructs associated with join data source such as base table filter. This key prefix - * can be used in segment level cache or result level cache. The function can return following wrapped in an - * Optional - * - Non-empty byte array - If there is join datasource involved and caching is possible. The result includes - * join condition expression, join type and cache key returned by joinable factory for each {@link PreJoinableClause} - * - NULL - There is a join but caching is not possible. It may happen if one of the participating datasource - * in the JOIN is not cacheable. - * - * @param dataSourceAnalysis for the join datasource - * - * @return the optional cache key to be used as part of query cache key - * - * @throws {@link IAE} if this operation is called on a non-join data source - */ - public Optional computeJoinDataSourceCacheKey( - final DataSourceAnalysis dataSourceAnalysis - ) - { - final List clauses = dataSourceAnalysis.getPreJoinableClauses(); - if (clauses.isEmpty()) { - throw new IAE("No join clauses to build the cache key for data source [%s]", dataSourceAnalysis.getDataSource()); - } - - final CacheKeyBuilder keyBuilder; - keyBuilder = new CacheKeyBuilder(JOIN_OPERATION); - if (dataSourceAnalysis.getJoinBaseTableFilter().isPresent()) { - keyBuilder.appendCacheable(dataSourceAnalysis.getJoinBaseTableFilter().get()); - } - for (PreJoinableClause clause : clauses) { - Optional bytes = joinableFactory.computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); - if (!bytes.isPresent()) { - // Encountered a data source which didn't support cache yet - log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); - return Optional.empty(); - } - keyBuilder.appendByteArray(bytes.get()); - keyBuilder.appendString(clause.getCondition().getOriginalExpression()); - keyBuilder.appendString(clause.getPrefix()); - keyBuilder.appendString(clause.getJoinType().name()); - } - return Optional.of(keyBuilder.build()); - } - - /** * Converts any join clauses to filters that can be converted, and returns the rest as-is. - * + *

* See {@link #convertJoinToFilter} for details on the logic. */ @VisibleForTesting @@ -191,16 +123,16 @@ public static Pair, List> convertJoinsToFilters( /** * Converts a join clause into an "in" filter if possible. - * + *

* The requirements are: - * + *

* - it must be an INNER equi-join * - the right-hand columns referenced by the condition must not have any duplicate values. If there are duplicates - * values in the column, then the join is tried to be converted to a filter while maintaining the join clause on top - * as well for correct results. + * values in the column, then the join is tried to be converted to a filter while maintaining the join clause on top + * as well for correct results. * - no columns from the right-hand side can appear in "requiredColumns". If the columns from right side are required - * (ie they are directly or indirectly projected in the join output), then the join is tried to be converted to a - * filter while maintaining the join clause on top as well for correct results. + * (ie they are directly or indirectly projected in the join output), then the join is tried to be converted to a + * filter while maintaining the join clause on top as well for correct results. * * @return {@link JoinClauseToFilterConversion} object which contains the converted filter for the clause and a boolean * to represent whether the converted filter encapsulates the whole clause or not. More semantics of the object are @@ -260,12 +192,60 @@ static JoinClauseToFilterConversion convertJoinToFilter( return new JoinClauseToFilterConversion(null, false); } + public JoinableFactory getJoinableFactory() + { + return joinableFactory; + } + + /** + * Compute a cache key prefix for a join data source. This includes the data sources that participate in the RHS of a + * join as well as any query specific constructs associated with join data source such as base table filter. This key prefix + * can be used in segment level cache or result level cache. The function can return following wrapped in an + * Optional + * - Non-empty byte array - If there is join datasource involved and caching is possible. The result includes + * join condition expression, join type and cache key returned by joinable factory for each {@link PreJoinableClause} + * - NULL - There is a join but caching is not possible. It may happen if one of the participating datasource + * in the JOIN is not cacheable. + * + * @param dataSourceAnalysis for the join datasource + * @return the optional cache key to be used as part of query cache key + * @throws {@link IAE} if this operation is called on a non-join data source + */ + public Optional computeJoinDataSourceCacheKey( + final DataSourceAnalysis dataSourceAnalysis + ) + { + final List clauses = dataSourceAnalysis.getPreJoinableClauses(); + if (clauses.isEmpty()) { + throw new IAE("No join clauses to build the cache key for data source [%s]", dataSourceAnalysis.getDataSource()); + } + + final CacheKeyBuilder keyBuilder; + keyBuilder = new CacheKeyBuilder(JOIN_OPERATION); + if (dataSourceAnalysis.getJoinBaseTableFilter().isPresent()) { + keyBuilder.appendCacheable(dataSourceAnalysis.getJoinBaseTableFilter().get()); + } + for (PreJoinableClause clause : clauses) { + Optional bytes = joinableFactory.computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); + if (!bytes.isPresent()) { + // Encountered a data source which didn't support cache yet + log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); + return Optional.empty(); + } + keyBuilder.appendByteArray(bytes.get()); + keyBuilder.appendString(clause.getCondition().getOriginalExpression()); + keyBuilder.appendString(clause.getPrefix()); + keyBuilder.appendString(clause.getJoinType().name()); + } + return Optional.of(keyBuilder.build()); + } + /** * Encapsulates the conversion which happened for a joinable clause. * convertedFilter represents the filter which got generated from the conversion. * joinClauseFullyConverted represents whether convertedFilter fully encapsulated the joinable clause or not. * Encapsulation of the clause means that the filter can replace the whole joinable clause. - * + *

* If convertedFilter is null and joinClauseFullyConverted is true, it means that all parts of the joinable clause can * be broken into filters. Further, all the clause conditions are on columns where the right side is only null values. * In that case, we replace joinable with a FalseFilter. diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java index bf3ad2125ef3..198c25782fce 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java @@ -28,7 +28,6 @@ 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; @@ -36,9 +35,6 @@ 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; @@ -46,10 +42,6 @@ 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,7 +49,6 @@ 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; @@ -65,16 +56,12 @@ 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 +72,12 @@ public class JoinableFactoryWrapperTest extends NullHandlingTest private static final Map TEST_LOOKUP = ImmutableMap.builder() - .put("MX", "Mexico") - .put("NO", "Norway") - .put("SV", "El Salvador") - .put("US", "United States") - .put("", "Empty key") - .build(); + .put("MX", "Mexico") + .put("NO", "Norway") + .put("SV", "El Salvador") + .put("US", "United States") + .put("", "Empty key") + .build(); private static final Set TEST_LOOKUP_KEYS = NullHandling.sqlCompatible() @@ -452,10 +439,12 @@ public void test_convertJoinsToPartialFilters_convertInnerJoin() 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 ); diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java index 6dad479e4b00..c3cdf6b7764f 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java @@ -71,7 +71,8 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query, AtomicLong cpuTime + Query query, + AtomicLong cpuTime ) { return Function.identity(); diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index f06e947ea312..333525ee5810 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -59,7 +59,6 @@ import org.apache.druid.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; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index 659b771ae3e8..f685f889e213 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -125,7 +125,8 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( - Query query, AtomicLong cpuTime + Query query, + AtomicLong cpuTime ) { return Function.identity(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 4476c9e500fd..e641af0a1201 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -146,7 +146,7 @@ public static PlannerContext create( final SqlEngine engine, final QueryContext queryContext, final JoinableFactoryWrapper jfw - ) + ) { final DateTime utcNow; final DateTimeZone timeZone; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java index f094c1b18026..b16d18559b1a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java @@ -75,7 +75,7 @@ class ExpressionTestHelper { - private static final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); + private static final JoinableFactoryWrapper JOINABLE_FACTORY_WRAPPER = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); private static final PlannerContext PLANNER_CONTEXT = PlannerContext.create( "SELECT 1", // The actual query isn't important for this test CalciteTests.createOperatorTable(), @@ -91,7 +91,7 @@ NamedViewSchema.NAME, new NamedViewSchema(EasyMock.createMock(ViewSchema.class)) ), null /* Don't need engine */, new QueryContext(), - joinableFactoryWrapper + JOINABLE_FACTORY_WRAPPER ); private final RowSignature rowSignature; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java index 100b700d32bb..844569d4ab6a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidQueryTest.java @@ -19,7 +19,6 @@ package org.apache.druid.sql.calcite.rel; -import org.apache.calcite.rel.core.Calc; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 110ba8d69c05..a02520f99be8 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -60,7 +60,6 @@ import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryResponse; import org.apache.druid.server.QueryScheduler; From c078ce8a9fc8daa017d86bc54c5c15b463e79038 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Mon, 19 Sep 2022 19:48:10 -0700 Subject: [PATCH 06/31] Patching Eric's fix for injection --- .../apache/druid/query/JoinDataSource.java | 6 ++++++ .../java/org/apache/druid/query/Queries.java | 20 ++++++++++++++----- .../sql/calcite/BaseCalciteQueryTest.java | 4 ++++ 3 files changed, 25 insertions(+), 5 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 52ca56d6ee35..cb0db448ed69 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -223,6 +223,12 @@ public DimFilter getLeftFilter() return leftFilter; } + @Nullable + public JoinableFactoryWrapper getJoinableFactoryWrapper() + { + return joinableFactoryWrapper; + } + @Override public List getChildren() { diff --git a/processing/src/main/java/org/apache/druid/query/Queries.java b/processing/src/main/java/org/apache/druid/query/Queries.java index f5938d51c5f7..63ebe2b6767b 100644 --- a/processing/src/main/java/org/apache/druid/query/Queries.java +++ b/processing/src/main/java/org/apache/druid/query/Queries.java @@ -191,11 +191,19 @@ public static Query withBaseDataSource(final Query query, final DataSo { final Query retVal; - if (query.getDataSource() instanceof QueryDataSource) { - final Query subQuery = ((QueryDataSource) query.getDataSource()).getQuery(); + /* + * Currently, this method is implemented in terms of a static walk doing a bunch of instanceof checks. + * We should likely look into moving this functionality into the DataSource object itself so that they + * can walk and create new objects on their own. This will be necessary as we expand the set of DataSources + * that do actual work, as each of them will need to show up in this if/then waterfall. + */ + final DataSource theDataSource = query.getDataSource(); + if (theDataSource instanceof QueryDataSource) { + final Query subQuery = ((QueryDataSource) theDataSource).getQuery(); retVal = query.withDataSource(new QueryDataSource(withBaseDataSource(subQuery, newBaseDataSource))); - } else { - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); + } else if (theDataSource instanceof JoinDataSource) { + JoinDataSource joinDataSource = (JoinDataSource) theDataSource; + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(theDataSource); DataSource current = newBaseDataSource; DimFilter joinBaseFilter = analysis.getJoinBaseTableFilter().orElse(null); @@ -208,12 +216,14 @@ public static Query withBaseDataSource(final Query query, final DataSo clause.getCondition(), clause.getJoinType(), joinBaseFilter, - null + joinDataSource.getJoinableFactoryWrapper() ); joinBaseFilter = null; } retVal = query.withDataSource(current); + } else { + retVal = query.withDataSource(newBaseDataSource); } // Verify postconditions, just in case. diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 765e865b8b0f..30d217b02b58 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -572,6 +572,10 @@ public final void setMapperInjectableValues(ObjectMapper mapper, Map Date: Thu, 22 Sep 2022 00:10:14 -0700 Subject: [PATCH 07/31] Checkstyle and fixing some CI issues --- .../org/apache/druid/benchmark/query/SqlBenchmark.java | 3 ++- .../druid/benchmark/query/SqlExpressionBenchmark.java | 3 ++- .../druid/benchmark/query/SqlNestedDataBenchmark.java | 3 ++- .../druid/benchmark/query/SqlVsNativeBenchmark.java | 3 ++- .../apache/druid/sql/calcite/BaseCalciteQueryTest.java | 9 ++++----- 5 files changed, 12 insertions(+), 9 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 35974e4430df..6f390ef345e6 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -448,7 +448,8 @@ public void setup() AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java index e1c27afc9889..117367a587dd 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java @@ -322,7 +322,8 @@ public void setup() AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); try { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java index ab3f5de9cef0..a2ace75f011c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java @@ -288,7 +288,8 @@ public void setup() AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); try { diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java index b11188eb98c9..b05166904f9c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java @@ -127,7 +127,8 @@ public void setup() AuthTestUtils.TEST_AUTHORIZER_MAPPER, CalciteTests.getJsonMapper(), CalciteTests.DRUID_SCHEMA_NAME, - new CalciteRulesManager(ImmutableSet.of()) + new CalciteRulesManager(ImmutableSet.of()), + CalciteTests.createJoinableFactoryWrapper() ); groupByQuery = GroupByQuery .builder() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 30d217b02b58..5fcc4e05b57f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -572,11 +572,10 @@ public final void setMapperInjectableValues(ObjectMapper mapper, Map Date: Thu, 22 Sep 2022 08:51:35 -0700 Subject: [PATCH 08/31] Fixing code inspections and some failed tests and one injector for test in avatica --- .../java/org/apache/druid/query/JoinDataSource.java | 2 +- .../apache/druid/segment/join/HashJoinSegment.java | 13 +------------ .../druid/server/ClientQuerySegmentWalkerTest.java | 10 ++++++++-- .../druid/sql/avatica/DruidAvaticaHandlerTest.java | 1 + 4 files changed, 11 insertions(+), 15 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index cb0db448ed69..2ff3fd268cd4 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -156,7 +156,7 @@ public static JoinDataSource create( final JoinConditionAnalysis conditionAnalysis, final JoinType joinType, final DimFilter leftFilter, - final JoinableFactoryWrapper joinableFactoryWrapper + @Nullable @JacksonInject final JoinableFactoryWrapper joinableFactoryWrapper ) { return new JoinDataSource( diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java index c726a8335af6..55dd5ebdc01e 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegment.java @@ -39,7 +39,7 @@ /** * Represents a deep, left-heavy join of a left-hand side baseSegment onto a series of right-hand side clauses. - * + *

* In other words, logically the operation is: join(join(join(baseSegment, clauses[0]), clauses[1]), clauses[2]) etc. */ public class HashJoinSegment implements SegmentReference @@ -148,15 +148,4 @@ public Optional acquireReferences() return Optional.empty(); } } - - @Nullable - public Filter getBaseFilter() - { - return baseFilter; - } - - public List getClauses() - { - return clauses; - } } diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index 029d21ad136a..5e0701680cd0 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -504,7 +504,10 @@ public void testJoinOnGroupByOnTable() JoinType.INNER, null, ExprMacroTable.nil(), - null + new JoinableFactoryWrapper(QueryStackTests.makeJoinableFactoryFromDefault( + null, + null, + null)) ) ) .setGranularity(Granularities.ALL) @@ -573,7 +576,10 @@ public void testJoinOnGroupByOnUnionOfTables() JoinType.INNER, null, ExprMacroTable.nil(), - null + new JoinableFactoryWrapper(QueryStackTests.makeJoinableFactoryFromDefault( + null, + null, + null)) ) ) .setGranularity(Granularities.ALL) diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java index 134de75f4dd9..382c17faef5c 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -224,6 +224,7 @@ public void setUp() throws Exception binder.install(new SqlModule.SqlStatementFactoryModule()); binder.bind(new TypeLiteral>(){}).toInstance(Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of()))); binder.bind(CalciteRulesManager.class).toInstance(new CalciteRulesManager(ImmutableSet.of())); + binder.bind(JoinableFactoryWrapper.class).toInstance(CalciteTests.createJoinableFactoryWrapper()); } ) .build(); From 21405f119e5a6d0f16511fc0149186816ea5e49c Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Thu, 22 Sep 2022 12:12:32 -0700 Subject: [PATCH 09/31] Another set of changes for CI...almost there --- .../apache/druid/query/JoinDataSource.java | 5 +++-- .../druid/query/JoinDataSourceTest.java | 16 +++++++++------- .../org/apache/druid/segment/TestHelper.java | 19 +++++++++++++++++++ .../planner/CalcitePlannerModuleTest.java | 1 + 4 files changed, 32 insertions(+), 9 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 2ff3fd268cd4..7d5dafb6ea54 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -303,13 +303,14 @@ public boolean equals(Object o) Objects.equals(rightPrefix, that.rightPrefix) && Objects.equals(conditionAnalysis, that.conditionAnalysis) && Objects.equals(leftFilter, that.leftFilter) && - joinType == that.joinType; + joinType == that.joinType && + joinableFactoryWrapper == that.joinableFactoryWrapper; } @Override public int hashCode() { - return Objects.hash(left, right, rightPrefix, conditionAnalysis, joinType, leftFilter); + return Objects.hash(left, right, rightPrefix, conditionAnalysis, joinType, leftFilter, joinableFactoryWrapper); } @Override diff --git a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java index 4bc801cf70d1..bd36087c0d81 100644 --- a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java @@ -27,6 +27,8 @@ import org.apache.druid.query.filter.TrueDimFilter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.join.JoinType; +import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.easymock.Mock; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -35,15 +37,12 @@ import java.util.Collections; + public class JoinDataSourceTest { - @Rule - public ExpectedException expectedException = ExpectedException.none(); - 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, @@ -54,7 +53,6 @@ public class JoinDataSourceTest ExprMacroTable.nil(), null ); - private final JoinDataSource joinTableToTable = JoinDataSource.create( fooTable, barTable, @@ -65,6 +63,10 @@ public class JoinDataSourceTest ExprMacroTable.nil(), null ); + @Rule + public ExpectedException expectedException = ExpectedException.none(); + @Mock + private JoinableFactoryWrapper joinableFactoryWrapper; @Test public void test_getTableNames_tableToTable() @@ -157,7 +159,7 @@ public void test_equals() @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"), @@ -166,7 +168,7 @@ public void test_serde() throws Exception JoinType.LEFT, TrueDimFilter.instance(), ExprMacroTable.nil(), - null + joinableFactoryWrapper ); final JoinDataSource deserialized = (JoinDataSource) jsonMapper.readValue( diff --git a/processing/src/test/java/org/apache/druid/segment/TestHelper.java b/processing/src/test/java/org/apache/druid/segment/TestHelper.java index 4ee90243ddd5..296906ff84ed 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -42,6 +42,7 @@ import org.apache.druid.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; @@ -98,11 +99,29 @@ public static ObjectMapper makeJsonMapper() 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) + ); + return mapper; + } + + public static ObjectMapper makeJsonMapperForJoinable(JoinableFactoryWrapper jfw) + { + 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, jfw) ); return mapper; } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java index 995b31159d23..622d01ef2d35 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/CalcitePlannerModuleTest.java @@ -127,6 +127,7 @@ public void onMatch(RelOptRuleCall call) binder.bind(Key.get(new TypeLiteral>() {})).toInstance(aggregators); binder.bind(Key.get(new TypeLiteral>() {})).toInstance(operatorConversions); binder.bind(DruidSchemaCatalog.class).toInstance(rootSchema); + binder.bind(JoinableFactoryWrapper.class).toInstance(joinableFactoryWrapper); }, target, binder -> { From abe497310095ae4db3b5cbe5eb6030d818130669 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Thu, 22 Sep 2022 15:27:17 -0700 Subject: [PATCH 10/31] Equals and hashcode part update --- .../src/main/java/org/apache/druid/query/JoinDataSource.java | 5 ++--- .../test/java/org/apache/druid/query/JoinDataSourceTest.java | 1 + 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 7d5dafb6ea54..2ff3fd268cd4 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -303,14 +303,13 @@ public boolean equals(Object o) Objects.equals(rightPrefix, that.rightPrefix) && Objects.equals(conditionAnalysis, that.conditionAnalysis) && Objects.equals(leftFilter, that.leftFilter) && - joinType == that.joinType && - joinableFactoryWrapper == that.joinableFactoryWrapper; + joinType == that.joinType; } @Override public int hashCode() { - return Objects.hash(left, right, rightPrefix, conditionAnalysis, joinType, leftFilter, joinableFactoryWrapper); + return Objects.hash(left, right, rightPrefix, conditionAnalysis, joinType, leftFilter); } @Override diff --git a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java index bd36087c0d81..c1fe72d18eee 100644 --- a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java @@ -153,6 +153,7 @@ public void test_equals() EqualsVerifier.forClass(JoinDataSource.class) .usingGetClass() .withNonnullFields("left", "right", "rightPrefix", "conditionAnalysis", "joinType") + .withIgnoredFields("joinableFactoryWrapper") .verify(); } From 149599d59f2b4bfb1d75ae20bd9fabaed36b235c Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Mon, 26 Sep 2022 19:13:09 -0700 Subject: [PATCH 11/31] Fixing injector from Eric + refactoring for broadcastJoinHelper --- .../druid/guice/DruidSecondaryModule.java | 3 - .../msq/querykit/BaseLeafFrameProcessor.java | 23 +++- .../msq/querykit/BroadcastJoinHelper.java | 15 +-- .../druid/msq/querykit/DataSourcePlan.java | 2 +- .../msq/querykit/InputNumberDataSource.java | 11 +- .../apache/druid/msq/test/MSQTestBase.java | 125 +++++++++--------- 6 files changed, 92 insertions(+), 87 deletions(-) diff --git a/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java b/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java index bb03146a05bc..d2a3fd0d52fb 100644 --- a/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java +++ b/core/src/main/java/org/apache/druid/guice/DruidSecondaryModule.java @@ -22,7 +22,6 @@ 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 ObjectMapper getSmileMapper(Injector injector) return smileMapper; } - @VisibleForTesting public static void setupJackson(Injector injector, final ObjectMapper mapper) { mapper.setInjectableValues(new GuiceInjectableValues(injector)); setupAnnotationIntrospector(mapper, new GuiceAnnotationIntrospector()); } - @VisibleForTesting public static void setupAnnotationIntrospector( final ObjectMapper mapper, final AnnotationIntrospector annotationIntrospector diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index f971a4cc73f8..c318944256e6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -47,6 +47,7 @@ import java.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 @@ -152,13 +153,23 @@ private boolean initializeSegmentMapFn(final IntSet readableInputs) segmentMapFn = Function.identity(); return true; } else { - final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); - - if (retVal) { - segmentMapFn = broadcastJoinHelper.makeSegmentMapFn(query); + if (query.getDataSource() instanceof InputNumberDataSource) { + final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); + if (retVal) { + InputNumberDataSource inputNumberDataSource = (InputNumberDataSource) query.getDataSource(); + // The InputNumberData source was going through the broadcastJoinHelper which + // was using the JoinableFactoryWrapper to create segment map function. + // After refactoring, the segment map function creation is moved to data source + // Hence for InputNumberDataSource we are setting the broadcast join helper for the data source + // and moving the segment map function creation there + inputNumberDataSource.setBroadcastJoinHelper(broadcastJoinHelper); + segmentMapFn = inputNumberDataSource.createSegmentMapFunction(query, new AtomicLong()); + } + return retVal; + } else { + segmentMapFn = Function.identity(); + return true; } - - return retVal; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java index 4d65cb712cdb..d6e7679023b4 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BroadcastJoinHelper.java @@ -19,7 +19,6 @@ package org.apache.druid.msq.querykit; -import com.google.common.annotations.VisibleForTesting; import it.unimi.dsi.fastutil.ints.Int2IntMap; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntOpenHashSet; @@ -32,17 +31,12 @@ 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.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 @@ -137,16 +131,9 @@ public IntSet getSideChannelNumbers() return sideChannelNumbers; } - public Function makeSegmentMapFn(final Query query) - { - final DataSource dataSourceWithInlinedChannelData = inlineChannelData(query.getDataSource()); - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(dataSourceWithInlinedChannelData); - return analysis.getDataSource().createSegmentMapFunction(query, new AtomicLong()); - } - @VisibleForTesting - DataSource inlineChannelData(final DataSource originalDataSource) + public DataSource inlineChannelData(final DataSource originalDataSource) { if (originalDataSource instanceof InputNumberDataSource) { final int inputNumber = ((InputNumberDataSource) originalDataSource).getInputNumber(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java index d50ae98e3748..73ff9bf030c0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/DataSourcePlan.java @@ -317,7 +317,7 @@ private static DataSourcePlan forJoin( 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, - null + dataSource.getJoinableFactoryWrapper() ); inputSpecs.addAll(clausePlan.getInputSpecs()); clausePlan.getBroadcastInputs().intStream().forEach(n -> broadcastInputs.add(n + shift)); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index fcc53370b1a9..483344de3944 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -25,6 +25,7 @@ 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 java.util.Collections; @@ -38,6 +39,7 @@ public class InputNumberDataSource implements DataSource { private final int inputNumber; + private BroadcastJoinHelper broadcastJoinHelper; @JsonCreator public InputNumberDataSource(@JsonProperty("inputNumber") int inputNumber) @@ -45,6 +47,11 @@ public InputNumberDataSource(@JsonProperty("inputNumber") int inputNumber) this.inputNumber = inputNumber; } + public void setBroadcastJoinHelper(BroadcastJoinHelper broadcastJoinHelper) + { + this.broadcastJoinHelper = broadcastJoinHelper; + } + @Override public Set getTableNames() { @@ -91,7 +98,9 @@ public Function createSegmentMapFunction( AtomicLong cpuTime ) { - return Function.identity(); + final DataSource dataSourceWithInlinedChannelData = broadcastJoinHelper.inlineChannelData(query.getDataSource()); + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(dataSourceWithInlinedChannelData); + return analysis.getDataSource().createSegmentMapFunction(query, new AtomicLong()); } @JsonProperty diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 66dcab53803b..77c4236d9beb 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.test; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; @@ -30,10 +29,8 @@ import com.google.common.collect.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.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; @@ -269,70 +267,70 @@ public void setUp2() 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"; - } - }; + return "test"; + } + }; - GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); + GroupByQueryConfig groupByQueryConfig = new GroupByQueryConfig(); - binder.bind(DruidProcessingConfig.class).toInstance(druidProcessingConfig); - binder.bind(new TypeLiteral>() - { - }).annotatedWith(Self.class).toInstance(ImmutableSet.of(NodeRole.PEON)); - binder.bind(QueryProcessingPool.class) - .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); - binder.bind(DataSegmentProvider.class) - .toInstance((dataSegment, channelCounters) -> - new LazyResourceHolder<>(getSupplierForSegment(dataSegment))); - binder.bind(IndexIO.class).toInstance(indexIO); - binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(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(DruidProcessingConfig.class).toInstance(druidProcessingConfig); + binder.bind(new TypeLiteral>() + { + }).annotatedWith(Self.class).toInstance(ImmutableSet.of(NodeRole.PEON)); + binder.bind(QueryProcessingPool.class) + .toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool"))); + binder.bind(DataSegmentProvider.class) + .toInstance((dataSegment, channelCounters) -> + new LazyResourceHolder<>(getSupplierForSegment(dataSegment))); + binder.bind(IndexIO.class).toInstance(indexIO); + binder.bind(SpecificSegmentsQuerySegmentWalker.class).toInstance(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(LocalDataSegmentPuller.class).toInstance(new LocalDataSegmentPuller()); + + 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 @@ -566,6 +564,8 @@ private ObjectMapper setupObjectMapper(Injector injector) "compaction" ) ).registerSubtypes(ExternalDataSource.class)); + DruidSecondaryModule.setupJackson(injector, mapper); +/* mapper.setInjectableValues( new InjectableValues.Std() .addValue(ObjectMapper.class, mapper) @@ -574,6 +574,7 @@ private ObjectMapper setupObjectMapper(Injector injector) .addValue(LocalDataSegmentPuller.class, new LocalDataSegmentPuller()) .addValue(ExprMacroTable.class, CalciteTests.createExprMacroTable()) ); +*/ mapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local")); From 63313f44896f62e860eff4106f8634edde590f1b Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Mon, 26 Sep 2022 23:44:46 -0700 Subject: [PATCH 12/31] Updating second injector. Might revert later if better way found --- .../java/org/apache/druid/msq/test/MSQTestBase.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 77c4236d9beb..8bade8bd52cc 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -248,7 +248,14 @@ public void tearDown2() @Before public void setUp2() { - 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); + } + ) + ); groupByBuffers = TestGroupByBuffers.createDefault(); @@ -332,6 +339,7 @@ public String getFormatString() binder.bind(ExprMacroTable.class).toInstance(CalciteTests.createExprMacroTable()); binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT); }, + binder -> { // Requirements of JoinableFactoryModule binder.bind(SegmentManager.class).toInstance(EasyMock.createMock(SegmentManager.class)); From 557081a11ef06f068802d5de11f473ab68eb2fbf Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Fri, 30 Sep 2022 15:13:28 -0700 Subject: [PATCH 13/31] Fixing guice issue in JoinableFactory --- .../segment/join/NoopJoinableFactory.java | 27 +++++++++++++++++++ .../java/org/apache/druid/cli/CliRouter.java | 7 ++++- 2 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java diff --git a/processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java b/processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java new file mode 100644 index 000000000000..f014c7254eb5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java @@ -0,0 +1,27 @@ +package org.apache.druid.segment.join; + +import org.apache.druid.query.DataSource; + +import java.util.Optional; + +public class NoopJoinableFactory implements JoinableFactory +{ + public static final NoopJoinableFactory INSTANCE = new NoopJoinableFactory(); + + protected NoopJoinableFactory() + { + // Singleton. + } + + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return false; + } + + @Override + public Optional build(DataSource dataSource, JoinConditionAnalysis condition) + { + return Optional.empty(); + } +} \ No newline at end of file diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index af0ca7c6eb24..6355db856f3e 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -26,6 +26,7 @@ import com.google.inject.Module; import com.google.inject.TypeLiteral; import com.google.inject.name.Names; +import com.google.inject.util.Providers; import org.apache.druid.curator.discovery.DiscoveryModule; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Jerseys; @@ -41,6 +42,9 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupSerdeModule; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.server.AsyncQueryForwardingServlet; import org.apache.druid.server.NoopQuerySegmentWalker; import org.apache.druid.server.http.RouterResource; @@ -112,7 +116,8 @@ protected List getModules() binder.bind(new TypeLiteral>() {}) .toProvider(TieredBrokerSelectorStrategiesProvider.class) .in(LazySingleton.class); - + + binder.bind(JoinableFactory.class).to(NoopJoinableFactory.class).in(LazySingleton.class); binder.bind(QueryCountStatsProvider.class).to(AsyncQueryForwardingServlet.class).in(LazySingleton.class); binder.bind(JettyServerInitializer.class).to(RouterJettyServerInitializer.class).in(LazySingleton.class); From 1b75daf81136670adda4e32cbec4ba3f9bb2dd6b Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Sun, 2 Oct 2022 12:16:44 -0700 Subject: [PATCH 14/31] Addressing review comments part 1 --- .../msq/querykit/BaseLeafFrameProcessor.java | 11 +++-- .../apache/druid/msq/test/MSQTestBase.java | 10 ---- .../apache/druid/query/JoinDataSource.java | 2 +- .../segment/join/NoopJoinableFactory.java | 21 ++++++++- .../segment/join/NoopJoinableFactory.java | 46 ------------------- .../org/apache/druid/client/CacheUtil.java | 4 +- .../druid/server/LocalQuerySegmentWalker.java | 11 +++-- .../java/org/apache/druid/cli/CliRouter.java | 2 - 8 files changed, 35 insertions(+), 72 deletions(-) delete mode 100644 processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index c318944256e6..d1d576d0c2c5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -157,11 +157,12 @@ private boolean initializeSegmentMapFn(final IntSet readableInputs) final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); if (retVal) { InputNumberDataSource inputNumberDataSource = (InputNumberDataSource) query.getDataSource(); - // The InputNumberData source was going through the broadcastJoinHelper which - // was using the JoinableFactoryWrapper to create segment map function. - // After refactoring, the segment map function creation is moved to data source - // Hence for InputNumberDataSource we are setting the broadcast join helper for the data source - // and moving the segment map function creation there + // The InputNumberDataSource requires a BroadcastJoinHelper to be able to create its + // segment map function. It would be a lot better if the InputNumberDataSource actually + // had a way to get that injected into it on its own, but the relationship between these objects + // was figured out during a refactor and using a setter here seemed like the least-bad way to + // make progress on the refactor without breaking functionality. Hopefully, some future + // developer will move this away from a setter. inputNumberDataSource.setBroadcastJoinHelper(broadcastJoinHelper); segmentMapFn = inputNumberDataSource.createSegmentMapFunction(query, new AtomicLong()); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 8bade8bd52cc..72cdc8e7350b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -573,16 +573,6 @@ private ObjectMapper setupObjectMapper(Injector injector) ) ).registerSubtypes(ExternalDataSource.class)); DruidSecondaryModule.setupJackson(injector, mapper); -/* - 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()) - ); -*/ mapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local")); diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 2ff3fd268cd4..a3dda5ea6646 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -156,7 +156,7 @@ public static JoinDataSource create( final JoinConditionAnalysis conditionAnalysis, final JoinType joinType, final DimFilter leftFilter, - @Nullable @JacksonInject final JoinableFactoryWrapper joinableFactoryWrapper + @Nullable final JoinableFactoryWrapper joinableFactoryWrapper ) { return new JoinDataSource( diff --git a/processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java b/processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java index f014c7254eb5..f168ea9e1c86 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/join/NoopJoinableFactory.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.segment.join; import org.apache.druid.query.DataSource; @@ -24,4 +43,4 @@ public Optional build(DataSource dataSource, JoinConditionAnalysis con { return Optional.empty(); } -} \ No newline at end of file +} diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java b/processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java deleted file mode 100644 index f168ea9e1c86..000000000000 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.join; - -import org.apache.druid.query.DataSource; - -import java.util.Optional; - -public class NoopJoinableFactory implements JoinableFactory -{ - public static final NoopJoinableFactory INSTANCE = new NoopJoinableFactory(); - - protected NoopJoinableFactory() - { - // Singleton. - } - - @Override - public boolean isDirectlyJoinable(DataSource dataSource) - { - return false; - } - - @Override - public Optional build(DataSource dataSource, JoinConditionAnalysis condition) - { - return Optional.empty(); - } -} diff --git a/server/src/main/java/org/apache/druid/client/CacheUtil.java b/server/src/main/java/org/apache/druid/client/CacheUtil.java index 88d713c19aba..3a8b753704e9 100644 --- a/server/src/main/java/org/apache/druid/client/CacheUtil.java +++ b/server/src/main/java/org/apache/druid/client/CacheUtil.java @@ -108,9 +108,9 @@ public static boolean isUseSegmentCache( ServerType serverType ) { - return isQueryCacheable(query, cacheStrategy, cacheConfig, serverType) + return cacheConfig.isUseCache() && QueryContexts.isUseCache(query) - && cacheConfig.isUseCache(); + && isQueryCacheable(query, cacheStrategy, cacheConfig, serverType); } /** diff --git a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java index a330f4352de3..2e4357b86186 100644 --- a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java @@ -93,11 +93,12 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final final AtomicLong cpuAccumulator = new AtomicLong(0L); - Function segmentMapFn = analysis.getDataSource() - .createSegmentMapFunction( - query, - cpuAccumulator - ); + Function segmentMapFn = analysis + .getDataSource() + .createSegmentMapFunction( + query, + cpuAccumulator + ); final QueryRunnerFactory> queryRunnerFactory = conglomerate.findFactory(query); diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index 6355db856f3e..fe73f3abb1dc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -26,7 +26,6 @@ import com.google.inject.Module; import com.google.inject.TypeLiteral; import com.google.inject.name.Names; -import com.google.inject.util.Providers; import org.apache.druid.curator.discovery.DiscoveryModule; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Jerseys; @@ -43,7 +42,6 @@ import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.server.AsyncQueryForwardingServlet; import org.apache.druid.server.NoopQuerySegmentWalker; From 9da42a9ef05bbeefddfc62b4019227fd7c975f93 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Mon, 3 Oct 2022 12:50:13 -0700 Subject: [PATCH 15/31] Temp changes refactoring --- .../msq/querykit/InputNumberDataSource.java | 6 + .../org/apache/druid/query/DataSource.java | 9 + .../apache/druid/query/InlineDataSource.java | 7 + .../apache/druid/query/JoinDataSource.java | 33 +++ .../apache/druid/query/LookupDataSource.java | 7 + .../apache/druid/query/QueryDataSource.java | 7 + .../apache/druid/query/TableDataSource.java | 7 + .../apache/druid/query/UnionDataSource.java | 7 + .../druid/query/JoinDataSourceTest.java | 39 +++ .../join/JoinableFactoryWrapperTest.java | 271 ------------------ .../druid/segment/join/NoopDataSource.java | 7 + .../druid/client/CachingClusteredClient.java | 3 +- .../appenderator/SinkQuerySegmentWalker.java | 2 +- .../server/coordination/ServerManager.java | 2 +- .../calcite/external/ExternalDataSource.java | 7 + 15 files changed, 139 insertions(+), 275 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index 483344de3944..2cbddaf910b5 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -103,6 +103,12 @@ public Function createSegmentMapFunction( return analysis.getDataSource().createSegmentMapFunction(query, new AtomicLong()); } + @Override + public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) + { + return new byte[]{}; + } + @JsonProperty public int getInputNumber() { diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index aad3ac1ef278..f515b0ed292a 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -21,6 +21,7 @@ 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 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); } diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index dc7606e1be27..200e6bab4356 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -26,6 +26,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.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 Function createSegmentMapFunction( 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. diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index a3dda5ea6646..4df0a62d796c 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -33,7 +33,9 @@ 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.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 Function createSegmentMapFunction( ); return segmentMapFn; } + + @Override + public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) + { + final List clauses = dataSourceAnalysis.getPreJoinableClauses(); + if (clauses.isEmpty()) { + throw new IAE("No join clauses to build the cache key for data source [%s]", dataSourceAnalysis.getDataSource()); + } + + final CacheKeyBuilder keyBuilder; + keyBuilder = new CacheKeyBuilder(JOIN_OPERATION); + if (dataSourceAnalysis.getJoinBaseTableFilter().isPresent()) { + keyBuilder.appendCacheable(dataSourceAnalysis.getJoinBaseTableFilter().get()); + } + for (PreJoinableClause clause : clauses) { + Optional bytes = joinableFactoryWrapper.getJoinableFactory().computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); + if (!bytes.isPresent()) { + // Encountered a data source which didn't support cache yet + log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); + return new byte[]{}; + } + keyBuilder.appendByteArray(bytes.get()); + keyBuilder.appendString(clause.getCondition().getOriginalExpression()); + keyBuilder.appendString(clause.getPrefix()); + keyBuilder.appendString(clause.getJoinType().name()); + } + return keyBuilder.build(); + } } diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index 098a50374b1f..c36fc6b424d8 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -23,6 +23,7 @@ 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 Function createSegmentMapFunction( return Function.identity(); } + @Override + public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) + { + return new byte[]{}; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 5cff91e3ed1c..807ea4bc2aa5 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -25,6 +25,7 @@ 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 Function createSegmentMapFunction( return Function.identity(); } + @Override + public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) + { + return new byte[]{}; + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index f547a1d263d2..ac89b0b45843 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -24,6 +24,7 @@ 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 Function createSegmentMapFunction( return Function.identity(); } + @Override + public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) + { + return new byte[]{}; + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index 6fd36f1188fc..a857897e7272 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -26,6 +26,7 @@ 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 Function createSegmentMapFunction( return Function.identity(); } + @Override + public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) + { + return new byte[]{}; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java index c1fe72d18eee..2cdac7efadd6 100644 --- a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java @@ -22,12 +22,22 @@ 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.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 void test_serde() throws Exception 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() { diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java index 198c25782fce..e5eb8be18dc4 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java @@ -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 cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - - Assert.assertFalse(cacheKey.isPresent()); - } - - @Test - public void test_computeJoinDataSourceCacheKey_cachingUnsupported() - { - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - DataSource dataSource = new LookupDataSource("lookup"); - PreJoinableClause clause2 = makePreJoinableClause(dataSource, "x == \"h.x\"", "h.", JoinType.LEFT); - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes(); - EasyMock.replay(analysis); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - Optional cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - - Assert.assertFalse(cacheKey.isPresent()); - } - - @Test - public void test_computeJoinDataSourceCacheKey_usableClauses() - { - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"h.x\"", "h."); - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - Optional cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - - Assert.assertTrue(cacheKey.isPresent()); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithExpression() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "y == \"j.y\"", "j."); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinType() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.LEFT); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.INNER); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithPrefix() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "xy"); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithBaseFilter() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(FalseDimFilter.instance())).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinable() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"j.x\"", "j."); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_sameKeyForSameJoin() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertArrayEquals(cacheKey1.get(), cacheKey2.get()); - } @Test public void test_checkClausePrefixesForDuplicatesAndShadowing_noConflicts() @@ -824,53 +602,4 @@ public void test_convertJoinsToFilters_partialConvertJoinsDependedOnByLaterJoins conversion ); } - - private PreJoinableClause makeGlobalPreJoinableClause(String tableName, String expression, String prefix) - { - return makeGlobalPreJoinableClause(tableName, expression, prefix, JoinType.LEFT); - } - - private PreJoinableClause makeGlobalPreJoinableClause( - String tableName, - String expression, - String prefix, - JoinType joinType - ) - { - GlobalTableDataSource dataSource = new GlobalTableDataSource(tableName); - return makePreJoinableClause(dataSource, expression, prefix, joinType); - } - - private PreJoinableClause makePreJoinableClause( - DataSource dataSource, - String expression, - String prefix, - JoinType joinType - ) - { - JoinConditionAnalysis conditionAnalysis = JoinConditionAnalysis.forExpression( - expression, - prefix, - ExprMacroTable.nil() - ); - return new PreJoinableClause( - prefix, - dataSource, - joinType, - conditionAnalysis - ); - } - - private static class JoinableFactoryWithCacheKey extends NoopJoinableFactory - { - @Override - public Optional computeJoinCacheKey(DataSource dataSource, JoinConditionAnalysis condition) - { - if (dataSource.isCacheable(false) && condition.canHashJoin()) { - String tableName = Iterators.getOnlyElement(dataSource.getTableNames().iterator()); - return Optional.of(StringUtils.toUtf8(tableName)); - } - return Optional.empty(); - } - } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java index c3cdf6b7764f..e134a7a9f8c3 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java @@ -21,6 +21,7 @@ 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 Function createSegmentMapFunction( { return Function.identity(); } + + @Override + public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) + { + return new byte[]{}; + } } diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 69952e22ce3a..c3ab1b4f05cf 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -837,8 +837,7 @@ private byte[] computeQueryCacheKeyWithJoin() { 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 } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 3456a9e6ec92..1d5f13d8701f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -178,7 +178,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final // We compute the join cache key here itself so it doesn't need to be re-computed for every segment final Optional cacheKeyPrefix = analysis.isJoin() - ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) + ? Optional.of(analysis.getDataSource().getCacheKey(analysis)) : Optional.of(StringUtils.EMPTY_BYTES); Iterable> perSegmentRunners = Iterables.transform( diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index 333525ee5810..2681fdc8f43c 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -198,7 +198,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable cacheKeyPrefix = analysis.isJoin() - ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) + ? Optional.of(analysis.getDataSource().getCacheKey(analysis)) : Optional.of(StringUtils.EMPTY_BYTES); final FunctionalIterable> queryRunners = FunctionalIterable diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index f685f889e213..81b05dfd3685 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -28,6 +28,7 @@ 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 Function createSegmentMapFunction( return Function.identity(); } + @Override + public byte[] getCacheKey(final DataSourceAnalysis dataSourceAnalysis) + { + return new byte[]{}; + } + @Override public boolean equals(Object o) { From 4022054ca78491aad729e37bbdd36235c1248d60 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Mon, 3 Oct 2022 12:54:35 -0700 Subject: [PATCH 16/31] Revert "Temp changes refactoring" This reverts commit 9da42a9ef05bbeefddfc62b4019227fd7c975f93. --- .../msq/querykit/InputNumberDataSource.java | 6 - .../org/apache/druid/query/DataSource.java | 9 - .../apache/druid/query/InlineDataSource.java | 7 - .../apache/druid/query/JoinDataSource.java | 33 --- .../apache/druid/query/LookupDataSource.java | 7 - .../apache/druid/query/QueryDataSource.java | 7 - .../apache/druid/query/TableDataSource.java | 7 - .../apache/druid/query/UnionDataSource.java | 7 - .../druid/query/JoinDataSourceTest.java | 39 --- .../join/JoinableFactoryWrapperTest.java | 271 ++++++++++++++++++ .../druid/segment/join/NoopDataSource.java | 7 - .../druid/client/CachingClusteredClient.java | 3 +- .../appenderator/SinkQuerySegmentWalker.java | 2 +- .../server/coordination/ServerManager.java | 2 +- .../calcite/external/ExternalDataSource.java | 7 - 15 files changed, 275 insertions(+), 139 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index 2cbddaf910b5..483344de3944 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -103,12 +103,6 @@ public Function createSegmentMapFunction( return analysis.getDataSource().createSegmentMapFunction(query, new AtomicLong()); } - @Override - public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) - { - return new byte[]{}; - } - @JsonProperty public int getInputNumber() { diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index f515b0ed292a..aad3ac1ef278 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -21,7 +21,6 @@ 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; @@ -100,12 +99,4 @@ public interface DataSource * @return the segment function */ Function 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); } diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index 200e6bab4356..dc7606e1be27 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -26,7 +26,6 @@ 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; @@ -244,12 +243,6 @@ public Function createSegmentMapFunction( 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. diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 4df0a62d796c..a3dda5ea6646 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -33,9 +33,7 @@ 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; @@ -61,7 +59,6 @@ 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; @@ -89,8 +86,6 @@ 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; @@ -430,32 +425,4 @@ public Function createSegmentMapFunction( ); return segmentMapFn; } - - @Override - public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) - { - final List clauses = dataSourceAnalysis.getPreJoinableClauses(); - if (clauses.isEmpty()) { - throw new IAE("No join clauses to build the cache key for data source [%s]", dataSourceAnalysis.getDataSource()); - } - - final CacheKeyBuilder keyBuilder; - keyBuilder = new CacheKeyBuilder(JOIN_OPERATION); - if (dataSourceAnalysis.getJoinBaseTableFilter().isPresent()) { - keyBuilder.appendCacheable(dataSourceAnalysis.getJoinBaseTableFilter().get()); - } - for (PreJoinableClause clause : clauses) { - Optional bytes = joinableFactoryWrapper.getJoinableFactory().computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); - if (!bytes.isPresent()) { - // Encountered a data source which didn't support cache yet - log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); - return new byte[]{}; - } - keyBuilder.appendByteArray(bytes.get()); - keyBuilder.appendString(clause.getCondition().getOriginalExpression()); - keyBuilder.appendString(clause.getPrefix()); - keyBuilder.appendString(clause.getJoinType().name()); - } - return keyBuilder.build(); - } } diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index c36fc6b424d8..098a50374b1f 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -23,7 +23,6 @@ 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; @@ -109,12 +108,6 @@ public Function createSegmentMapFunction( return Function.identity(); } - @Override - public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) - { - return new byte[]{}; - } - @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 807ea4bc2aa5..5cff91e3ed1c 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -25,7 +25,6 @@ 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; @@ -101,12 +100,6 @@ public Function createSegmentMapFunction( return Function.identity(); } - @Override - public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) - { - return new byte[]{}; - } - @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index ac89b0b45843..f547a1d263d2 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -24,7 +24,6 @@ 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; @@ -106,12 +105,6 @@ public Function createSegmentMapFunction( return Function.identity(); } - @Override - public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) - { - return new byte[]{}; - } - @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index a857897e7272..6fd36f1188fc 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -26,7 +26,6 @@ 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; @@ -118,12 +117,6 @@ public Function createSegmentMapFunction( return Function.identity(); } - @Override - public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) - { - return new byte[]{}; - } - @Override public boolean equals(Object o) { diff --git a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java index 2cdac7efadd6..c1fe72d18eee 100644 --- a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java @@ -22,22 +22,12 @@ 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; @@ -45,9 +35,7 @@ import org.junit.rules.ExpectedException; import org.mockito.Mockito; -import java.util.Arrays; import java.util.Collections; -import java.util.Optional; public class JoinDataSourceTest @@ -192,33 +180,6 @@ public void test_serde() throws Exception 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() { diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java index e5eb8be18dc4..198c25782fce 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java @@ -124,6 +124,228 @@ 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 cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + + Assert.assertFalse(cacheKey.isPresent()); + } + + @Test + public void test_computeJoinDataSourceCacheKey_cachingUnsupported() + { + PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); + DataSource dataSource = new LookupDataSource("lookup"); + PreJoinableClause clause2 = makePreJoinableClause(dataSource, "x == \"h.x\"", "h.", JoinType.LEFT); + DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes(); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes(); + EasyMock.replay(analysis); + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + Optional cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + + Assert.assertFalse(cacheKey.isPresent()); + } + + @Test + public void test_computeJoinDataSourceCacheKey_usableClauses() + { + + PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); + PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"h.x\"", "h."); + DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes(); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + EasyMock.replay(analysis); + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + Optional cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + + Assert.assertTrue(cacheKey.isPresent()); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithExpression() + { + DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "y == \"j.y\"", "j."); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); + EasyMock.replay(analysis); + + Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey1.isPresent()); + Assert.assertNotEquals(0, cacheKey1.get().length); + + PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); + EasyMock.reset(analysis); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + EasyMock.replay(analysis); + Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey2.isPresent()); + + Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinType() + { + DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.LEFT); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); + EasyMock.replay(analysis); + + Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey1.isPresent()); + Assert.assertNotEquals(0, cacheKey1.get().length); + + PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.INNER); + EasyMock.reset(analysis); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + EasyMock.replay(analysis); + Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey2.isPresent()); + + Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithPrefix() + { + DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); + EasyMock.replay(analysis); + + Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey1.isPresent()); + Assert.assertNotEquals(0, cacheKey1.get().length); + + PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "xy"); + EasyMock.reset(analysis); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + EasyMock.replay(analysis); + Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey2.isPresent()); + + Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithBaseFilter() + { + DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes(); + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); + EasyMock.replay(analysis); + + Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey1.isPresent()); + Assert.assertNotEquals(0, cacheKey1.get().length); + + PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); + EasyMock.reset(analysis); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(FalseDimFilter.instance())).anyTimes(); + EasyMock.replay(analysis); + Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey2.isPresent()); + + Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinable() + { + DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); + EasyMock.replay(analysis); + + Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey1.isPresent()); + Assert.assertNotEquals(0, cacheKey1.get().length); + + PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"j.x\"", "j."); + EasyMock.reset(analysis); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + + EasyMock.replay(analysis); + Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey2.isPresent()); + + Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); + } + + @Test + public void test_computeJoinDataSourceCacheKey_sameKeyForSameJoin() + { + DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + EasyMock.replay(analysis); + + Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey1.isPresent()); + Assert.assertNotEquals(0, cacheKey1.get().length); + + PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); + EasyMock.reset(analysis); + EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); + EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); + EasyMock.replay(analysis); + Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); + Assert.assertTrue(cacheKey2.isPresent()); + + Assert.assertArrayEquals(cacheKey1.get(), cacheKey2.get()); + } @Test public void test_checkClausePrefixesForDuplicatesAndShadowing_noConflicts() @@ -602,4 +824,53 @@ public void test_convertJoinsToFilters_partialConvertJoinsDependedOnByLaterJoins conversion ); } + + private PreJoinableClause makeGlobalPreJoinableClause(String tableName, String expression, String prefix) + { + return makeGlobalPreJoinableClause(tableName, expression, prefix, JoinType.LEFT); + } + + private PreJoinableClause makeGlobalPreJoinableClause( + String tableName, + String expression, + String prefix, + JoinType joinType + ) + { + GlobalTableDataSource dataSource = new GlobalTableDataSource(tableName); + return makePreJoinableClause(dataSource, expression, prefix, joinType); + } + + private PreJoinableClause makePreJoinableClause( + DataSource dataSource, + String expression, + String prefix, + JoinType joinType + ) + { + JoinConditionAnalysis conditionAnalysis = JoinConditionAnalysis.forExpression( + expression, + prefix, + ExprMacroTable.nil() + ); + return new PreJoinableClause( + prefix, + dataSource, + joinType, + conditionAnalysis + ); + } + + private static class JoinableFactoryWithCacheKey extends NoopJoinableFactory + { + @Override + public Optional computeJoinCacheKey(DataSource dataSource, JoinConditionAnalysis condition) + { + if (dataSource.isCacheable(false) && condition.canHashJoin()) { + String tableName = Iterators.getOnlyElement(dataSource.getTableNames().iterator()); + return Optional.of(StringUtils.toUtf8(tableName)); + } + return Optional.empty(); + } + } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java index e134a7a9f8c3..c3cdf6b7764f 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java @@ -21,7 +21,6 @@ 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; @@ -78,10 +77,4 @@ public Function createSegmentMapFunction( { return Function.identity(); } - - @Override - public byte[] getCacheKey(DataSourceAnalysis dataSourceAnalysis) - { - return new byte[]{}; - } } diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index c3ab1b4f05cf..69952e22ce3a 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -837,7 +837,8 @@ private byte[] computeQueryCacheKeyWithJoin() { Preconditions.checkNotNull(strategy, "strategy cannot be null"); if (dataSourceAnalysis.isJoin()) { - byte[] joinDataSourceCacheKey = dataSourceAnalysis.getDataSource().getCacheKey(dataSourceAnalysis); + byte[] joinDataSourceCacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis) + .orElse(null); if (null == joinDataSourceCacheKey) { return null; // A join operation that does not support caching } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 1d5f13d8701f..3456a9e6ec92 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -178,7 +178,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final // We compute the join cache key here itself so it doesn't need to be re-computed for every segment final Optional cacheKeyPrefix = analysis.isJoin() - ? Optional.of(analysis.getDataSource().getCacheKey(analysis)) + ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) : Optional.of(StringUtils.EMPTY_BYTES); Iterable> perSegmentRunners = Iterables.transform( diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index 2681fdc8f43c..333525ee5810 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -198,7 +198,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable cacheKeyPrefix = analysis.isJoin() - ? Optional.of(analysis.getDataSource().getCacheKey(analysis)) + ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) : Optional.of(StringUtils.EMPTY_BYTES); final FunctionalIterable> queryRunners = FunctionalIterable diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index 81b05dfd3685..f685f889e213 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -28,7 +28,6 @@ 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; @@ -133,12 +132,6 @@ public Function createSegmentMapFunction( return Function.identity(); } - @Override - public byte[] getCacheKey(final DataSourceAnalysis dataSourceAnalysis) - { - return new byte[]{}; - } - @Override public boolean equals(Object o) { From e7bf43488b514ad23cfc17b87136c7a56a88feeb Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Thu, 6 Oct 2022 10:24:01 -0700 Subject: [PATCH 17/31] temp --- .../druid/benchmark/query/SqlBenchmark.java | 37 +++++++++++++++---- .../org/apache/druid/query/DataSource.java | 2 + .../apache/druid/query/QueryDataSource.java | 20 ++++++++++ .../calcite/external/ExternalDataSource.java | 17 +++++++++ 4 files changed, 68 insertions(+), 8 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 6f390ef345e6..cf8f5ec3ea08 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -26,7 +26,6 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QueryContext; @@ -76,7 +75,6 @@ import org.openjdk.jmh.infra.Blackhole; import javax.annotation.Nullable; - import java.util.HashSet; import java.util.List; import java.util.Map; @@ -88,8 +86,8 @@ */ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 5) -@Measurement(iterations = 15) +@Warmup(iterations = 3) +@Measurement(iterations = 10) public class SqlBenchmark { static { @@ -401,10 +399,33 @@ public class SqlBenchmark @Param({"force"}) private String vectorize; - @Param({"0", "10", "18"}) + @Param({ + "0", + "1", + "2", + "3", + "4", + "5", + "6", + "7", + "8", + "9", + "10", + "11", + "12", + "13", + "14", + "15", + "16", + "17", + "18", + "19", + "20" + }) private String query; - @Param({STORAGE_MMAP, STORAGE_FRAME_ROW, STORAGE_FRAME_COLUMNAR}) + //@Param({STORAGE_MMAP, STORAGE_FRAME_ROW, STORAGE_FRAME_COLUMNAR}) + @Param({STORAGE_MMAP}) private String storageType; private SqlEngine engine; @@ -507,7 +528,7 @@ public void tearDown() throws Exception closer.close(); } - @Benchmark + /*@Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void querySql(Blackhole blackhole) throws Exception @@ -523,7 +544,7 @@ public void querySql(Blackhole blackhole) throws Exception final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in); blackhole.consume(lastRow); } - } + }*/ @Benchmark @BenchmarkMode(Mode.AverageTime) diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index aad3ac1ef278..f95f32b894a9 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -99,4 +99,6 @@ public interface DataSource * @return the segment function */ Function createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc); + + Query withBaseDataSource(final Query query, final DataSource newBaseDataSource); } diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 5cff91e3ed1c..b826ac7a4d76 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -25,6 +25,8 @@ import com.google.common.base.Preconditions; 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.Collections; @@ -100,6 +102,24 @@ public Function createSegmentMapFunction( return Function.identity(); } + @Override + public Query withBaseDataSource(Query query, DataSource newBaseDataSource) + { + final Query retVal; + final DataSource theDataSource = query.getDataSource(); + final Query subQuery = ((QueryDataSource) theDataSource).getQuery(); + retVal = query.withDataSource(subQuery.getDataSource()); + + // 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; + } + @Override public String toString() { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index f685f889e213..921edba71070 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -26,8 +26,10 @@ 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.java.util.common.ISE; 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 +134,21 @@ public Function createSegmentMapFunction( return Function.identity(); } + @Override + public Query withBaseDataSource(Query query, DataSource newBaseDataSource) + { + final Query retVal; + retVal = query.withDataSource(newBaseDataSource); + // 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; + } + @Override public boolean equals(Object o) { From af696a3c22d0fcd4f534b21d2070af052fbf4eea Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Thu, 6 Oct 2022 12:08:23 -0700 Subject: [PATCH 18/31] Temp discussions --- .../src/main/java/org/apache/druid/query/DataSource.java | 4 ++++ .../java/org/apache/druid/query/InlineDataSource.java | 6 ++++++ .../src/main/java/org/apache/druid/query/Queries.java | 4 ++++ .../java/org/apache/druid/query/QueryDataSource.java | 9 +++++++-- .../apache/druid/query/planning/DataSourceAnalysis.java | 4 ++++ 5 files changed, 25 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index f95f32b894a9..50a5295b1ef9 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -101,4 +101,8 @@ public interface DataSource Function createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc); Query withBaseDataSource(final Query query, final DataSource newBaseDataSource); + + DataSource withBaseDataSource(final DataSource newSource); + + DataSourceAnalysis getAnalysis(Query query); } diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index dc7606e1be27..2cdaa0c3a2c5 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -243,6 +243,12 @@ public Function createSegmentMapFunction( return Function.identity(); } + @Override + public Query withBaseDataSource(Query query, DataSource newBaseDataSource) + { + 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. diff --git a/processing/src/main/java/org/apache/druid/query/Queries.java b/processing/src/main/java/org/apache/druid/query/Queries.java index 63ebe2b6767b..6fc604accc35 100644 --- a/processing/src/main/java/org/apache/druid/query/Queries.java +++ b/processing/src/main/java/org/apache/druid/query/Queries.java @@ -189,6 +189,10 @@ public static Query withSpecificSegments(final Query query, final List */ public static Query withBaseDataSource(final Query query, final DataSource newBaseDataSource) { + if (true) { + return query.withDataSource(query.getDataSource().withBaseDataSource(newBaseDataSource)); + } + final Query retVal; /* diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index b826ac7a4d76..59565264120c 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -102,11 +102,16 @@ public Function createSegmentMapFunction( return Function.identity(); } + public DataSource withBaseDataSource(DataSource newSource) { + return new QueryDataSource(query.withDataSource(query.getDataSource()).withDataSource(newSource)); + } + @Override - public Query withBaseDataSource(Query query, DataSource newBaseDataSource) + public Query + withBaseDataSource(Query query, DataSource newBaseDataSource) { final Query retVal; - final DataSource theDataSource = query.getDataSource(); + final DataSource theDataSource = this.query.getDataSource().withBaseDataSource(); final Query subQuery = ((QueryDataSource) theDataSource).getQuery(); retVal = query.withDataSource(subQuery.getDataSource()); diff --git a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java index c329e3a57089..16d5c84362a6 100644 --- a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java +++ b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java @@ -177,6 +177,10 @@ public DataSource getDataSource() return dataSource; } + public DataSourceAnalysis withDataSource(DataSource dataSource) { + // TODO: + } + /** * Returns the base (bottom-leftmost) datasource. */ From 79d2db3424314e560adeee7a9a8ba86c1e44a2d1 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Tue, 11 Oct 2022 14:37:52 -0700 Subject: [PATCH 19/31] Refactoring temp --- .../org/apache/druid/query/DataSource.java | 6 ++--- .../java/org/apache/druid/query/Queries.java | 2 +- .../apache/druid/query/QueryDataSource.java | 22 +++---------------- .../apache/druid/query/TableDataSource.java | 6 +++++ 4 files changed, 12 insertions(+), 24 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 50a5295b1ef9..1fddc765350e 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -21,6 +21,7 @@ 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; @@ -100,9 +101,6 @@ public interface DataSource */ Function createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc); - Query withBaseDataSource(final Query query, final DataSource newBaseDataSource); + DataSource withUpdatedDataSource(final DataSource newSource); - DataSource withBaseDataSource(final DataSource newSource); - - DataSourceAnalysis getAnalysis(Query query); } diff --git a/processing/src/main/java/org/apache/druid/query/Queries.java b/processing/src/main/java/org/apache/druid/query/Queries.java index 6fc604accc35..eefe3cb890b2 100644 --- a/processing/src/main/java/org/apache/druid/query/Queries.java +++ b/processing/src/main/java/org/apache/druid/query/Queries.java @@ -190,7 +190,7 @@ public static Query withSpecificSegments(final Query query, final List public static Query withBaseDataSource(final Query query, final DataSource newBaseDataSource) { if (true) { - return query.withDataSource(query.getDataSource().withBaseDataSource(newBaseDataSource)); + return query.withDataSource(query.getDataSource().withUpdatedDataSource(newBaseDataSource)); } final Query retVal; diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 59565264120c..4c28083ed93e 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -102,29 +102,13 @@ public Function createSegmentMapFunction( return Function.identity(); } - public DataSource withBaseDataSource(DataSource newSource) { - return new QueryDataSource(query.withDataSource(query.getDataSource()).withDataSource(newSource)); - } - @Override - public Query - withBaseDataSource(Query query, DataSource newBaseDataSource) + public DataSource withUpdatedDataSource(DataSource newSource) { - final Query retVal; - final DataSource theDataSource = this.query.getDataSource().withBaseDataSource(); - final Query subQuery = ((QueryDataSource) theDataSource).getQuery(); - retVal = query.withDataSource(subQuery.getDataSource()); - - // 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 new QueryDataSource(query.withDataSource(query.getDataSource()).withDataSource(newSource)); } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index f547a1d263d2..1fba96831fa2 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -105,6 +105,12 @@ public Function createSegmentMapFunction( return Function.identity(); } + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + @Override public String toString() { From 38306f607e4788f327fbbd6eb554e46aaead9043 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Wed, 12 Oct 2022 12:27:09 -0700 Subject: [PATCH 20/31] Refatoring the query rewrite to refer to a datasource --- .../msq/querykit/InputNumberDataSource.java | 6 +++ .../org/apache/druid/query/DataSource.java | 12 ++++-- .../apache/druid/query/InlineDataSource.java | 4 +- .../apache/druid/query/JoinDataSource.java | 22 ++++++++++ .../apache/druid/query/LookupDataSource.java | 6 +++ .../java/org/apache/druid/query/Queries.java | 41 +------------------ .../apache/druid/query/QueryDataSource.java | 4 +- .../apache/druid/query/UnionDataSource.java | 6 +++ .../query/planning/DataSourceAnalysis.java | 4 -- .../druid/segment/join/NoopDataSource.java | 6 +++ .../calcite/external/ExternalDataSource.java | 15 +------ 11 files changed, 61 insertions(+), 65 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index 483344de3944..a79884d7d1f3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -103,6 +103,12 @@ public Function createSegmentMapFunction( return analysis.getDataSource().createSegmentMapFunction(query, new AtomicLong()); } + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + @JsonProperty public int getInputNumber() { diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 1fddc765350e..479bb713032d 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -87,8 +87,8 @@ 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(); @@ -101,6 +101,12 @@ public interface DataSource */ Function createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc); - DataSource withUpdatedDataSource(final DataSource newSource); + /** + * 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); } diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index 2cdaa0c3a2c5..64775de108b3 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -244,9 +244,9 @@ public Function createSegmentMapFunction( } @Override - public Query withBaseDataSource(Query query, DataSource newBaseDataSource) + public DataSource withUpdatedDataSource(DataSource newSource) { - return null; + return newSource; } /** diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index a3dda5ea6646..24875ac2dbad 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -425,4 +425,26 @@ public Function createSegmentMapFunction( ); return segmentMapFn; } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(this); + 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; + } } diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index 098a50374b1f..12b417e382d2 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -108,6 +108,12 @@ public Function createSegmentMapFunction( return Function.identity(); } + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/Queries.java b/processing/src/main/java/org/apache/druid/query/Queries.java index eefe3cb890b2..c7b771744cf9 100644 --- a/processing/src/main/java/org/apache/druid/query/Queries.java +++ b/processing/src/main/java/org/apache/druid/query/Queries.java @@ -29,7 +29,6 @@ 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; @@ -189,46 +188,8 @@ public static Query withSpecificSegments(final Query query, final List */ public static Query withBaseDataSource(final Query query, final DataSource newBaseDataSource) { - if (true) { - return query.withDataSource(query.getDataSource().withUpdatedDataSource(newBaseDataSource)); - } - final Query retVal; - - /* - * Currently, this method is implemented in terms of a static walk doing a bunch of instanceof checks. - * We should likely look into moving this functionality into the DataSource object itself so that they - * can walk and create new objects on their own. This will be necessary as we expand the set of DataSources - * that do actual work, as each of them will need to show up in this if/then waterfall. - */ - final DataSource theDataSource = query.getDataSource(); - if (theDataSource instanceof QueryDataSource) { - final Query subQuery = ((QueryDataSource) theDataSource).getQuery(); - retVal = query.withDataSource(new QueryDataSource(withBaseDataSource(subQuery, newBaseDataSource))); - } else if (theDataSource instanceof JoinDataSource) { - JoinDataSource joinDataSource = (JoinDataSource) theDataSource; - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(theDataSource); - - 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, - joinDataSource.getJoinableFactoryWrapper() - ); - joinBaseFilter = null; - } - - retVal = query.withDataSource(current); - } else { - retVal = query.withDataSource(newBaseDataSource); - } + retVal = query.withDataSource(query.getDataSource().withUpdatedDataSource(newBaseDataSource)); // Verify postconditions, just in case. final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(retVal.getDataSource()); diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index 4c28083ed93e..d07bf65216ee 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -25,8 +25,6 @@ import com.google.common.base.Preconditions; 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.Collections; @@ -105,7 +103,7 @@ public Function createSegmentMapFunction( @Override public DataSource withUpdatedDataSource(DataSource newSource) { - return new QueryDataSource(query.withDataSource(query.getDataSource()).withDataSource(newSource)); + return new QueryDataSource(query.withDataSource(query.getDataSource().withUpdatedDataSource(newSource))); } diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index 6fd36f1188fc..df8dfbcfb342 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -117,6 +117,12 @@ public Function createSegmentMapFunction( return Function.identity(); } + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java index 16d5c84362a6..c329e3a57089 100644 --- a/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java +++ b/processing/src/main/java/org/apache/druid/query/planning/DataSourceAnalysis.java @@ -177,10 +177,6 @@ public DataSource getDataSource() return dataSource; } - public DataSourceAnalysis withDataSource(DataSource dataSource) { - // TODO: - } - /** * Returns the base (bottom-leftmost) datasource. */ diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java index c3cdf6b7764f..14c969520bab 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java @@ -77,4 +77,10 @@ public Function createSegmentMapFunction( { return Function.identity(); } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index 921edba71070..c555e0d0dd23 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -26,10 +26,8 @@ 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.java.util.common.ISE; 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; @@ -135,18 +133,9 @@ public Function createSegmentMapFunction( } @Override - public Query withBaseDataSource(Query query, DataSource newBaseDataSource) + public DataSource withUpdatedDataSource(DataSource newSource) { - final Query retVal; - retVal = query.withDataSource(newBaseDataSource); - // 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 newSource; } @Override From a10fa76d0a4a5fa575e09985028d12d1cf4511a6 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Thu, 13 Oct 2022 16:50:02 -0700 Subject: [PATCH 21/31] Refactoring getCacheKey by moving it inside data source --- .../msq/querykit/InputNumberDataSource.java | 6 + .../org/apache/druid/query/DataSource.java | 17 ++ .../apache/druid/query/InlineDataSource.java | 6 + .../apache/druid/query/JoinDataSource.java | 36 ++- .../apache/druid/query/LookupDataSource.java | 6 + .../java/org/apache/druid/query/Queries.java | 12 +- .../apache/druid/query/QueryDataSource.java | 6 + .../apache/druid/query/TableDataSource.java | 6 + .../apache/druid/query/UnionDataSource.java | 6 + .../segment/join/JoinableFactoryWrapper.java | 50 +--- .../druid/query/JoinDataSourceTest.java | 280 ++++++++++++++++- .../join/JoinableFactoryWrapperTest.java | 283 ------------------ .../druid/segment/join/NoopDataSource.java | 6 + .../druid/client/CachingClusteredClient.java | 27 +- .../appenderator/SinkQuerySegmentWalker.java | 2 +- .../server/coordination/ServerManager.java | 2 +- ...ingClusteredClientCacheKeyManagerTest.java | 63 ++-- .../calcite/external/ExternalDataSource.java | 6 + 18 files changed, 409 insertions(+), 411 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index a79884d7d1f3..041900315c96 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -109,6 +109,12 @@ public DataSource withUpdatedDataSource(DataSource newSource) return newSource; } + @Override + public byte[] getCacheKey() + { + return null; + } + @JsonProperty public int getInputNumber() { diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 479bb713032d..2c3cc81098a4 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -21,7 +21,9 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.planning.PreJoinableClause; import org.apache.druid.segment.SegmentReference; import java.util.List; @@ -109,4 +111,19 @@ public interface DataSource */ 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 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. + * + * @return the 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 + */ + byte[] getCacheKey(); + } diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index 64775de108b3..040d6885782a 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -249,6 +249,12 @@ 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. diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 24875ac2dbad..544fd8045dc5 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -33,7 +33,9 @@ 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.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; @@ -91,6 +94,8 @@ public class JoinDataSource implements DataSource 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, @@ -114,6 +119,7 @@ private JoinDataSource( ); this.leftFilter = leftFilter; this.joinableFactoryWrapper = joinableFactoryWrapper; + this.analysis = DataSourceAnalysis.forDataSource(this); } /** @@ -415,6 +421,7 @@ public Function createSegmentMapFunction( AtomicLong cpuTimeAccumulator ) { + final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); final Function segmentMapFn = createSegmentMapFn( @@ -429,7 +436,6 @@ public Function createSegmentMapFunction( @Override public DataSource withUpdatedDataSource(DataSource newSource) { - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(this); DataSource current = newSource; DimFilter joinBaseFilter = analysis.getJoinBaseTableFilter().orElse(null); @@ -447,4 +453,32 @@ public DataSource withUpdatedDataSource(DataSource newSource) } return current; } + + @Override + public byte[] getCacheKey() + { + final List clauses = analysis.getPreJoinableClauses(); + if (clauses.isEmpty()) { + throw new IAE("No join clauses to build the cache key for data source [%s]", analysis.getDataSource()); + } + + final CacheKeyBuilder keyBuilder; + keyBuilder = new CacheKeyBuilder(JoinableFactoryWrapper.JOIN_OPERATION); + if (analysis.getJoinBaseTableFilter().isPresent()) { + keyBuilder.appendCacheable(analysis.getJoinBaseTableFilter().get()); + } + for (PreJoinableClause clause : clauses) { + Optional bytes = joinableFactoryWrapper.getJoinableFactory().computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); + if (!bytes.isPresent()) { + // Encountered a data source which didn't support cache yet + log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); + return new byte[]{}; + } + keyBuilder.appendByteArray(bytes.get()); + keyBuilder.appendString(clause.getCondition().getOriginalExpression()); + keyBuilder.appendString(clause.getPrefix()); + keyBuilder.appendString(clause.getJoinType().name()); + } + return keyBuilder.build(); + } } diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index 12b417e382d2..0b40aea020d9 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -114,6 +114,12 @@ public DataSource withUpdatedDataSource(DataSource newSource) return newSource; } + @Override + public byte[] getCacheKey() + { + return null; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/Queries.java b/processing/src/main/java/org/apache/druid/query/Queries.java index c7b771744cf9..2a34a6fabf49 100644 --- a/processing/src/main/java/org/apache/druid/query/Queries.java +++ b/processing/src/main/java/org/apache/druid/query/Queries.java @@ -188,17 +188,7 @@ public static Query withSpecificSegments(final Query query, final List */ public static Query withBaseDataSource(final Query query, final DataSource newBaseDataSource) { - final Query retVal; - retVal = query.withDataSource(query.getDataSource().withUpdatedDataSource(newBaseDataSource)); - - // Verify postconditions, just in case. - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(retVal.getDataSource()); - - if (!newBaseDataSource.equals(analysis.getBaseDataSource())) { - throw new ISE("Unable to replace base dataSource"); - } - - return retVal; + return query.withDataSource(query.getDataSource().withUpdatedDataSource(newBaseDataSource)); } /** diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index d07bf65216ee..c495d0832f24 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -106,6 +106,12 @@ public DataSource withUpdatedDataSource(DataSource newSource) return new QueryDataSource(query.withDataSource(query.getDataSource().withUpdatedDataSource(newSource))); } + @Override + public byte[] getCacheKey() + { + return null; + } + @Override public String toString() diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index 1fba96831fa2..6182b7039451 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -111,6 +111,12 @@ public DataSource withUpdatedDataSource(DataSource newSource) return newSource; } + @Override + public byte[] getCacheKey() + { + return null; + } + @Override public String toString() { diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index df8dfbcfb342..50802dc72524 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -123,6 +123,12 @@ public DataSource withUpdatedDataSource(DataSource newSource) return newSource; } + @Override + public byte[] getCacheKey() + { + return null; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java index 3c7dd2c06207..74a76020b92a 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java @@ -25,21 +25,16 @@ import com.google.common.collect.Multiset; import com.google.common.collect.Sets; import com.google.inject.Inject; -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.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.filter.FalseFilter; import org.apache.druid.segment.filter.Filters; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -49,7 +44,7 @@ public class JoinableFactoryWrapper { - private static final byte JOIN_OPERATION = 0x1; + public static final byte JOIN_OPERATION = 0x1; private static final Logger log = new Logger(JoinableFactoryWrapper.class); private final JoinableFactory joinableFactory; @@ -197,49 +192,6 @@ public JoinableFactory getJoinableFactory() return joinableFactory; } - /** - * Compute a cache key prefix for a join data source. This includes the data sources that participate in the RHS of a - * join as well as any query specific constructs associated with join data source such as base table filter. This key prefix - * can be used in segment level cache or result level cache. The function can return following wrapped in an - * Optional - * - Non-empty byte array - If there is join datasource involved and caching is possible. The result includes - * join condition expression, join type and cache key returned by joinable factory for each {@link PreJoinableClause} - * - NULL - There is a join but caching is not possible. It may happen if one of the participating datasource - * in the JOIN is not cacheable. - * - * @param dataSourceAnalysis for the join datasource - * @return the optional cache key to be used as part of query cache key - * @throws {@link IAE} if this operation is called on a non-join data source - */ - public Optional computeJoinDataSourceCacheKey( - final DataSourceAnalysis dataSourceAnalysis - ) - { - final List clauses = dataSourceAnalysis.getPreJoinableClauses(); - if (clauses.isEmpty()) { - throw new IAE("No join clauses to build the cache key for data source [%s]", dataSourceAnalysis.getDataSource()); - } - - final CacheKeyBuilder keyBuilder; - keyBuilder = new CacheKeyBuilder(JOIN_OPERATION); - if (dataSourceAnalysis.getJoinBaseTableFilter().isPresent()) { - keyBuilder.appendCacheable(dataSourceAnalysis.getJoinBaseTableFilter().get()); - } - for (PreJoinableClause clause : clauses) { - Optional bytes = joinableFactory.computeJoinCacheKey(clause.getDataSource(), clause.getCondition()); - if (!bytes.isPresent()) { - // Encountered a data source which didn't support cache yet - log.debug("skipping caching for join since [%s] does not support caching", clause.getDataSource()); - return Optional.empty(); - } - keyBuilder.appendByteArray(bytes.get()); - keyBuilder.appendString(clause.getCondition().getOriginalExpression()); - keyBuilder.appendString(clause.getPrefix()); - keyBuilder.appendString(clause.getJoinType().name()); - } - return Optional.of(keyBuilder.build()); - } - /** * Encapsulates the conversion which happened for a joinable clause. * convertedFilter represents the filter which got generated from the conversion. diff --git a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java index c1fe72d18eee..814250e1d254 100644 --- a/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/JoinDataSourceTest.java @@ -22,12 +22,18 @@ 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; @@ -35,11 +41,17 @@ import org.junit.rules.ExpectedException; import org.mockito.Mockito; +import java.util.Arrays; import java.util.Collections; +import java.util.Optional; public class JoinDataSourceTest { + + 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"); @@ -153,7 +165,7 @@ public void test_equals() EqualsVerifier.forClass(JoinDataSource.class) .usingGetClass() .withNonnullFields("left", "right", "rightPrefix", "conditionAnalysis", "joinType") - .withIgnoredFields("joinableFactoryWrapper") + .withIgnoredFields("joinableFactoryWrapper", "analysis") .verify(); } @@ -228,4 +240,270 @@ public void testVirtualColumnCandidates() ); Assert.assertEquals(dataSource.getVirtualColumnCandidates(), ImmutableSet.of("x")); } + + @Test + public void test_computeJoinDataSourceCacheKey_noHashJoin() + { + JoinDataSource dataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + NOOP_JOINABLE_FACTORY_WRAPPER + ); + + Optional cacheKey = Optional.ofNullable(dataSource.getCacheKey()); + + Assert.assertTrue(cacheKey.isPresent()); + } + + @Test + public void test_computeJoinDataSourceCacheKey_sameKeyForSameJoin() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertTrue(Arrays.equals(cacheKey1, cacheKey2)); + + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithTables() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table11"), + new TableDataSource("table12"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithExpressions() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "y == \"j.y\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinType() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.INNER, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithPrefix() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "ab.", + "x == \"ab.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_keyChangesWithBaseFilter() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + NullHandling.initializeForTests(); + final InDimFilter expectedInDimFilter = new InDimFilter("dimTest", Arrays.asList("good", "bad"), null); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + expectedInDimFilter, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + JoinDataSource joinDataSource1 = JoinDataSource.create( + new TableDataSource("table1"), + new TableDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + byte[] cacheKey2 = joinDataSource1.getCacheKey(); + + Assert.assertNotEquals(cacheKey1.length, 0); + Assert.assertNotEquals(cacheKey2.length, 0); + Assert.assertFalse(Arrays.equals(cacheKey1, cacheKey2)); + } + + @Test + public void test_computeJoinDataSourceCacheKey_cachingUnsupported() + { + JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); + + JoinDataSource joinDataSource = JoinDataSource.create( + new TableDataSource("table1"), + new LookupDataSource("table2"), + "j.", + "x == \"j.x\"", + JoinType.LEFT, + null, + ExprMacroTable.nil(), + joinableFactoryWrapper + ); + + byte[] cacheKey1 = joinDataSource.getCacheKey(); + Assert.assertEquals(cacheKey1.length, 0); + } + + + private static class JoinableFactoryWithCacheKey extends NoopJoinableFactory + { + @Override + public Optional computeJoinCacheKey(DataSource dataSource, JoinConditionAnalysis condition) + { + if (dataSource.isCacheable(false) && condition.canHashJoin()) { + String tableName = Iterators.getOnlyElement(dataSource.getTableNames().iterator()); + return Optional.of(StringUtils.toUtf8(tableName)); + } + return Optional.empty(); + } + } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java index 198c25782fce..3d8a01785a4e 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinableFactoryWrapperTest.java @@ -22,26 +22,17 @@ 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.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.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.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.FalseFilter; @@ -49,7 +40,6 @@ 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.easymock.EasyMock; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -60,7 +50,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -124,229 +113,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 cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - - Assert.assertFalse(cacheKey.isPresent()); - } - - @Test - public void test_computeJoinDataSourceCacheKey_cachingUnsupported() - { - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - DataSource dataSource = new LookupDataSource("lookup"); - PreJoinableClause clause2 = makePreJoinableClause(dataSource, "x == \"h.x\"", "h.", JoinType.LEFT); - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes(); - EasyMock.replay(analysis); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - Optional cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - - Assert.assertFalse(cacheKey.isPresent()); - } - - @Test - public void test_computeJoinDataSourceCacheKey_usableClauses() - { - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"h.x\"", "h."); - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Arrays.asList(clause1, clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - Optional cacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - - Assert.assertTrue(cacheKey.isPresent()); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithExpression() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "y == \"j.y\"", "j."); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinType() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.LEFT); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j.", JoinType.INNER); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithPrefix() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "xy"); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithBaseFilter() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(TrueDimFilter.instance())).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "abc == xyz", "ab"); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.of(FalseDimFilter.instance())).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_keyChangesWithJoinable() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_2", "x == \"j.x\"", "j."); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertFalse(Arrays.equals(cacheKey1.get(), cacheKey2.get())); - } - - @Test - public void test_computeJoinDataSourceCacheKey_sameKeyForSameJoin() - { - DataSourceAnalysis analysis = EasyMock.mock(DataSourceAnalysis.class); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(new JoinableFactoryWithCacheKey()); - - PreJoinableClause clause1 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause1)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - - Optional cacheKey1 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey1.isPresent()); - Assert.assertNotEquals(0, cacheKey1.get().length); - - PreJoinableClause clause2 = makeGlobalPreJoinableClause("dataSource_1", "x == \"j.x\"", "j."); - EasyMock.reset(analysis); - EasyMock.expect(analysis.getPreJoinableClauses()).andReturn(Collections.singletonList(clause2)).anyTimes(); - EasyMock.expect(analysis.getJoinBaseTableFilter()).andReturn(Optional.empty()).anyTimes(); - EasyMock.replay(analysis); - Optional cacheKey2 = joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis); - Assert.assertTrue(cacheKey2.isPresent()); - - Assert.assertArrayEquals(cacheKey1.get(), cacheKey2.get()); - } - @Test public void test_checkClausePrefixesForDuplicatesAndShadowing_noConflicts() { @@ -824,53 +590,4 @@ public void test_convertJoinsToFilters_partialConvertJoinsDependedOnByLaterJoins conversion ); } - - private PreJoinableClause makeGlobalPreJoinableClause(String tableName, String expression, String prefix) - { - return makeGlobalPreJoinableClause(tableName, expression, prefix, JoinType.LEFT); - } - - private PreJoinableClause makeGlobalPreJoinableClause( - String tableName, - String expression, - String prefix, - JoinType joinType - ) - { - GlobalTableDataSource dataSource = new GlobalTableDataSource(tableName); - return makePreJoinableClause(dataSource, expression, prefix, joinType); - } - - private PreJoinableClause makePreJoinableClause( - DataSource dataSource, - String expression, - String prefix, - JoinType joinType - ) - { - JoinConditionAnalysis conditionAnalysis = JoinConditionAnalysis.forExpression( - expression, - prefix, - ExprMacroTable.nil() - ); - return new PreJoinableClause( - prefix, - dataSource, - joinType, - conditionAnalysis - ); - } - - private static class JoinableFactoryWithCacheKey extends NoopJoinableFactory - { - @Override - public Optional computeJoinCacheKey(DataSource dataSource, JoinConditionAnalysis condition) - { - if (dataSource.isCacheable(false) && condition.canHashJoin()) { - String tableName = Iterators.getOnlyElement(dataSource.getTableNames().iterator()); - return Optional.of(StringUtils.toUtf8(tableName)); - } - return Optional.empty(); - } - } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java index 14c969520bab..b6eb168b6246 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/join/NoopDataSource.java @@ -83,4 +83,10 @@ public DataSource withUpdatedDataSource(DataSource newSource) { return newSource; } + + @Override + public byte[] getCacheKey() + { + return new byte[]{}; + } } diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 69952e22ce3a..095afe2a761d 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -294,9 +294,7 @@ private class SpecificQueryRunnable query, strategy, useCache, - populateCache, - dataSourceAnalysis, - joinableFactoryWrapper + populateCache ); } @@ -757,24 +755,18 @@ static class CacheKeyManager { private final Query query; private final CacheStrategy> strategy; - private final DataSourceAnalysis dataSourceAnalysis; - private final JoinableFactoryWrapper joinableFactoryWrapper; private final boolean isSegmentLevelCachingEnable; CacheKeyManager( final Query query, final CacheStrategy> strategy, final boolean useCache, - final boolean populateCache, - final DataSourceAnalysis dataSourceAnalysis, - final JoinableFactoryWrapper joinableFactoryWrapper + final boolean populateCache ) { this.query = query; this.strategy = strategy; - this.dataSourceAnalysis = dataSourceAnalysis; - this.joinableFactoryWrapper = joinableFactoryWrapper; this.isSegmentLevelCachingEnable = ((populateCache || useCache) && !QueryContexts.isBySegment(query)); // explicit bySegment queries are never cached @@ -836,15 +828,14 @@ String computeResultLevelCachingEtag( private byte[] computeQueryCacheKeyWithJoin() { Preconditions.checkNotNull(strategy, "strategy cannot be null"); - if (dataSourceAnalysis.isJoin()) { - byte[] joinDataSourceCacheKey = joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis) - .orElse(null); - if (null == joinDataSourceCacheKey) { - return null; // A join operation that does not support caching - } - return Bytes.concat(joinDataSourceCacheKey, strategy.computeCacheKey(query)); + byte[] dataSourceCacheKey = query.getDataSource().getCacheKey(); + if (null == dataSourceCacheKey) { + return null; + } else if (dataSourceCacheKey.length > 0) { + return Bytes.concat(dataSourceCacheKey, strategy.computeCacheKey(query)); + } else { + return strategy.computeCacheKey(query); } - return strategy.computeCacheKey(query); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 3456a9e6ec92..f08e14076044 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -178,7 +178,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final // We compute the join cache key here itself so it doesn't need to be re-computed for every segment final Optional cacheKeyPrefix = analysis.isJoin() - ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) + ? Optional.ofNullable(query.getDataSource().getCacheKey()) : Optional.of(StringUtils.EMPTY_BYTES); Iterable> perSegmentRunners = Iterables.transform( diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index 333525ee5810..d41b0cfc4287 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -198,7 +198,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable cacheKeyPrefix = analysis.isJoin() - ? joinableFactoryWrapper.computeJoinDataSourceCacheKey(analysis) + ? Optional.ofNullable(query.getDataSource().getCacheKey()) : Optional.of(StringUtils.EMPTY_BYTES); final FunctionalIterable> queryRunners = FunctionalIterable diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientCacheKeyManagerTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientCacheKeyManagerTest.java index c49b02c3a081..b159c8c799fc 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientCacheKeyManagerTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientCacheKeyManagerTest.java @@ -24,10 +24,12 @@ 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.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; @@ -40,7 +42,6 @@ import org.junit.Test; import org.junit.runner.RunWith; -import java.util.Optional; import java.util.Set; import static org.easymock.EasyMock.expect; @@ -58,6 +59,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}; @@ -137,7 +140,7 @@ public void testComputeEtag_DifferentQueryCacheKey() @Test public void testComputeEtag_nonJoinDataSource() { - expect(dataSourceAnalysis.isJoin()).andReturn(false); + expect(query.getDataSource()).andReturn(new NoopDataSource()); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); Set selectors = ImmutableSet.of( @@ -159,8 +162,8 @@ public void testComputeEtag_nonJoinDataSource() @Test public void testComputeEtag_joinWithUnsupportedCaching() { - expect(dataSourceAnalysis.isJoin()).andReturn(true); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.empty()); + expect(query.getDataSource()).andReturn(dataSource); + expect(dataSource.getCacheKey()).andReturn(null); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); Set selectors = ImmutableSet.of( @@ -171,38 +174,13 @@ public void testComputeEtag_joinWithUnsupportedCaching() Assert.assertNull(actual); } - @Test - public void testComputeEtag_joinWithSupportedCaching() - { - expect(dataSourceAnalysis.isJoin()).andReturn(true).anyTimes(); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(JOIN_KEY)); - replayAll(); - CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); - Set selectors = ImmutableSet.of( - makeHistoricalServerSelector(1), - makeHistoricalServerSelector(1) - ); - String actual1 = keyManager.computeResultLevelCachingEtag(selectors, null); - Assert.assertNotNull(actual1); - - reset(joinableFactoryWrapper); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(new byte[]{9})); - replay(joinableFactoryWrapper); - selectors = ImmutableSet.of( - makeHistoricalServerSelector(1), - makeHistoricalServerSelector(1) - ); - String actual2 = keyManager.computeResultLevelCachingEtag(selectors, null); - Assert.assertNotNull(actual2); - Assert.assertNotEquals(actual1, actual2); - } - @Test public void testComputeEtag_noEffectifBySegment() { expect(dataSourceAnalysis.isJoin()).andReturn(false); reset(query); expect(query.getContextValue(QueryContexts.BY_SEGMENT_KEY)).andReturn(true).anyTimes(); + expect(query.getDataSource()).andReturn(new NoopDataSource()); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); Set selectors = ImmutableSet.of( @@ -217,14 +195,13 @@ public void testComputeEtag_noEffectifBySegment() public void testComputeEtag_noEffectIfUseAndPopulateFalse() { expect(dataSourceAnalysis.isJoin()).andReturn(false); + expect(query.getDataSource()).andReturn(new NoopDataSource()); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = new CachingClusteredClient.CacheKeyManager<>( query, strategy, false, - false, - dataSourceAnalysis, - joinableFactoryWrapper + false ); Set selectors = ImmutableSet.of( makeHistoricalServerSelector(1), @@ -237,7 +214,7 @@ public void testComputeEtag_noEffectIfUseAndPopulateFalse() @Test public void testSegmentQueryCacheKey_nonJoinDataSource() { - expect(dataSourceAnalysis.isJoin()).andReturn(false); + expect(query.getDataSource()).andReturn(new NoopDataSource()); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey(); @@ -247,20 +224,18 @@ public void testSegmentQueryCacheKey_nonJoinDataSource() @Test public void testSegmentQueryCacheKey_joinWithUnsupportedCaching() { - expect(dataSourceAnalysis.isJoin()).andReturn(true); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.empty()); + expect(query.getDataSource()).andReturn(new NoopDataSource()); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey(); - Assert.assertNull(cacheKey); + Assert.assertNotNull(cacheKey); } @Test public void testSegmentQueryCacheKey_joinWithSupportedCaching() { - - expect(dataSourceAnalysis.isJoin()).andReturn(true); - expect(joinableFactoryWrapper.computeJoinDataSourceCacheKey(dataSourceAnalysis)).andReturn(Optional.of(JOIN_KEY)); + expect(query.getDataSource()).andReturn(dataSource); + expect(dataSource.getCacheKey()).andReturn(JOIN_KEY); replayAll(); CachingClusteredClient.CacheKeyManager keyManager = makeKeyManager(); byte[] cacheKey = keyManager.computeSegmentLevelQueryCacheKey(); @@ -285,9 +260,7 @@ public void testSegmentQueryCacheKey_useAndPopulateCacheFalse() query, strategy, false, - false, - dataSourceAnalysis, - joinableFactoryWrapper + false ).computeSegmentLevelQueryCacheKey()); } @@ -297,9 +270,7 @@ private CachingClusteredClient.CacheKeyManager makeKeyManager() query, strategy, true, - true, - dataSourceAnalysis, - joinableFactoryWrapper + true ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index c555e0d0dd23..bd2c8cb73803 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -138,6 +138,12 @@ public DataSource withUpdatedDataSource(DataSource newSource) return newSource; } + @Override + public byte[] getCacheKey() + { + return null; + } + @Override public boolean equals(Object o) { From eaf5c4b049cd3d556f919ebf11fa547169836def Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Thu, 13 Oct 2022 17:10:16 -0700 Subject: [PATCH 22/31] Nullable annotation check in injector --- .../apache/druid/guice/GuiceInjectableValues.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java index d12497e01964..7544206f20e6 100644 --- a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java +++ b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java @@ -22,10 +22,13 @@ 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; + /** */ public class GuiceInjectableValues extends InjectableValues @@ -50,7 +53,15 @@ public Object findInjectableValue( // 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); + try { + return injector.getInstance((Key) valueId); + } + catch (ConfigurationException ce) { + //check if annotation is nullable + if (forProperty.getAnnotation(Nullable.class).annotationType().isAnnotation()) { + return null; + } + } } throw new IAE( "Unknown class type [%s] for valueId [%s]", From 1763b5150e02543f5c8655355f02b9d670d1f42f Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Fri, 14 Oct 2022 14:26:17 -0700 Subject: [PATCH 23/31] Addressing some comments, removing 2 analysis.isJoin() checks and correcting the benchmark files --- .../druid/benchmark/query/SqlBenchmark.java | 33 +++++-------------- .../druid/math/expr/ExprMacroTable.java | 1 - .../msq/querykit/BaseLeafFrameProcessor.java | 3 +- .../msq/querykit/InputNumberDataSource.java | 9 ++++- .../org/apache/druid/query/DataSource.java | 3 +- .../apache/druid/query/InlineDataSource.java | 2 +- .../apache/druid/query/JoinDataSource.java | 1 - .../apache/druid/query/LookupDataSource.java | 2 +- .../apache/druid/query/QueryDataSource.java | 2 +- .../apache/druid/query/TableDataSource.java | 2 +- .../apache/druid/query/UnionDataSource.java | 2 +- .../segment/join/JoinableFactoryWrapper.java | 3 -- .../appenderator/SinkQuerySegmentWalker.java | 5 +-- .../server/coordination/ServerManager.java | 4 +-- .../java/org/apache/druid/cli/CliRouter.java | 3 -- .../calcite/external/ExternalDataSource.java | 2 +- 16 files changed, 27 insertions(+), 50 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index cf8f5ec3ea08..f9704f47052d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -26,6 +26,7 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QueryContext; @@ -86,8 +87,8 @@ */ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 3) -@Measurement(iterations = 10) +@Warmup(iterations = 5) +@Measurement(iterations = 15) public class SqlBenchmark { static { @@ -401,31 +402,13 @@ public class SqlBenchmark @Param({ "0", - "1", - "2", - "3", - "4", - "5", - "6", - "7", - "8", - "9", "10", - "11", - "12", - "13", - "14", - "15", - "16", - "17", - "18", - "19", - "20" + "18" }) private String query; - //@Param({STORAGE_MMAP, STORAGE_FRAME_ROW, STORAGE_FRAME_COLUMNAR}) - @Param({STORAGE_MMAP}) + @Param({STORAGE_MMAP, STORAGE_FRAME_ROW, STORAGE_FRAME_COLUMNAR}) + private String storageType; private SqlEngine engine; @@ -528,7 +511,7 @@ public void tearDown() throws Exception closer.close(); } - /*@Benchmark + @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) public void querySql(Blackhole blackhole) throws Exception @@ -544,7 +527,7 @@ public void querySql(Blackhole blackhole) throws Exception final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in); blackhole.consume(lastRow); } - }*/ + } @Benchmark @BenchmarkMode(Mode.AverageTime) diff --git a/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java b/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java index 718e10dade13..ec7c2036c903 100644 --- a/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java +++ b/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java @@ -44,7 +44,6 @@ public class ExprMacroTable { private static final ExprMacroTable NIL = new ExprMacroTable(Collections.emptyList()); - private final Map macroMap; public ExprMacroTable(final List macros) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index d1d576d0c2c5..8f01aa608b52 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -147,6 +147,7 @@ protected SegmentReference mapSegment(final Segment segment) private boolean initializeSegmentMapFn(final IntSet readableInputs) { + final AtomicLong cpuAccumulator = new AtomicLong(); if (segmentMapFn != null) { return true; } else if (broadcastJoinHelper == null) { @@ -164,7 +165,7 @@ private boolean initializeSegmentMapFn(final IntSet readableInputs) // make progress on the refactor without breaking functionality. Hopefully, some future // developer will move this away from a setter. inputNumberDataSource.setBroadcastJoinHelper(broadcastJoinHelper); - segmentMapFn = inputNumberDataSource.createSegmentMapFunction(query, new AtomicLong()); + segmentMapFn = inputNumberDataSource.createSegmentMapFunction(query, cpuAccumulator); } return retVal; } else { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index 041900315c96..51258ab45700 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -98,6 +98,13 @@ public Function createSegmentMapFunction( AtomicLong cpuTime ) { + if (broadcastJoinHelper == null) { + throw new IAE( + "No helper for broadcast join found on data source [%s]. " + + "Please make sure to set this before this call. ", + query.getDataSource() + ); + } final DataSource dataSourceWithInlinedChannelData = broadcastJoinHelper.inlineChannelData(query.getDataSource()); final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(dataSourceWithInlinedChannelData); return analysis.getDataSource().createSegmentMapFunction(query, new AtomicLong()); @@ -112,7 +119,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return null; + return new byte[0]; } @JsonProperty diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 2c3cc81098a4..e019b9fce3c9 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -114,8 +114,7 @@ public interface DataSource /** * 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 wrapped in an - * Optional + * 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 diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index 040d6885782a..431b21af64a9 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -252,7 +252,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return null; + return new byte[0]; } /** diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 544fd8045dc5..832fda42f7c9 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -421,7 +421,6 @@ public Function createSegmentMapFunction( AtomicLong cpuTimeAccumulator ) { - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); final Function segmentMapFn = createSegmentMapFn( diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index 0b40aea020d9..923986965d84 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -117,7 +117,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return null; + return new byte[0]; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index c495d0832f24..ae3f7235fd3a 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -109,7 +109,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return null; + return new byte[0]; } diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index 6182b7039451..2fbc6ca15f2b 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -114,7 +114,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return null; + return new byte[0]; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index 50802dc72524..7938b62696cd 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -126,7 +126,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return null; + return new byte[0]; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java index 74a76020b92a..020d23317f7f 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactoryWrapper.java @@ -26,7 +26,6 @@ import com.google.common.collect.Sets; import com.google.inject.Inject; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.segment.filter.FalseFilter; @@ -45,7 +44,6 @@ public class JoinableFactoryWrapper { public static final byte JOIN_OPERATION = 0x1; - private static final Logger log = new Logger(JoinableFactoryWrapper.class); private final JoinableFactory joinableFactory; @@ -60,7 +58,6 @@ public JoinableFactoryWrapper(final JoinableFactory joinableFactory) *

* See {@link #convertJoinToFilter} for details on the logic. */ - @VisibleForTesting public static Pair, List> convertJoinsToFilters( final List clauses, final Set requiredColumns, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index f08e14076044..c0c95e85840b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -31,7 +31,6 @@ import org.apache.druid.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; @@ -177,9 +176,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final // We compute the join cache key here itself so it doesn't need to be re-computed for every segment - final Optional cacheKeyPrefix = analysis.isJoin() - ? Optional.ofNullable(query.getDataSource().getCacheKey()) - : Optional.of(StringUtils.EMPTY_BYTES); + final Optional cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey()); Iterable> perSegmentRunners = Iterables.transform( specs, diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index d41b0cfc4287..2e32fa97686d 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -197,9 +197,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable segmentMapFn = analysis.getDataSource().createSegmentMapFunction(query, cpuTimeAccumulator); // We compute the join cache key here itself so it doesn't need to be re-computed for every segment - final Optional cacheKeyPrefix = analysis.isJoin() - ? Optional.ofNullable(query.getDataSource().getCacheKey()) - : Optional.of(StringUtils.EMPTY_BYTES); + final Optional cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey()); final FunctionalIterable> queryRunners = FunctionalIterable .create(specs) diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index fe73f3abb1dc..2bd1314cbe33 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -41,8 +41,6 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupSerdeModule; -import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.server.AsyncQueryForwardingServlet; import org.apache.druid.server.NoopQuerySegmentWalker; import org.apache.druid.server.http.RouterResource; @@ -115,7 +113,6 @@ protected List getModules() .toProvider(TieredBrokerSelectorStrategiesProvider.class) .in(LazySingleton.class); - binder.bind(JoinableFactory.class).to(NoopJoinableFactory.class).in(LazySingleton.class); binder.bind(QueryCountStatsProvider.class).to(AsyncQueryForwardingServlet.class).in(LazySingleton.class); binder.bind(JettyServerInitializer.class).to(RouterJettyServerInitializer.class).in(LazySingleton.class); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index bd2c8cb73803..ab35620754b1 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -141,7 +141,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return null; + return new byte[0]; } @Override From a8291f1f30535f59b06cc00f299a83de274acf32 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Mon, 17 Oct 2022 12:51:40 -0700 Subject: [PATCH 24/31] Minor changes for refactoring --- .../msq/querykit/BaseLeafFrameProcessor.java | 23 +++++-------------- .../msq/querykit/InputNumberDataSource.java | 2 +- .../apache/druid/query/InlineDataSource.java | 2 +- .../apache/druid/query/JoinDataSource.java | 1 + .../apache/druid/query/LookupDataSource.java | 2 +- .../apache/druid/query/QueryDataSource.java | 2 +- .../apache/druid/query/UnionDataSource.java | 2 +- 7 files changed, 12 insertions(+), 22 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index 8f01aa608b52..1e1b9cd3516f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -38,6 +38,7 @@ import org.apache.druid.query.DataSource; import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.Query; +import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; @@ -154,24 +155,12 @@ private boolean initializeSegmentMapFn(final IntSet readableInputs) segmentMapFn = Function.identity(); return true; } else { - if (query.getDataSource() instanceof InputNumberDataSource) { - final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); - if (retVal) { - InputNumberDataSource inputNumberDataSource = (InputNumberDataSource) query.getDataSource(); - // The InputNumberDataSource requires a BroadcastJoinHelper to be able to create its - // segment map function. It would be a lot better if the InputNumberDataSource actually - // had a way to get that injected into it on its own, but the relationship between these objects - // was figured out during a refactor and using a setter here seemed like the least-bad way to - // make progress on the refactor without breaking functionality. Hopefully, some future - // developer will move this away from a setter. - inputNumberDataSource.setBroadcastJoinHelper(broadcastJoinHelper); - segmentMapFn = inputNumberDataSource.createSegmentMapFunction(query, cpuAccumulator); - } - return retVal; - } else { - segmentMapFn = Function.identity(); - return true; + final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); + if (retVal) { + final DataSource dataSourceWithInlinedChannelData = broadcastJoinHelper.inlineChannelData(query.getDataSource()); + segmentMapFn = dataSourceWithInlinedChannelData.createSegmentMapFunction(query, cpuAccumulator); } + return true; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index 51258ab45700..84b11269172f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -119,7 +119,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return new byte[0]; + return null; } @JsonProperty diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index 431b21af64a9..040d6885782a 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -252,7 +252,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return new byte[0]; + return null; } /** diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 832fda42f7c9..849119f991d8 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -97,6 +97,7 @@ public class JoinDataSource implements DataSource private static final Logger log = new Logger(JoinDataSource.class); private final DataSourceAnalysis analysis; + private JoinDataSource( DataSource left, DataSource right, diff --git a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java index 923986965d84..0b40aea020d9 100644 --- a/processing/src/main/java/org/apache/druid/query/LookupDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/LookupDataSource.java @@ -117,7 +117,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return new byte[0]; + return null; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java index ae3f7235fd3a..c495d0832f24 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/QueryDataSource.java @@ -109,7 +109,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return new byte[0]; + return null; } diff --git a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java index 7938b62696cd..50802dc72524 100644 --- a/processing/src/main/java/org/apache/druid/query/UnionDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/UnionDataSource.java @@ -126,7 +126,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return new byte[0]; + return null; } @Override From 656934b5d578b22aedac2f823e93c02799321326 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Mon, 17 Oct 2022 23:04:40 -0700 Subject: [PATCH 25/31] Addressing reviews part 1 --- .../msq/querykit/BaseLeafFrameProcessor.java | 19 +++++++++++++++---- .../msq/querykit/InputNumberDataSource.java | 4 ++-- .../apache/druid/query/JoinDataSource.java | 4 ++-- .../druid/server/LocalQuerySegmentWalker.java | 2 +- .../sql/calcite/planner/PlannerContext.java | 12 ++++++------ 5 files changed, 26 insertions(+), 15 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index 1e1b9cd3516f..f0d7b594f06a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -38,7 +38,6 @@ import org.apache.druid.query.DataSource; import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.Query; -import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentReference; @@ -156,11 +155,23 @@ private boolean initializeSegmentMapFn(final IntSet readableInputs) return true; } else { final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); + DataSource inlineChannelDataSource = broadcastJoinHelper.inlineChannelData(query.getDataSource()); if (retVal) { - final DataSource dataSourceWithInlinedChannelData = broadcastJoinHelper.inlineChannelData(query.getDataSource()); - segmentMapFn = dataSourceWithInlinedChannelData.createSegmentMapFunction(query, cpuAccumulator); + if (inlineChannelDataSource instanceof InputNumberDataSource) { + InputNumberDataSource inputNumberDataSource = (InputNumberDataSource) query.getDataSource(); + // The InputNumberDataSource requires a BroadcastJoinHelper to be able to create its + // segment map function. It would be a lot better if the InputNumberDataSource actually + // had a way to get that injected into it on its own, but the relationship between these objects + // was figured out during a refactor and using a setter here seemed like the least-bad way to + // make progress on the refactor without breaking functionality. Hopefully, some future + // developer will move this away from a setter. + inputNumberDataSource.setBroadcastJoinHelper(broadcastJoinHelper); + segmentMapFn = inputNumberDataSource.createSegmentMapFunction(query, cpuAccumulator); + } else { + segmentMapFn = inlineChannelDataSource.createSegmentMapFunction(query, cpuAccumulator); + } } - return true; + return retVal; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index 84b11269172f..7a6dc0fce5c1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -95,7 +95,7 @@ public boolean isConcrete() @Override public Function createSegmentMapFunction( Query query, - AtomicLong cpuTime + AtomicLong cpuTimeAcc ) { if (broadcastJoinHelper == null) { @@ -107,7 +107,7 @@ public Function createSegmentMapFunction( } final DataSource dataSourceWithInlinedChannelData = broadcastJoinHelper.inlineChannelData(query.getDataSource()); final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(dataSourceWithInlinedChannelData); - return analysis.getDataSource().createSegmentMapFunction(query, new AtomicLong()); + return analysis.getDataSource().createSegmentMapFunction(query, cpuTimeAcc); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java index 849119f991d8..e17e33c9a610 100644 --- a/processing/src/main/java/org/apache/druid/query/JoinDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/JoinDataSource.java @@ -98,6 +98,7 @@ public class JoinDataSource implements DataSource private final DataSourceAnalysis analysis; + private JoinDataSource( DataSource left, DataSource right, @@ -416,14 +417,13 @@ public Function createSegmentMapFn( ); } + @Override public Function createSegmentMapFunction( Query query, AtomicLong cpuTimeAccumulator ) { - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(query.getDataSource()); - final Function segmentMapFn = createSegmentMapFn( analysis.getJoinBaseTableFilter().map(Filters::toFilter).orElse(null), analysis.getPreJoinableClauses(), diff --git a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java index 2e4357b86186..730eb3121d47 100644 --- a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java @@ -93,7 +93,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final final AtomicLong cpuAccumulator = new AtomicLong(0L); - Function segmentMapFn = analysis + final Function segmentMapFn = analysis .getDataSource() .createSegmentMapFunction( query, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index e641af0a1201..607475d6c1f7 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -78,7 +78,7 @@ public class PlannerContext private final String sql; private final DruidOperatorTable operatorTable; private final ExprMacroTable macroTable; - private final JoinableFactoryWrapper jfw; + private final JoinableFactoryWrapper joinableFactoryWrapper; private final ObjectMapper jsonMapper; private final PlannerConfig plannerConfig; private final DateTime localNow; @@ -113,7 +113,7 @@ private PlannerContext( final DruidSchemaCatalog rootSchema, final SqlEngine engine, final QueryContext queryContext, - JoinableFactoryWrapper jfw + JoinableFactoryWrapper joinableFactoryWrapper ) { this.sql = sql; @@ -126,7 +126,7 @@ private PlannerContext( this.queryContext = queryContext; this.localNow = Preconditions.checkNotNull(localNow, "localNow"); this.stringifyArrays = stringifyArrays; - this.jfw = jfw; + this.joinableFactoryWrapper = joinableFactoryWrapper; String sqlQueryId = (String) this.queryContext.get(CTX_SQL_QUERY_ID); // special handling for DruidViewMacro, normal client will allocate sqlid in SqlLifecyle @@ -145,7 +145,7 @@ public static PlannerContext create( final DruidSchemaCatalog rootSchema, final SqlEngine engine, final QueryContext queryContext, - final JoinableFactoryWrapper jfw + final JoinableFactoryWrapper joinableFactoryWrapper ) { final DateTime utcNow; @@ -185,7 +185,7 @@ public static PlannerContext create( rootSchema, engine, queryContext, - jfw + joinableFactoryWrapper ); } @@ -221,7 +221,7 @@ public DateTimeZone getTimeZone() public JoinableFactoryWrapper getJoinableFactoryWrapper() { - return jfw; + return joinableFactoryWrapper; } @Nullable From 899add0f608c828c2ade6eceeca0b8863e5c7c56 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Tue, 18 Oct 2022 09:44:12 -0700 Subject: [PATCH 26/31] Refactoring part 2 with new test cases for broadcast join --- .../druid/benchmark/query/SqlBenchmark.java | 7 +- .../druid/guice/GuiceInjectableValues.java | 4 +- .../apache/druid/msq/exec/MSQSelectTest.java | 117 ++++++++++++++++++ .../org/apache/druid/query/DataSource.java | 2 - .../org/apache/druid/segment/TestHelper.java | 6 +- .../appenderator/SinkQuerySegmentWalker.java | 2 +- .../server/coordination/ServerManager.java | 2 +- .../java/org/apache/druid/cli/CliRouter.java | 2 +- .../calcite/external/ExternalDataSource.java | 2 +- .../sql/calcite/planner/PlannerContext.java | 2 +- .../sql/calcite/planner/PlannerFactory.java | 8 +- .../druid/sql/calcite/rel/DruidQuery.java | 4 +- .../sql/calcite/BaseCalciteQueryTest.java | 2 +- 13 files changed, 135 insertions(+), 25 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index f9704f47052d..55eca6d9eed6 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -400,15 +400,10 @@ public class SqlBenchmark @Param({"force"}) private String vectorize; - @Param({ - "0", - "10", - "18" - }) + @Param({"0", "10", "18"}) private String query; @Param({STORAGE_MMAP, STORAGE_FRAME_ROW, STORAGE_FRAME_COLUMNAR}) - private String storageType; private SqlEngine engine; diff --git a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java index 7544206f20e6..266a2ba5a9a1 100644 --- a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java +++ b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java @@ -57,8 +57,8 @@ public Object findInjectableValue( return injector.getInstance((Key) valueId); } catch (ConfigurationException ce) { - //check if annotation is nullable - if (forProperty.getAnnotation(Nullable.class).annotationType().isAnnotation()) { + // check if nullable annotation is present for this + if (forProperty.getAnnotation(Nullable.class) != null) { return null; } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 10c67e27a28b..f9c9e08c07fa 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -34,6 +34,7 @@ import org.apache.druid.msq.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 void testJoin() .verifyResults(); } + @Test + public void testBroadcastJoin() + { + final RowSignature resultSignature = RowSignature.builder() + .add("dim2", ColumnType.STRING) + .add("EXPR$1", ColumnType.DOUBLE) + .build(); + + final ImmutableList expectedResults; + + if (NullHandling.sqlCompatible()) { + expectedResults = ImmutableList.of( + new Object[]{null, 4.0}, + new Object[]{"", 3.0}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ); + } else { + expectedResults = ImmutableList.of( + new Object[]{null, 3.6666666666666665}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ); + } + + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2", "m1", "m2") + .context( + defaultScanQueryContext( + RowSignature.builder() + .add("dim2", ColumnType.STRING) + .add("m1", ColumnType.FLOAT) + .add("m2", ColumnType.DOUBLE) + .build() + ) + ) + .limit(10) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.ofColumn(ColumnType.FLOAT, "m1"), + DruidExpression.ofColumn(ColumnType.FLOAT, "j0.m1") + ), + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimensions(new DefaultDimensionSpec("j0.dim2", "d0", ColumnType.STRING)) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + useDefault + ? aggregators( + new DoubleSumAggregatorFactory("a0:sum", "j0.m2"), + new CountAggregatorFactory("a0:count") + ) + : aggregators( + new DoubleSumAggregatorFactory("a0:sum", "m2"), + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0:count"), + not(selector("m2", null, null)), + + // Not sure why the name is only set in SQL-compatible null mode. Seems strange. + // May be due to JSON serialization: name is set on the serialized aggregator even + // if it was originally created with no name. + NullHandling.sqlCompatible() ? "a0:count" : null + ) + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "a0:sum"), + new FieldAccessPostAggregator(null, "a0:count") + ) + ) + + ) + ) + .setContext(DEFAULT_MSQ_CONTEXT) + .build(); + + testSelectQuery() + .setSql( + "SELECT t1.dim2, AVG(t1.m2) FROM " + + "foo " + + "INNER JOIN (SELECT * FROM foo LIMIT 10) AS t1 " + + "ON t1.m1 = foo.m1 " + + "GROUP BY t1.dim2" + ) + .setExpectedMSQSpec( + MSQSpec.builder() + .query(query) + .columnMappings(new ColumnMappings(ImmutableList.of( + new ColumnMapping("d0", "dim2"), + new ColumnMapping("a0", "EXPR$1") + ))) + .tuningConfig(MSQTuningConfig.defaultConfig()) + .build() + ) + .setExpectedRowSignature(resultSignature) + .setExpectedResultRows(expectedResults) + .verifyResults(); + } + @Test public void testGroupByOrderByAggregation() { diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index e019b9fce3c9..f56a3550a34b 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.planning.PreJoinableClause; import org.apache.druid.segment.SegmentReference; @@ -121,7 +120,6 @@ public interface DataSource * in the JOIN is not cacheable. * * @return the 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 */ byte[] getCacheKey(); diff --git a/processing/src/test/java/org/apache/druid/segment/TestHelper.java b/processing/src/test/java/org/apache/druid/segment/TestHelper.java index 296906ff84ed..525fb61b7c30 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -96,7 +96,7 @@ public Object findInjectableValueId(AnnotatedMember m) public static ObjectMapper makeJsonMapper() { final ObjectMapper mapper = new DefaultObjectMapper(); - AnnotationIntrospector introspector = makeAnnotationIntrospector(); + final AnnotationIntrospector introspector = makeAnnotationIntrospector(); DruidSecondaryModule.setupAnnotationIntrospector(mapper, introspector); @@ -109,7 +109,7 @@ public static ObjectMapper makeJsonMapper() return mapper; } - public static ObjectMapper makeJsonMapperForJoinable(JoinableFactoryWrapper jfw) + public static ObjectMapper makeJsonMapperForJoinable(JoinableFactoryWrapper joinableFactoryWrapper) { final ObjectMapper mapper = new DefaultObjectMapper(); AnnotationIntrospector introspector = makeAnnotationIntrospector(); @@ -121,7 +121,7 @@ public static ObjectMapper makeJsonMapperForJoinable(JoinableFactoryWrapper jfw) .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class.getName(), mapper) .addValue(PruneSpecsHolder.class, PruneSpecsHolder.DEFAULT) - .addValue(JoinableFactoryWrapper.class, jfw) + .addValue(JoinableFactoryWrapper.class, joinableFactoryWrapper) ); return mapper; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index c0c95e85840b..11412b73bc56 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -168,7 +168,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final } // segmentMapFn maps each base Segment into a joined Segment if necessary. - Function segmentMapFn = analysis.getDataSource() + final Function segmentMapFn = analysis.getDataSource() .createSegmentMapFunction( query, cpuTimeAccumulator diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index 2e32fa97686d..26aabc2ddd68 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -194,7 +194,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable(Lists.newArrayList(specs)); } - Function segmentMapFn = analysis.getDataSource().createSegmentMapFunction(query, cpuTimeAccumulator); + final Function segmentMapFn = query.getDataSource().createSegmentMapFunction(query, cpuTimeAccumulator); // We compute the join cache key here itself so it doesn't need to be re-computed for every segment final Optional cacheKeyPrefix = Optional.ofNullable(query.getDataSource().getCacheKey()); diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index 2bd1314cbe33..af0ca7c6eb24 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -112,7 +112,7 @@ protected List getModules() binder.bind(new TypeLiteral>() {}) .toProvider(TieredBrokerSelectorStrategiesProvider.class) .in(LazySingleton.class); - + binder.bind(QueryCountStatsProvider.class).to(AsyncQueryForwardingServlet.class).in(LazySingleton.class); binder.bind(JettyServerInitializer.class).to(RouterJettyServerInitializer.class).in(LazySingleton.class); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java index ab35620754b1..bd2c8cb73803 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/external/ExternalDataSource.java @@ -141,7 +141,7 @@ public DataSource withUpdatedDataSource(DataSource newSource) @Override public byte[] getCacheKey() { - return new byte[0]; + return null; } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 607475d6c1f7..b9d586108bde 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -113,7 +113,7 @@ private PlannerContext( final DruidSchemaCatalog rootSchema, final SqlEngine engine, final QueryContext queryContext, - JoinableFactoryWrapper joinableFactoryWrapper + final JoinableFactoryWrapper joinableFactoryWrapper ) { this.sql = sql; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java index d8b2157c9a8e..495db61a6065 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerFactory.java @@ -72,7 +72,7 @@ public class PlannerFactory private final AuthorizerMapper authorizerMapper; private final String druidSchemaName; private final CalciteRulesManager calciteRuleManager; - private final JoinableFactoryWrapper jfw; + private final JoinableFactoryWrapper joinableFactoryWrapper; @Inject public PlannerFactory( @@ -84,7 +84,7 @@ public PlannerFactory( final @Json ObjectMapper jsonMapper, final @DruidSchemaName String druidSchemaName, final CalciteRulesManager calciteRuleManager, - final JoinableFactoryWrapper jfw + final JoinableFactoryWrapper joinableFactoryWrapper ) { this.rootSchema = rootSchema; @@ -95,7 +95,7 @@ public PlannerFactory( this.jsonMapper = jsonMapper; this.druidSchemaName = druidSchemaName; this.calciteRuleManager = calciteRuleManager; - this.jfw = jfw; + this.joinableFactoryWrapper = joinableFactoryWrapper; } /** @@ -112,7 +112,7 @@ public DruidPlanner createPlanner(final SqlEngine engine, final String sql, fina rootSchema, engine, queryContext, - jfw + joinableFactoryWrapper ); return new DruidPlanner(buildFrameworkConfig(context), context, engine); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index b2581120a510..94e470d9ec7d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -740,7 +740,7 @@ static Pair getFiltration( DataSource dataSource, DimFilter filter, VirtualColumnRegistry virtualColumnRegistry, - JoinableFactoryWrapper jfw + JoinableFactoryWrapper joinableFactoryWrapper ) { if (!(dataSource instanceof JoinDataSource)) { @@ -769,7 +769,7 @@ static Pair getFiltration( joinDataSource.getConditionAnalysis(), joinDataSource.getJoinType(), leftFiltration.getDimFilter(), - jfw + joinableFactoryWrapper ); return Pair.of(newDataSource, queryFiltration); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 5fcc4e05b57f..8fcb905ead1b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -1083,7 +1083,7 @@ public SqlStatementFactory getSqlStatementFactory( authorizerMapper ); - JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); + final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(CalciteTests.createDefaultJoinableFactory()); final PlannerFactory plannerFactory = new PlannerFactory( rootSchema, From 0474963998e3d79a46818cb4b4e4637395501bdb Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Wed, 19 Oct 2022 10:09:16 -0700 Subject: [PATCH 27/31] Set for nullables --- .../org/apache/druid/guice/GuiceInjectableValues.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java index 266a2ba5a9a1..4e4dff037d00 100644 --- a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java +++ b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java @@ -28,16 +28,20 @@ import org.apache.druid.java.util.common.IAE; import javax.annotation.Nullable; +import java.util.HashSet; +import java.util.Set; /** */ public class GuiceInjectableValues extends InjectableValues { private final Injector injector; + private Set nullables; public GuiceInjectableValues(Injector injector) { this.injector = injector; + this.nullables = new HashSet<>(); } @Override @@ -52,13 +56,17 @@ public Object findInjectableValue( // 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) { + if (nullables.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) { + nullables.add((Key) valueId); return null; } } From e45e07e0073cb7dea96ffa148a65baa5824ab374 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Thu, 20 Oct 2022 19:21:23 -0700 Subject: [PATCH 28/31] removing instance of checks --- .../druid/guice/GuiceInjectableValues.java | 11 +- .../msq/querykit/BaseLeafFrameProcessor.java | 118 ++++++++---------- .../msq/querykit/InputNumberDataSource.java | 18 +-- 3 files changed, 56 insertions(+), 91 deletions(-) diff --git a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java index 4e4dff037d00..c813fe46cd50 100644 --- a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java +++ b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java @@ -28,20 +28,16 @@ import org.apache.druid.java.util.common.IAE; import javax.annotation.Nullable; -import java.util.HashSet; -import java.util.Set; /** */ public class GuiceInjectableValues extends InjectableValues { private final Injector injector; - private Set nullables; public GuiceInjectableValues(Injector injector) { this.injector = injector; - this.nullables = new HashSet<>(); } @Override @@ -56,19 +52,16 @@ public Object findInjectableValue( // whatever provider needs" // Currently we should only be dealing with `Key` instances, and anything more advanced should be handled with // great care - if (nullables.contains((Key) valueId)) { - return null; - } - else if (valueId instanceof Key) { + 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) { - nullables.add((Key) valueId); return null; } + throw ce; } } throw new IAE( diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java index f0d7b594f06a..e8b761c8007b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/BaseLeafFrameProcessor.java @@ -89,6 +89,58 @@ protected BaseLeafFrameProcessor( this.broadcastJoinHelper = inputChannelsAndBroadcastJoinHelper.rhs; } + /** + * Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up their primary and side channels. + */ + private static Pair, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper( + final DataSource dataSource, + final ReadableInput baseInput, + final Int2ObjectMap sideChannels, + final JoinableFactoryWrapper joinableFactory, + final long memoryReservedForBroadcastJoin + ) + { + if (!(dataSource instanceof JoinDataSource) && !sideChannels.isEmpty()) { + throw new ISE("Did not expect side channels for dataSource [%s]", dataSource); + } + + final List inputChannels = new ArrayList<>(); + final BroadcastJoinHelper broadcastJoinHelper; + + if (baseInput.hasChannel()) { + inputChannels.add(baseInput.getChannel()); + } + + if (dataSource instanceof JoinDataSource) { + final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap(); + final List channelReaders = new ArrayList<>(); + + if (baseInput.hasChannel()) { + // BroadcastJoinHelper doesn't need to read the base channel, so stub in a null reader. + channelReaders.add(null); + } + + for (Int2ObjectMap.Entry sideChannelEntry : sideChannels.int2ObjectEntrySet()) { + final int inputNumber = sideChannelEntry.getIntKey(); + inputNumberToProcessorChannelMap.put(inputNumber, inputChannels.size()); + inputChannels.add(sideChannelEntry.getValue().getChannel()); + channelReaders.add(sideChannelEntry.getValue().getChannelFrameReader()); + } + + broadcastJoinHelper = new BroadcastJoinHelper( + inputNumberToProcessorChannelMap, + inputChannels, + channelReaders, + joinableFactory, + memoryReservedForBroadcastJoin + ); + } else { + broadcastJoinHelper = null; + } + + return Pair.of(inputChannels, broadcastJoinHelper); + } + @Override public List inputChannels() { @@ -157,73 +209,9 @@ private boolean initializeSegmentMapFn(final IntSet readableInputs) final boolean retVal = broadcastJoinHelper.buildBroadcastTablesIncrementally(readableInputs); DataSource inlineChannelDataSource = broadcastJoinHelper.inlineChannelData(query.getDataSource()); if (retVal) { - if (inlineChannelDataSource instanceof InputNumberDataSource) { - InputNumberDataSource inputNumberDataSource = (InputNumberDataSource) query.getDataSource(); - // The InputNumberDataSource requires a BroadcastJoinHelper to be able to create its - // segment map function. It would be a lot better if the InputNumberDataSource actually - // had a way to get that injected into it on its own, but the relationship between these objects - // was figured out during a refactor and using a setter here seemed like the least-bad way to - // make progress on the refactor without breaking functionality. Hopefully, some future - // developer will move this away from a setter. - inputNumberDataSource.setBroadcastJoinHelper(broadcastJoinHelper); - segmentMapFn = inputNumberDataSource.createSegmentMapFunction(query, cpuAccumulator); - } else { - segmentMapFn = inlineChannelDataSource.createSegmentMapFunction(query, cpuAccumulator); - } + segmentMapFn = inlineChannelDataSource.createSegmentMapFunction(query, cpuAccumulator); } return retVal; } } - - /** - * Helper that enables implementations of {@link BaseLeafFrameProcessorFactory} to set up their primary and side channels. - */ - private static Pair, BroadcastJoinHelper> makeInputChannelsAndBroadcastJoinHelper( - final DataSource dataSource, - final ReadableInput baseInput, - final Int2ObjectMap sideChannels, - final JoinableFactoryWrapper joinableFactory, - final long memoryReservedForBroadcastJoin - ) - { - if (!(dataSource instanceof JoinDataSource) && !sideChannels.isEmpty()) { - throw new ISE("Did not expect side channels for dataSource [%s]", dataSource); - } - - final List inputChannels = new ArrayList<>(); - final BroadcastJoinHelper broadcastJoinHelper; - - if (baseInput.hasChannel()) { - inputChannels.add(baseInput.getChannel()); - } - - if (dataSource instanceof JoinDataSource) { - final Int2IntMap inputNumberToProcessorChannelMap = new Int2IntOpenHashMap(); - final List channelReaders = new ArrayList<>(); - - if (baseInput.hasChannel()) { - // BroadcastJoinHelper doesn't need to read the base channel, so stub in a null reader. - channelReaders.add(null); - } - - for (Int2ObjectMap.Entry sideChannelEntry : sideChannels.int2ObjectEntrySet()) { - final int inputNumber = sideChannelEntry.getIntKey(); - inputNumberToProcessorChannelMap.put(inputNumber, inputChannels.size()); - inputChannels.add(sideChannelEntry.getValue().getChannel()); - channelReaders.add(sideChannelEntry.getValue().getChannelFrameReader()); - } - - broadcastJoinHelper = new BroadcastJoinHelper( - inputNumberToProcessorChannelMap, - inputChannels, - channelReaders, - joinableFactory, - memoryReservedForBroadcastJoin - ); - } else { - broadcastJoinHelper = null; - } - - return Pair.of(inputChannels, broadcastJoinHelper); - } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java index 7a6dc0fce5c1..2fb4a61ee8ca 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/InputNumberDataSource.java @@ -25,7 +25,6 @@ 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 java.util.Collections; @@ -39,7 +38,6 @@ public class InputNumberDataSource implements DataSource { private final int inputNumber; - private BroadcastJoinHelper broadcastJoinHelper; @JsonCreator public InputNumberDataSource(@JsonProperty("inputNumber") int inputNumber) @@ -47,11 +45,6 @@ public InputNumberDataSource(@JsonProperty("inputNumber") int inputNumber) this.inputNumber = inputNumber; } - public void setBroadcastJoinHelper(BroadcastJoinHelper broadcastJoinHelper) - { - this.broadcastJoinHelper = broadcastJoinHelper; - } - @Override public Set getTableNames() { @@ -98,16 +91,7 @@ public Function createSegmentMapFunction( AtomicLong cpuTimeAcc ) { - if (broadcastJoinHelper == null) { - throw new IAE( - "No helper for broadcast join found on data source [%s]. " - + "Please make sure to set this before this call. ", - query.getDataSource() - ); - } - final DataSource dataSourceWithInlinedChannelData = broadcastJoinHelper.inlineChannelData(query.getDataSource()); - final DataSourceAnalysis analysis = DataSourceAnalysis.forDataSource(dataSourceWithInlinedChannelData); - return analysis.getDataSource().createSegmentMapFunction(query, cpuTimeAcc); + return Function.identity(); } @Override From bef8349ec041653a3f309e653de3e894a2c11137 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Fri, 21 Oct 2022 12:16:23 -0700 Subject: [PATCH 29/31] Storing nullables in guice to avoid checking on reruns --- .../apache/druid/guice/GuiceInjectableValues.java | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java index c813fe46cd50..bd77bc3a7dc8 100644 --- a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java +++ b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java @@ -28,16 +28,23 @@ import org.apache.druid.java.util.common.IAE; import javax.annotation.Nullable; +import java.util.HashSet; +import java.util.concurrent.atomic.AtomicReference; /** + * */ public class GuiceInjectableValues extends InjectableValues { private final Injector injector; + private final AtomicReference> nullables; public GuiceInjectableValues(Injector injector) { this.injector = injector; + HashSet initalNullables = new HashSet<>(); + this.nullables = new AtomicReference<>(); + this.nullables.set(initalNullables); } @Override @@ -58,7 +65,12 @@ public Object findInjectableValue( } catch (ConfigurationException ce) { // check if nullable annotation is present for this - if (forProperty.getAnnotation(Nullable.class) != null) { + if (nullables.get().contains((Key) valueId)) { + return null; + } else if (forProperty.getAnnotation(Nullable.class) != null) { + HashSet newNullables = nullables.get(); + newNullables.add((Key) valueId); + nullables.set(newNullables); return null; } throw ce; From dd00ccd6ae3f9756dae8b58a8b1cb7495fcdec62 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Sat, 22 Oct 2022 14:42:14 -0700 Subject: [PATCH 30/31] Fixing a test case and removing an irrelevant line --- .../org/apache/druid/guice/GuiceInjectableValues.java | 9 ++++----- .../java/org/apache/druid/msq/exec/MSQSelectTest.java | 4 ++-- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java index bd77bc3a7dc8..3c928f212854 100644 --- a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java +++ b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java @@ -42,9 +42,8 @@ public class GuiceInjectableValues extends InjectableValues public GuiceInjectableValues(Injector injector) { this.injector = injector; - HashSet initalNullables = new HashSet<>(); this.nullables = new AtomicReference<>(); - this.nullables.set(initalNullables); + this.nullables.set(new HashSet<>()); } @Override @@ -68,9 +67,9 @@ public Object findInjectableValue( if (nullables.get().contains((Key) valueId)) { return null; } else if (forProperty.getAnnotation(Nullable.class) != null) { - HashSet newNullables = nullables.get(); - newNullables.add((Key) valueId); - nullables.set(newNullables); + HashSet encounteredNullables = nullables.get(); + encounteredNullables.add((Key) valueId); + nullables.set(encounteredNullables); return null; } throw ce; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index e408995f4f47..de7038700b8f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -504,10 +504,10 @@ public void testBroadcastJoin() new CountAggregatorFactory("a0:count") ) : aggregators( - new DoubleSumAggregatorFactory("a0:sum", "m2"), + new DoubleSumAggregatorFactory("a0:sum", "j0.m2"), new FilteredAggregatorFactory( new CountAggregatorFactory("a0:count"), - not(selector("m2", null, null)), + 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 From 40d372459598ad58a17b18bffadf3ca92e8a0503 Mon Sep 17 00:00:00 2001 From: Soumyava Das Date: Mon, 24 Oct 2022 09:13:51 -0700 Subject: [PATCH 31/31] Addressing the atomic reference review comments --- .../apache/druid/guice/GuiceInjectableValues.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java index 3c928f212854..a02bf3d054a3 100644 --- a/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java +++ b/core/src/main/java/org/apache/druid/guice/GuiceInjectableValues.java @@ -42,8 +42,7 @@ public class GuiceInjectableValues extends InjectableValues public GuiceInjectableValues(Injector injector) { this.injector = injector; - this.nullables = new AtomicReference<>(); - this.nullables.set(new HashSet<>()); + this.nullables = new AtomicReference<>(new HashSet<>()); } @Override @@ -58,16 +57,16 @@ public Object findInjectableValue( // 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) { + 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 (nullables.get().contains((Key) valueId)) { - return null; - } else if (forProperty.getAnnotation(Nullable.class) != null) { - HashSet encounteredNullables = nullables.get(); + if (forProperty.getAnnotation(Nullable.class) != null) { + HashSet encounteredNullables = new HashSet<>(nullables.get()); encounteredNullables.add((Key) valueId); nullables.set(encounteredNullables); return null;