diff --git a/docs/configuration/index.md b/docs/configuration/index.md index ea12f7dfb68..100922b04b5 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1306,6 +1306,7 @@ Druid uses Jetty to serve HTTP requests. |`druid.server.http.unannouncePropagationDelay`|How long to wait for zookeeper unannouncements to propagate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0S` (do not wait)| |`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](../querying/query-context.html) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE| |`druid.server.http.maxRequestHeaderSize`|Maximum size of a request header in bytes. Larger headers consume more memory and can make a server more vulnerable to denial of service attacks.|8 * 1024| +|`druid.server.http.maxSubqueryRows`|Maximum number of rows from subqueries per query. These rows are stored in memory.|100000| |`druid.server.http.enableForwardedRequestCustomizer`|If enabled, adds Jetty ForwardedRequestCustomizer which reads X-Forwarded-* request headers to manipulate servlet request object when Druid is used behind a proxy.|false| #### Indexer Processing Resources @@ -1632,6 +1633,10 @@ The Druid SQL server is configured through the following properties on the Broke |`druid.sql.planner.sqlTimeZone`|Sets the default time zone for the server, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|UTC| |`druid.sql.planner.serializeComplexValues`|Whether to serialize "complex" output values, false will return the class name instead of the serialized value.|true| +> Previous versions of Druid had properties named `druid.sql.planner.maxQueryCount` and `druid.sql.planner.maxSemiJoinRowsInMemory`. +> These properties are no longer available. Since Druid 0.18.0, you can use `druid.server.http.maxSubqueryRows` to control the maximum +> number of rows permitted across all subqueries. + #### Broker Caching You can optionally only configure caching to be enabled on the Broker by setting caching configs here. diff --git a/docs/querying/sql.md b/docs/querying/sql.md index 42d38922b28..0a56aa93192 100644 --- a/docs/querying/sql.md +++ b/docs/querying/sql.md @@ -938,8 +938,6 @@ The Druid SQL server is configured through the following properties on the Broke |`druid.sql.avatica.maxStatementsPerConnection`|Maximum number of simultaneous open statements per Avatica client connection.|4| |`druid.sql.avatica.connectionIdleTimeout`|Avatica client connection idle timeout.|PT5M| |`druid.sql.http.enable`|Whether to enable JSON over HTTP querying at `/druid/v2/sql/`.|true| -|`druid.sql.planner.maxQueryCount`|Maximum number of queries to issue, including nested queries. Set to 1 to disable sub-queries, or set to 0 for unlimited.|8| -|`druid.sql.planner.maxSemiJoinRowsInMemory`|Maximum number of rows to keep in memory for executing two-stage semi-join queries like `SELECT * FROM Employee WHERE DeptName IN (SELECT DeptName FROM Dept)`.|100000| |`druid.sql.planner.maxTopNLimit`|Maximum threshold for a [TopN query](../querying/topnquery.md). Higher limits will be planned as [GroupBy queries](../querying/groupbyquery.md) instead.|100000| |`druid.sql.planner.metadataRefreshPeriod`|Throttle for metadata refreshes.|PT1M| |`druid.sql.planner.useApproximateCountDistinct`|Whether to use an approximate cardinality algorithm for `COUNT(DISTINCT foo)`.|true| @@ -949,6 +947,10 @@ The Druid SQL server is configured through the following properties on the Broke |`druid.sql.planner.metadataSegmentCacheEnable`|Whether to keep a cache of published segments in broker. If true, broker polls coordinator in background to get segments from metadata store and maintains a local cache. If false, coordinator's REST API will be invoked when broker needs published segments info.|false| |`druid.sql.planner.metadataSegmentPollPeriod`|How often to poll coordinator for published segments list if `druid.sql.planner.metadataSegmentCacheEnable` is set to true. Poll period is in milliseconds. |60000| +> Previous versions of Druid had properties named `druid.sql.planner.maxQueryCount` and `druid.sql.planner.maxSemiJoinRowsInMemory`. +> These properties are no longer available. Since Druid 0.18.0, you can use `druid.server.http.maxSubqueryRows` to control the maximum +> number of rows permitted across all subqueries. + ## SQL Metrics Broker will emit the following metrics for SQL. 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 a45e3d3119e..9c2c8458ce6 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -31,6 +31,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; +import javax.annotation.Nullable; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -116,6 +117,7 @@ public class InlineDataSource implements DataSource return signature.getColumnNames(); } + @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) public List getColumnTypes() diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index e708dd35900..03e26fe92a9 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -131,8 +131,15 @@ public interface Query @Nullable String getId(); + /** + * Returns a copy of this query with a new subQueryId (see {@link #getSubQueryId()}. + */ Query withSubQueryId(String subQueryId); + /** + * Returns the subQueryId of this query. This is set by ClientQuerySegmentWalker (the entry point for the Broker's + * query stack) on any subqueries that it issues. It is null for the main query. + */ @Nullable String getSubQueryId(); diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index 2b148acee2f..ab3c02c353e 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -47,6 +47,7 @@ public class QueryContexts public static final String BROKER_PARALLELISM = "parallelMergeParallelism"; public static final String VECTORIZE_KEY = "vectorize"; public static final String VECTOR_SIZE_KEY = "vectorSize"; + public static final String MAX_SUBQUERY_ROWS_KEY = "maxSubqueryRows"; public static final String JOIN_FILTER_PUSH_DOWN_KEY = "enableJoinFilterPushDown"; public static final String JOIN_FILTER_REWRITE_ENABLE_KEY = "enableJoinFilterRewrite"; public static final String JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY = "enableJoinFilterRewriteValueColumnFilters"; @@ -188,6 +189,11 @@ public class QueryContexts return parseInt(query, VECTOR_SIZE_KEY, defaultSize); } + public static int getMaxSubqueryRows(Query query, int defaultSize) + { + return parseInt(query, MAX_SUBQUERY_ROWS_KEY, defaultSize); + } + public static int getUncoveredIntervalsLimit(Query query) { return getUncoveredIntervalsLimit(query, DEFAULT_UNCOVERED_INTERVALS_LIMIT); diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java index 11250714a84..620fbfe77c6 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java @@ -101,14 +101,13 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory final ValueType valueType = rowSignature.getColumnType(columnName).orElse(null); // Do _not_ set isDictionaryEncoded or hasBitmapIndexes, because Row-based columns do not have those things. - // Do set hasMultipleValues, because we might return multiple values. + // Do not set hasMultipleValues, because even though we might return multiple values, setting it affirmatively + // causes expression selectors to always treat us as arrays. If we might have multiple values (i.e. if our type + // is nonnumeric), set isComplete false to compensate. if (valueType != null) { return new ColumnCapabilitiesImpl() .setType(valueType) - // Non-numeric types might have multiple values - .setHasMultipleValues(!valueType.isNumeric()) - // Numeric types should be reported as complete, but not STRING or COMPLEX (because we don't have full info) .setIsComplete(valueType.isNumeric()); } else { diff --git a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java index d6b46d3d913..ce603514237 100644 --- a/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/lookup/LookupSegmentTest.java @@ -137,7 +137,7 @@ public class LookupSegmentTest // Note: the "k" column does not actually have multiple values, but the RowBasedStorageAdapter doesn't allow // reporting complete single-valued capabilities. It would be good to change this in the future, so query engines // running on top of lookups can take advantage of singly-valued optimizations. - Assert.assertTrue(capabilities.hasMultipleValues()); + Assert.assertFalse(capabilities.hasMultipleValues()); Assert.assertFalse(capabilities.isDictionaryEncoded()); Assert.assertFalse(capabilities.isComplete()); } @@ -151,7 +151,7 @@ public class LookupSegmentTest // reporting complete single-valued capabilities. It would be good to change this in the future, so query engines // running on top of lookups can take advantage of singly-valued optimizations. Assert.assertEquals(ValueType.STRING, capabilities.getType()); - Assert.assertTrue(capabilities.hasMultipleValues()); + Assert.assertFalse(capabilities.hasMultipleValues()); Assert.assertFalse(capabilities.isDictionaryEncoded()); Assert.assertFalse(capabilities.isComplete()); } diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index 185c4ef26fe..a4fc2f254b7 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -366,10 +366,10 @@ public class RowBasedStorageAdapterTest final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.STRING.name()); Assert.assertEquals(ValueType.STRING, capabilities.getType()); - // Note: unlike numeric types, STRING-typed columns report that they might have multiple values and that they - // are incomplete. It would be good in the future to support some way of changing this, when it is known ahead - // of time that multi-valuedness is impossible. - Assert.assertTrue(capabilities.hasMultipleValues()); + // Note: unlike numeric types, STRING-typed columns might have multiple values, so they report as incomplete. It + // would be good in the future to support some way of changing this, when it is known ahead of time that + // multi-valuedness is definitely happening or is definitely impossible. + Assert.assertFalse(capabilities.hasMultipleValues()); Assert.assertFalse(capabilities.isComplete()); } @@ -380,10 +380,9 @@ public class RowBasedStorageAdapterTest final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.COMPLEX.name()); - // Note: unlike numeric types, COMPLEX-typed columns report that they might have multiple values and that they - // are incomplete. + // Note: unlike numeric types, COMPLEX-typed columns report that they are incomplete. Assert.assertEquals(ValueType.COMPLEX, capabilities.getType()); - Assert.assertTrue(capabilities.hasMultipleValues()); + Assert.assertFalse(capabilities.hasMultipleValues()); Assert.assertFalse(capabilities.isComplete()); } 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 f3ef3391be5..ae4dc1689cc 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -100,7 +100,10 @@ import java.util.function.UnaryOperator; import java.util.stream.Collectors; /** + * This is the class on the Broker that is responsible for making native Druid queries to a cluster of data servers. * + * The main user of this class is {@link org.apache.druid.server.ClientQuerySegmentWalker}. In tests, its behavior + * is partially mimicked by TestClusterQuerySegmentWalker. */ public class CachingClusteredClient implements QuerySegmentWalker { @@ -480,7 +483,10 @@ public class CachingClusteredClient implements QuerySegmentWalker return Collections.emptyList(); } final List> alreadyCachedResults = new ArrayList<>(); - Map perSegmentCacheKeys = computePerSegmentCacheKeys(segments, queryCacheKey); + Map perSegmentCacheKeys = computePerSegmentCacheKeys( + segments, + queryCacheKey + ); // Pull cached segments from cache and remove from set of segments to query final Map cachedValues = computeCachedValues(perSegmentCacheKeys); diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java index 2f285244c7e..c3379bab021 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java @@ -35,6 +35,7 @@ import org.apache.druid.query.FluentQueryRunnerBuilder; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.PostProcessingOperator; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; @@ -137,7 +138,14 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker final QueryToolChest> toolChest = warehouse.getToolChest(query); // First, do an inlining dry run to see if any inlining is necessary, without actually running the queries. - final DataSource inlineDryRun = inlineIfNecessary(query.getDataSource(), toolChest, new AtomicInteger(), true); + final int maxSubqueryRows = QueryContexts.getMaxSubqueryRows(query, serverConfig.getMaxSubqueryRows()); + final DataSource inlineDryRun = inlineIfNecessary( + query.getDataSource(), + toolChest, + new AtomicInteger(), + maxSubqueryRows, + true + ); if (!canRunQueryUsingClusterWalker(query.withDataSource(inlineDryRun)) && !canRunQueryUsingLocalWalker(query.withDataSource(inlineDryRun))) { @@ -151,6 +159,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker query.getDataSource(), toolChest, new AtomicInteger(), + maxSubqueryRows, false ) ); @@ -245,6 +254,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker final DataSource dataSource, @Nullable final QueryToolChest toolChestIfOutermost, final AtomicInteger subqueryRowLimitAccumulator, + final int maxSubqueryRows, final boolean dryRun ) { @@ -266,7 +276,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker assert !(current instanceof QueryDataSource); - current = inlineIfNecessary(current, null, subqueryRowLimitAccumulator, dryRun); + current = inlineIfNecessary(current, null, subqueryRowLimitAccumulator, maxSubqueryRows, dryRun); while (!stack.isEmpty()) { current = stack.pop().withChildren(Collections.singletonList(current)); @@ -279,7 +289,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker } else { // Something happened during inlining that means the toolchest is no longer able to handle this subquery. // We need to consider inlining it. - return inlineIfNecessary(current, toolChestIfOutermost, subqueryRowLimitAccumulator, dryRun); + return inlineIfNecessary(current, toolChestIfOutermost, subqueryRowLimitAccumulator, maxSubqueryRows, dryRun); } } else if (canRunQueryUsingLocalWalker(subQuery) || canRunQueryUsingClusterWalker(subQuery)) { // Subquery needs to be inlined. Assign it a subquery id and run it. @@ -299,7 +309,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker queryResults, warehouse.getToolChest(subQueryWithId), subqueryRowLimitAccumulator, - serverConfig.getMaxSubqueryRows() + maxSubqueryRows ); } else { // Cannot inline subquery. Attempt to inline one level deeper, and then try again. @@ -310,12 +320,14 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker Iterables.getOnlyElement(dataSource.getChildren()), null, subqueryRowLimitAccumulator, + maxSubqueryRows, dryRun ) ) ), toolChestIfOutermost, subqueryRowLimitAccumulator, + maxSubqueryRows, dryRun ); } @@ -324,7 +336,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker return dataSource.withChildren( dataSource.getChildren() .stream() - .map(child -> inlineIfNecessary(child, null, subqueryRowLimitAccumulator, dryRun)) + .map(child -> inlineIfNecessary(child, null, subqueryRowLimitAccumulator, maxSubqueryRows, dryRun)) .collect(Collectors.toList()) ); } 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 3e8fd547ca0..ca26e0dd4f7 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 @@ -75,6 +75,8 @@ import java.util.function.Function; /** * Query handler for Historical processes (see CliHistorical). + * + * In tests, this class's behavior is partially mimicked by TestClusterQuerySegmentWalker. */ public class ServerManager implements QuerySegmentWalker { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java index 61ad6b29d5c..34ef9afe365 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerConfig.java @@ -35,15 +35,9 @@ public class PlannerConfig @JsonProperty private Period metadataRefreshPeriod = new Period("PT1M"); - @JsonProperty - private int maxSemiJoinRowsInMemory = 100000; - @JsonProperty private int maxTopNLimit = 100000; - @JsonProperty - private int maxQueryCount = 8; - @JsonProperty private boolean useApproximateCountDistinct = true; @@ -82,21 +76,11 @@ public class PlannerConfig return metadataRefreshPeriod; } - public int getMaxSemiJoinRowsInMemory() - { - return maxSemiJoinRowsInMemory; - } - public int getMaxTopNLimit() { return maxTopNLimit; } - public int getMaxQueryCount() - { - return maxQueryCount; - } - public boolean isUseApproximateCountDistinct() { return useApproximateCountDistinct; @@ -135,9 +119,7 @@ public class PlannerConfig final PlannerConfig newConfig = new PlannerConfig(); newConfig.metadataRefreshPeriod = getMetadataRefreshPeriod(); - newConfig.maxSemiJoinRowsInMemory = getMaxSemiJoinRowsInMemory(); newConfig.maxTopNLimit = getMaxTopNLimit(); - newConfig.maxQueryCount = getMaxQueryCount(); newConfig.useApproximateCountDistinct = getContextBoolean( context, CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT, @@ -185,9 +167,7 @@ public class PlannerConfig return false; } final PlannerConfig that = (PlannerConfig) o; - return maxSemiJoinRowsInMemory == that.maxSemiJoinRowsInMemory && - maxTopNLimit == that.maxTopNLimit && - maxQueryCount == that.maxQueryCount && + return maxTopNLimit == that.maxTopNLimit && useApproximateCountDistinct == that.useApproximateCountDistinct && useApproximateTopN == that.useApproximateTopN && requireTimeCondition == that.requireTimeCondition && @@ -205,9 +185,7 @@ public class PlannerConfig return Objects.hash( metadataRefreshPeriod, - maxSemiJoinRowsInMemory, maxTopNLimit, - maxQueryCount, useApproximateCountDistinct, useApproximateTopN, requireTimeCondition, @@ -224,9 +202,7 @@ public class PlannerConfig { return "PlannerConfig{" + "metadataRefreshPeriod=" + metadataRefreshPeriod + - ", maxSemiJoinRowsInMemory=" + maxSemiJoinRowsInMemory + ", maxTopNLimit=" + maxTopNLimit + - ", maxQueryCount=" + maxQueryCount + ", useApproximateCountDistinct=" + useApproximateCountDistinct + ", useApproximateTopN=" + useApproximateTopN + ", requireTimeCondition=" + requireTimeCondition + diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Rules.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Rules.java index bcc717f2f8c..05dd405b505 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Rules.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Rules.java @@ -40,14 +40,18 @@ import org.apache.calcite.rel.rules.AggregateRemoveRule; import org.apache.calcite.rel.rules.AggregateStarTableRule; import org.apache.calcite.rel.rules.AggregateValuesRule; import org.apache.calcite.rel.rules.CalcRemoveRule; +import org.apache.calcite.rel.rules.ExchangeRemoveConstantKeysRule; import org.apache.calcite.rel.rules.FilterAggregateTransposeRule; import org.apache.calcite.rel.rules.FilterJoinRule; import org.apache.calcite.rel.rules.FilterMergeRule; import org.apache.calcite.rel.rules.FilterProjectTransposeRule; import org.apache.calcite.rel.rules.FilterTableScanRule; +import org.apache.calcite.rel.rules.IntersectToDistinctRule; import org.apache.calcite.rel.rules.JoinCommuteRule; +import org.apache.calcite.rel.rules.JoinProjectTransposeRule; import org.apache.calcite.rel.rules.JoinPushExpressionsRule; import org.apache.calcite.rel.rules.JoinPushThroughJoinRule; +import org.apache.calcite.rel.rules.MatchRule; import org.apache.calcite.rel.rules.ProjectFilterTransposeRule; import org.apache.calcite.rel.rules.ProjectMergeRule; import org.apache.calcite.rel.rules.ProjectRemoveRule; @@ -58,9 +62,9 @@ import org.apache.calcite.rel.rules.PruneEmptyRules; import org.apache.calcite.rel.rules.ReduceExpressionsRule; import org.apache.calcite.rel.rules.SortJoinTransposeRule; import org.apache.calcite.rel.rules.SortProjectTransposeRule; +import org.apache.calcite.rel.rules.SortRemoveConstantKeysRule; import org.apache.calcite.rel.rules.SortRemoveRule; import org.apache.calcite.rel.rules.SortUnionTransposeRule; -import org.apache.calcite.rel.rules.SubQueryRemoveRule; import org.apache.calcite.rel.rules.TableScanRule; import org.apache.calcite.rel.rules.UnionMergeRule; import org.apache.calcite.rel.rules.UnionPullUpConstantsRule; @@ -74,7 +78,6 @@ import org.apache.calcite.tools.RelBuilder; import org.apache.druid.sql.calcite.rel.QueryMaker; import org.apache.druid.sql.calcite.rule.DruidRelToDruidRule; import org.apache.druid.sql.calcite.rule.DruidRules; -import org.apache.druid.sql.calcite.rule.DruidSemiJoinRule; import org.apache.druid.sql.calcite.rule.DruidTableScanRule; import org.apache.druid.sql.calcite.rule.ProjectAggregatePruneUnusedCallRule; import org.apache.druid.sql.calcite.rule.SortCollapseRule; @@ -86,9 +89,13 @@ public class Rules public static final int DRUID_CONVENTION_RULES = 0; public static final int BINDABLE_CONVENTION_RULES = 1; - // Rules from CalcitePrepareImpl's DEFAULT_RULES, minus AggregateExpandDistinctAggregatesRule - // and AggregateReduceFunctionsRule. - private static final List DEFAULT_RULES = + // Rules from RelOptUtil's registerBaseRules, minus: + // + // 1) AggregateExpandDistinctAggregatesRule (it'll be added back later if approximate count distinct is disabled) + // 2) AggregateReduceFunctionsRule (it'll be added back for the Bindable rule set, but we don't want it for Druid + // rules since it expands AVG, STDDEV, VAR, etc, and we have aggregators specifically designed for those + // functions). + private static final List BASE_RULES = ImmutableList.of( AggregateStarTableRule.INSTANCE, AggregateStarTableRule.INSTANCE2, @@ -99,40 +106,70 @@ public class Rules FilterProjectTransposeRule.INSTANCE, FilterJoinRule.FILTER_ON_JOIN, JoinPushExpressionsRule.INSTANCE, + AggregateCaseToFilterRule.INSTANCE, FilterAggregateTransposeRule.INSTANCE, ProjectWindowTransposeRule.INSTANCE, + MatchRule.INSTANCE, JoinCommuteRule.SWAP_OUTER, JoinPushThroughJoinRule.RIGHT, JoinPushThroughJoinRule.LEFT, SortProjectTransposeRule.INSTANCE, SortJoinTransposeRule.INSTANCE, - SortUnionTransposeRule.INSTANCE + SortRemoveConstantKeysRule.INSTANCE, + SortUnionTransposeRule.INSTANCE, + ExchangeRemoveConstantKeysRule.EXCHANGE_INSTANCE, + ExchangeRemoveConstantKeysRule.SORT_EXCHANGE_INSTANCE ); - // Rules from CalcitePrepareImpl's createPlanner. - private static final List MISCELLANEOUS_RULES = + // Rules for scanning via Bindable, embedded directly in RelOptUtil's registerDefaultRules. + private static final List DEFAULT_BINDABLE_RULES = ImmutableList.of( Bindables.BINDABLE_TABLE_SCAN_RULE, ProjectTableScanRule.INSTANCE, ProjectTableScanRule.INTERPRETER ); - // Rules from CalcitePrepareImpl's CONSTANT_REDUCTION_RULES. - private static final List CONSTANT_REDUCTION_RULES = + // Rules from RelOptUtil's registerReductionRules. + private static final List REDUCTION_RULES = ImmutableList.of( ReduceExpressionsRule.PROJECT_INSTANCE, - ReduceExpressionsRule.CALC_INSTANCE, - ReduceExpressionsRule.JOIN_INSTANCE, ReduceExpressionsRule.FILTER_INSTANCE, + ReduceExpressionsRule.CALC_INSTANCE, + ReduceExpressionsRule.WINDOW_INSTANCE, + ReduceExpressionsRule.JOIN_INSTANCE, ValuesReduceRule.FILTER_INSTANCE, ValuesReduceRule.PROJECT_FILTER_INSTANCE, ValuesReduceRule.PROJECT_INSTANCE, AggregateValuesRule.INSTANCE ); - // Rules from VolcanoPlanner's registerAbstractRelationalRules, minus JoinCommuteRule since it's already - // in DEFAULT_RULES. - private static final List VOLCANO_ABSTRACT_RULES = + // Rules from RelOptUtil's registerAbstractRules. + // Omit DateRangeRules due to https://issues.apache.org/jira/browse/CALCITE-1601 + private static final List ABSTRACT_RULES = + ImmutableList.of( + AggregateProjectPullUpConstantsRule.INSTANCE2, + UnionPullUpConstantsRule.INSTANCE, + PruneEmptyRules.UNION_INSTANCE, + PruneEmptyRules.INTERSECT_INSTANCE, + PruneEmptyRules.MINUS_INSTANCE, + PruneEmptyRules.PROJECT_INSTANCE, + PruneEmptyRules.FILTER_INSTANCE, + PruneEmptyRules.SORT_INSTANCE, + PruneEmptyRules.AGGREGATE_INSTANCE, + PruneEmptyRules.JOIN_LEFT_INSTANCE, + PruneEmptyRules.JOIN_RIGHT_INSTANCE, + PruneEmptyRules.SORT_FETCH_ZERO_INSTANCE, + UnionMergeRule.INSTANCE, + UnionMergeRule.INTERSECT_INSTANCE, + UnionMergeRule.MINUS_INSTANCE, + ProjectToWindowRule.PROJECT, + FilterMergeRule.INSTANCE, + IntersectToDistinctRule.INSTANCE + ); + + // Rules from RelOptUtil's registerAbstractRelationalRules, except AggregateMergeRule. (It causes + // testDoubleNestedGroupBy2 to fail). + private static final List ABSTRACT_RELATIONAL_RULES = ImmutableList.of( FilterJoinRule.FILTER_ON_JOIN, FilterJoinRule.JOIN, @@ -146,30 +183,11 @@ public class Rules SortRemoveRule.INSTANCE ); - // Rules from RelOptUtil's registerAbstractRels. - // Omit DateRangeRules due to https://issues.apache.org/jira/browse/CALCITE-1601 - private static final List RELOPTUTIL_ABSTRACT_RULES = + // Rules that pull projections up above a join. This lets us eliminate some subqueries. + private static final List JOIN_PROJECT_TRANSPOSE_RULES = ImmutableList.of( - AggregateProjectPullUpConstantsRule.INSTANCE2, - UnionPullUpConstantsRule.INSTANCE, - PruneEmptyRules.UNION_INSTANCE, - PruneEmptyRules.PROJECT_INSTANCE, - PruneEmptyRules.FILTER_INSTANCE, - PruneEmptyRules.SORT_INSTANCE, - PruneEmptyRules.AGGREGATE_INSTANCE, - PruneEmptyRules.JOIN_LEFT_INSTANCE, - PruneEmptyRules.JOIN_RIGHT_INSTANCE, - PruneEmptyRules.SORT_FETCH_ZERO_INSTANCE, - UnionMergeRule.INSTANCE, - ProjectToWindowRule.PROJECT, - FilterMergeRule.INSTANCE - ); - - private static final List SUB_QUERY_REMOVE_RULES = - ImmutableList.of( - SubQueryRemoveRule.PROJECT, - SubQueryRemoveRule.FILTER, - SubQueryRemoveRule.JOIN + JoinProjectTransposeRule.RIGHT_PROJECT, + JoinProjectTransposeRule.LEFT_PROJECT ); private Rules() @@ -182,7 +200,8 @@ public class Rules final Program hepProgram = Programs.sequence( Programs.subQuery(DefaultRelMetadataProvider.INSTANCE), - new DecorrelateAndTrimFieldsProgram() + new DecorrelateAndTrimFieldsProgram(), + Programs.hep(REDUCTION_RULES, true, DefaultRelMetadataProvider.INSTANCE) ); return ImmutableList.of( Programs.sequence(hepProgram, Programs.ofRules(druidConventionRuleSet(plannerContext, queryMaker))), @@ -201,10 +220,6 @@ public class Rules .add(new DruidTableScanRule(queryMaker)) .addAll(DruidRules.rules()); - if (plannerContext.getPlannerConfig().getMaxSemiJoinRowsInMemory() > 0) { - retVal.add(DruidSemiJoinRule.instance()); - } - return retVal.build(); } @@ -213,6 +228,7 @@ public class Rules return ImmutableList.builder() .addAll(baseRuleSet(plannerContext)) .addAll(Bindables.RULES) + .addAll(DEFAULT_BINDABLE_RULES) .add(AggregateReduceFunctionsRule.INSTANCE) .build(); } @@ -223,12 +239,10 @@ public class Rules final ImmutableList.Builder rules = ImmutableList.builder(); // Calcite rules. - rules.addAll(DEFAULT_RULES); - rules.addAll(MISCELLANEOUS_RULES); - rules.addAll(CONSTANT_REDUCTION_RULES); - rules.addAll(VOLCANO_ABSTRACT_RULES); - rules.addAll(RELOPTUTIL_ABSTRACT_RULES); - rules.addAll(SUB_QUERY_REMOVE_RULES); + rules.addAll(BASE_RULES); + rules.addAll(ABSTRACT_RULES); + rules.addAll(ABSTRACT_RELATIONAL_RULES); + rules.addAll(JOIN_PROJECT_TRANSPOSE_RULES); if (!plannerConfig.isUseApproximateCountDistinct()) { // For some reason, even though we support grouping sets, using AggregateExpandDistinctAggregatesRule.INSTANCE @@ -236,8 +250,8 @@ public class Rules rules.add(AggregateExpandDistinctAggregatesRule.JOIN); } + // Rules that we wrote. rules.add(SortCollapseRule.instance()); - rules.add(AggregateCaseToFilterRule.INSTANCE); rules.add(ProjectAggregatePruneUnusedCallRule.instance()); return rules.build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/CostEstimates.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/CostEstimates.java new file mode 100644 index 00000000000..c3b7c14946c --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/CostEstimates.java @@ -0,0 +1,89 @@ +/* + * 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.sql.calcite.rel; + +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.rel.metadata.RelMetadataQuery; + +/** + * Constants used by {@link PartialDruidQuery#estimateCost} and various + * {@link DruidRel#computeSelfCost(RelOptPlanner, RelMetadataQuery)} implementations. + */ +public class CostEstimates +{ + /** + * Per-row base cost. This represents the cost of walking through every row, but not actually reading anything + * from them or computing any aggregations. + */ + static final double COST_BASE = 1; + + /** + * Cost to read a value out of a column directly. + */ + static final double COST_COLUMN_READ = 0.05; + + /** + * Cost to compute and read an expression. + */ + static final double COST_EXPRESSION = 0.25; + + /** + * Cost to compute an aggregation. + */ + static final double COST_AGGREGATION = 0.05; + + /** + * Cost per GROUP BY dimension. + */ + static final double COST_DIMENSION = 0.25; + + /** + * Multiplier to apply when there is a WHERE filter. Encourages pushing down filters and limits through joins and + * subqueries when possible. + */ + static final double MULTIPLIER_FILTER = 0.1; + + /** + * Multiplier to apply when there is an ORDER BY. Encourages avoiding them when possible. + */ + static final double MULTIPLIER_ORDER_BY = 10; + + /** + * Multiplier to apply when there is a LIMIT. Encourages pushing down limits when possible. + */ + static final double MULTIPLIER_LIMIT = 0.5; + + /** + * Multiplier to apply to an outer query via {@link DruidOuterQueryRel}. Encourages pushing down time-saving + * operations to the lowest level of the query stack, because they'll have bigger impact there. + */ + static final double MULTIPLIER_OUTER_QUERY = 0.1; + + /** + * Multiplier to apply to a join when the left-hand side is a subquery. Encourages avoiding subqueries. Subqueries + * inside joins must be inlined, which incurs substantial reduction in scalability, so this high number is justified. + */ + static final double MULTIPLIER_JOIN_SUBQUERY = 1000000000; + + private CostEstimates() + { + // No instantiation. + } +} 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 0e9094bc85c..f52c844b24f 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 @@ -60,17 +60,38 @@ import java.util.stream.Collectors; public class DruidJoinQueryRel extends DruidRel { private static final TableDataSource DUMMY_DATA_SOURCE = new TableDataSource("__join__"); - private static final double COST_FACTOR = 100.0; private final PartialDruidQuery partialQuery; private final Join joinRel; private RelNode left; private RelNode right; + /** + * True if {@link #left} requires a subquery. + * + * This is useful to store in a variable because {@link #left} is sometimes not actually a {@link DruidRel} when + * {@link #computeSelfCost} is called. (It might be a {@link org.apache.calcite.plan.volcano.RelSubset}.) + * + * @see #computeLeftRequiresSubquery(DruidRel) + */ + private final boolean leftRequiresSubquery; + + /** + * True if {@link #right} requires a subquery. + * + * This is useful to store in a variable because {@link #left} is sometimes not actually a {@link DruidRel} when + * {@link #computeSelfCost} is called. (It might be a {@link org.apache.calcite.plan.volcano.RelSubset}.) + * + * @see #computeLeftRequiresSubquery(DruidRel) + */ + private final boolean rightRequiresSubquery; + private DruidJoinQueryRel( RelOptCluster cluster, RelTraitSet traitSet, Join joinRel, + boolean leftRequiresSubquery, + boolean rightRequiresSubquery, PartialDruidQuery partialQuery, QueryMaker queryMaker ) @@ -79,17 +100,28 @@ public class DruidJoinQueryRel extends DruidRel this.joinRel = joinRel; this.left = joinRel.getLeft(); this.right = joinRel.getRight(); + this.leftRequiresSubquery = leftRequiresSubquery; + this.rightRequiresSubquery = rightRequiresSubquery; this.partialQuery = partialQuery; } - public static DruidJoinQueryRel create(final Join joinRel, final QueryMaker queryMaker) + /** + * Create an instance from a Join that is based on two {@link DruidRel} inputs. + */ + public static DruidJoinQueryRel create( + final Join joinRel, + final DruidRel left, + final DruidRel right + ) { return new DruidJoinQueryRel( joinRel.getCluster(), joinRel.getTraitSet(), joinRel, + computeLeftRequiresSubquery(left), + computeRightRequiresSubquery(right), PartialDruidQuery.create(joinRel), - queryMaker + left.getQueryMaker() ); } @@ -117,6 +149,8 @@ public class DruidJoinQueryRel extends DruidRel getCluster(), getTraitSet().plusAll(newQueryBuilder.getRelTraits()), joinRel, + leftRequiresSubquery, + rightRequiresSubquery, newQueryBuilder, getQueryMaker() ); @@ -141,18 +175,20 @@ public class DruidJoinQueryRel extends DruidRel final RowSignature rightSignature = rightQuery.getOutputRowSignature(); final DataSource rightDataSource; - // Left rel: allow direct embedding of scans/mappings including those of joins. - if (DruidRels.isScanOrMapping(leftDruidRel, true)) { - leftDataSource = leftQuery.getDataSource(); - } else { + if (computeLeftRequiresSubquery(leftDruidRel)) { + assert leftRequiresSubquery; leftDataSource = new QueryDataSource(leftQuery.getQuery()); + } else { + assert !leftRequiresSubquery; + leftDataSource = leftQuery.getDataSource(); } - // Right rel: allow direct embedding of scans/mappings, excluding joins (those must be done as subqueries). - if (DruidRels.isScanOrMapping(rightDruidRel, false)) { - rightDataSource = rightQuery.getDataSource(); - } else { + if (computeRightRequiresSubquery(rightDruidRel)) { + assert rightRequiresSubquery; rightDataSource = new QueryDataSource(rightQuery.getQuery()); + } else { + assert !rightRequiresSubquery; + rightDataSource = rightQuery.getDataSource(); } final Pair prefixSignaturePair = computeJoinRowSignature(leftSignature, rightSignature); @@ -214,6 +250,8 @@ public class DruidJoinQueryRel extends DruidRel .map(input -> RelOptRule.convert(input, DruidConvention.instance())) .collect(Collectors.toList()) ), + leftRequiresSubquery, + rightRequiresSubquery, partialQuery, getQueryMaker() ); @@ -252,6 +290,8 @@ public class DruidJoinQueryRel extends DruidRel getCluster(), traitSet, joinRel.copy(joinRel.getTraitSet(), inputs), + leftRequiresSubquery, + rightRequiresSubquery, getPartialDruidQuery(), getQueryMaker() ); @@ -297,9 +337,9 @@ public class DruidJoinQueryRel extends DruidRel public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadataQuery mq) { return planner.getCostFactory() - .makeCost(mq.getRowCount(left), 0, 0) - .plus(planner.getCostFactory().makeCost(mq.getRowCount(right), 0, 0)) - .multiplyBy(COST_FACTOR); + .makeCost(partialQuery.estimateCost(), 0, 0) + .multiplyBy(leftRequiresSubquery ? CostEstimates.MULTIPLIER_JOIN_SUBQUERY : 1) + .multiplyBy(rightRequiresSubquery ? CostEstimates.MULTIPLIER_JOIN_SUBQUERY : 1); } private static JoinType toDruidJoinType(JoinRelType calciteJoinType) @@ -318,6 +358,19 @@ public class DruidJoinQueryRel extends DruidRel } } + private static boolean computeLeftRequiresSubquery(final DruidRel left) + { + // Left requires a subquery unless it's a scan or mapping on top of any table or a join. + return !DruidRels.isScanOrMapping(left, true); + } + + private static boolean computeRightRequiresSubquery(final DruidRel right) + { + // Right requires a subquery unless it's a scan or mapping on top of a global datasource. + return !(DruidRels.isScanOrMapping(right, false) + && DruidRels.dataSourceIfLeafRel(right).filter(DataSource::isGlobal).isPresent()); + } + /** * Returns a Pair of "rightPrefix" (for JoinDataSource) and the signature of rows that will result from * applying that prefix. diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java index fde3457e53d..f601523f691 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java @@ -40,7 +40,6 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.table.RowSignatures; -import javax.annotation.Nullable; import java.util.List; import java.util.Set; @@ -120,16 +119,12 @@ public class DruidOuterQueryRel extends DruidRel return 1 + ((DruidRel) sourceRel).getQueryCount(); } - @Nullable @Override public DruidQuery toDruidQuery(final boolean finalizeAggregations) { // Must finalize aggregations on subqueries. final DruidQuery subQuery = ((DruidRel) sourceRel).toDruidQuery(true); - if (subQuery == null) { - return null; - } final GroupByQuery groupByQuery = subQuery.toGroupByQuery(); if (groupByQuery == null) { @@ -234,6 +229,8 @@ public class DruidOuterQueryRel extends DruidRel @Override public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadataQuery mq) { - return planner.getCostFactory().makeCost(mq.getRowCount(sourceRel), 0, 0).multiplyBy(10); + return planner.getCostFactory() + .makeCost(partialQuery.estimateCost(), 0, 0) + .multiplyBy(CostEstimates.MULTIPLIER_OUTER_QUERY); } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java index a6fa53cb4d0..7e8aa9d72db 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java @@ -34,7 +34,6 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.sql.calcite.table.DruidTable; -import javax.annotation.Nonnull; import java.util.Set; /** @@ -42,15 +41,6 @@ import java.util.Set; */ public class DruidQueryRel extends DruidRel { - // Factors used for computing cost (see computeSelfCost). These are intended to encourage pushing down filters - // and limits through stacks of nested queries when possible. - private static final double COST_BASE = 1.0; - private static final double COST_PER_COLUMN = 0.001; - private static final double COST_FILTER_MULTIPLIER = 0.1; - private static final double COST_GROUPING_MULTIPLIER = 0.5; - private static final double COST_LIMIT_MULTIPLIER = 0.5; - private static final double COST_HAVING_MULTIPLIER = 5.0; - private final RelOptTable table; private final DruidTable druidTable; private final PartialDruidQuery partialQuery; @@ -91,7 +81,6 @@ public class DruidQueryRel extends DruidRel } @Override - @Nonnull public DruidQuery toDruidQuery(final boolean finalizeAggregations) { return partialQuery.build( @@ -163,6 +152,11 @@ public class DruidQueryRel extends DruidRel return getQueryMaker().runQuery(toDruidQuery(false)); } + public DruidTable getDruidTable() + { + return druidTable; + } + @Override public RelOptTable getTable() { @@ -195,38 +189,6 @@ public class DruidQueryRel extends DruidRel @Override public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadataQuery mq) { - double cost = COST_BASE; - - if (partialQuery.getSelectProject() != null) { - cost += COST_PER_COLUMN * partialQuery.getSelectProject().getChildExps().size(); - } - - if (partialQuery.getWhereFilter() != null) { - cost *= COST_FILTER_MULTIPLIER; - } - - if (partialQuery.getAggregate() != null) { - cost *= COST_GROUPING_MULTIPLIER; - cost += COST_PER_COLUMN * partialQuery.getAggregate().getGroupSet().size(); - cost += COST_PER_COLUMN * partialQuery.getAggregate().getAggCallList().size(); - } - - if (partialQuery.getAggregateProject() != null) { - cost += COST_PER_COLUMN * partialQuery.getAggregateProject().getChildExps().size(); - } - - if (partialQuery.getSort() != null && partialQuery.getSort().fetch != null) { - cost *= COST_LIMIT_MULTIPLIER; - } - - if (partialQuery.getSortProject() != null) { - cost += COST_PER_COLUMN * partialQuery.getSortProject().getChildExps().size(); - } - - if (partialQuery.getHavingFilter() != null) { - cost *= COST_HAVING_MULTIPLIER; - } - - return planner.getCostFactory().makeCost(cost, 0, 0); + return planner.getCostFactory().makeCost(partialQuery.estimateCost(), 0, 0); } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java index 9f56e2f5814..6ae232f1cb2 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidRel.java @@ -68,30 +68,23 @@ public abstract class DruidRel extends AbstractRelNode } /** - * Convert this DruidRel to a DruidQuery. This may be an expensive operation. For example, DruidSemiJoin needs to - * execute the right-hand side query in order to complete this method. + * Convert this DruidRel to a DruidQuery. This must be an inexpensive operation, since it is done often throughout + * query planning. * - * This method may return null if it knows that this rel will yield an empty result set. + * This method must not return null. * * @param finalizeAggregations true if this query should include explicit finalization for all of its * aggregators, where required. Useful for subqueries where Druid's native query layer * does not do this automatically. * - * @return query, or null if it is known in advance that this rel will yield an empty result set. - * * @throws CannotBuildQueryException */ - @Nullable public abstract DruidQuery toDruidQuery(boolean finalizeAggregations); /** - * Convert this DruidRel to a DruidQuery for purposes of explaining. This must be an inexpensive operation. For - * example, DruidSemiJoin will use a dummy dataSource in order to complete this method, rather than executing - * the right-hand side query. + * Convert this DruidRel to a DruidQuery for purposes of explaining. This must be an inexpensive operation. * - * This method may not return null. - * - * @return query + * This method must not return null. * * @throws CannotBuildQueryException */ diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidSemiJoin.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidSemiJoin.java deleted file mode 100644 index e31437b6962..00000000000 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidSemiJoin.java +++ /dev/null @@ -1,392 +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.sql.calcite.rel; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import org.apache.calcite.plan.RelOptCluster; -import org.apache.calcite.plan.RelOptCost; -import org.apache.calcite.plan.RelOptPlanner; -import org.apache.calcite.plan.RelOptRule; -import org.apache.calcite.plan.RelTraitSet; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.RelWriter; -import org.apache.calcite.rel.core.Filter; -import org.apache.calcite.rel.logical.LogicalFilter; -import org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rex.RexInputRef; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexUtil; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.guava.Accumulator; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.ResourceLimitExceededException; -import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.sql.calcite.planner.PlannerContext; - -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Set; - -/** - * DruidRel that has a main query, and also a subquery "right" that is used to filter the main query. - */ -public class DruidSemiJoin extends DruidRel -{ - private final List leftExpressions; - private final List rightKeys; - private final int maxSemiJoinRowsInMemory; - private DruidRel left; - private RelNode right; - - private DruidSemiJoin( - final RelOptCluster cluster, - final RelTraitSet traitSet, - final DruidRel left, - final RelNode right, - final List leftExpressions, - final List rightKeys, - final int maxSemiJoinRowsInMemory, - final QueryMaker queryMaker - ) - { - super(cluster, traitSet, queryMaker); - this.left = left; - this.right = right; - this.leftExpressions = ImmutableList.copyOf(leftExpressions); - this.rightKeys = ImmutableList.copyOf(rightKeys); - this.maxSemiJoinRowsInMemory = maxSemiJoinRowsInMemory; - } - - public static DruidSemiJoin create( - final DruidRel left, - final DruidRel right, - final List leftKeys, - final List rightKeys, - final PlannerContext plannerContext - ) - { - final ImmutableList.Builder listBuilder = ImmutableList.builder(); - - final PartialDruidQuery leftPartialQuery = left.getPartialDruidQuery(); - if (leftPartialQuery.stage().compareTo(PartialDruidQuery.Stage.AGGREGATE) >= 0) { - throw new ISE("LHS must not be an Aggregate"); - } - - if (leftPartialQuery.getSelectProject() != null) { - for (int key : leftKeys) { - listBuilder.add(leftPartialQuery.getSelectProject().getChildExps().get(key)); - } - } else { - for (int key : leftKeys) { - listBuilder.add(RexInputRef.of(key, leftPartialQuery.getRowType())); - } - } - - return new DruidSemiJoin( - left.getCluster(), - left.getTraitSet(), - left, - right, - listBuilder.build(), - rightKeys, - plannerContext.getPlannerConfig().getMaxSemiJoinRowsInMemory(), - left.getQueryMaker() - ); - } - - @Override - public PartialDruidQuery getPartialDruidQuery() - { - return left.getPartialDruidQuery(); - } - - @Override - public DruidSemiJoin withPartialQuery(final PartialDruidQuery newQueryBuilder) - { - return new DruidSemiJoin( - getCluster(), - getTraitSet().plusAll(newQueryBuilder.getRelTraits()), - left.withPartialQuery(newQueryBuilder), - right, - leftExpressions, - rightKeys, - maxSemiJoinRowsInMemory, - getQueryMaker() - ); - } - - @Nullable - @Override - public DruidQuery toDruidQuery(final boolean finalizeAggregations) - { - final DruidRel rel = getLeftRelWithFilter(); - return rel != null ? rel.toDruidQuery(finalizeAggregations) : null; - } - - @Override - public DruidQuery toDruidQueryForExplaining() - { - return left.toDruidQueryForExplaining(); - } - - @Override - public DruidSemiJoin asDruidConvention() - { - return new DruidSemiJoin( - getCluster(), - getTraitSet().replace(DruidConvention.instance()), - left, - RelOptRule.convert(right, DruidConvention.instance()), - leftExpressions, - rightKeys, - maxSemiJoinRowsInMemory, - getQueryMaker() - ); - } - - @Override - public Set getDataSourceNames() - { - final DruidRel druidRight = (DruidRel) this.right; - Set dataSourceNames = new LinkedHashSet<>(); - dataSourceNames.addAll(left.getDataSourceNames()); - dataSourceNames.addAll(druidRight.getDataSourceNames()); - return dataSourceNames; - } - - @Override - public int getQueryCount() - { - return left.getQueryCount() + ((DruidRel) right).getQueryCount(); - } - - @Override - public Sequence runQuery() - { - final DruidRel rel = getLeftRelWithFilter(); - if (rel != null) { - return rel.runQuery(); - } else { - return Sequences.empty(); - } - } - - @Override - protected RelDataType deriveRowType() - { - return left.getRowType(); - } - - @Override - public List getInputs() - { - return ImmutableList.of(right); - } - - @Override - public void replaceInput(int ordinalInParent, RelNode p) - { - if (ordinalInParent != 0) { - throw new IndexOutOfBoundsException(StringUtils.format("Invalid ordinalInParent[%s]", ordinalInParent)); - } - // 'right' is the only one Calcite concerns. See getInputs(). - this.right = p; - } - - @Override - public RelNode copy(final RelTraitSet traitSet, final List inputs) - { - return new DruidSemiJoin( - getCluster(), - getTraitSet(), - left, - Iterables.getOnlyElement(inputs), - leftExpressions, - rightKeys, - maxSemiJoinRowsInMemory, - getQueryMaker() - ); - } - - @Override - public RelWriter explainTerms(RelWriter pw) - { - final String queryString; - - try { - queryString = getQueryMaker().getJsonMapper().writeValueAsString(toDruidQueryForExplaining().getQuery()); - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - - return super.explainTerms(pw) - .input("right", right) - .item("query", queryString) - .item("leftExpressions", leftExpressions) - .item("rightKeys", rightKeys); - } - - @Override - public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadataQuery mq) - { - return right.computeSelfCost(planner, mq).plus(left.computeSelfCost(planner, mq).multiplyBy(50)); - } - - /** - * Returns a copy of the left rel with the filter applied from the right-hand side. This is an expensive operation - * since it actually executes the right-hand side query. - */ - private DruidRel getLeftRelWithFilter() - { - final DruidRel druidRight = (DruidRel) this.right; - - // Build list of acceptable values from right side. - final Set> valuess = new HashSet<>(); - final List conditions = druidRight.runQuery().accumulate( - new ArrayList<>(), - new Accumulator, Object[]>() - { - int numRows; - - @Override - public List accumulate(final List theConditions, final Object[] row) - { - final List values = new ArrayList<>(rightKeys.size()); - - for (int i : rightKeys) { - final Object value = row[i]; - if (value == null) { - // NULLs are not supposed to match NULLs in a join. So ignore them. - continue; - } - final String stringValue = DimensionHandlerUtils.convertObjectToString(value); - values.add(stringValue); - } - - if (valuess.add(values)) { - if (++numRows > maxSemiJoinRowsInMemory) { - throw new ResourceLimitExceededException( - StringUtils.format("maxSemiJoinRowsInMemory[%,d] exceeded", maxSemiJoinRowsInMemory) - ); - } - final List subConditions = new ArrayList<>(); - - for (int i = 0; i < values.size(); i++) { - final String value = values.get(i); - // NULLs are not supposed to match NULLs in a join. So ignore them. - if (value != null) { - subConditions.add( - getCluster().getRexBuilder().makeCall( - SqlStdOperatorTable.EQUALS, - leftExpressions.get(i), - getCluster().getRexBuilder().makeLiteral(value) - ) - ); - } - theConditions.add(makeAnd(subConditions)); - } - } - return theConditions; - } - } - ); - - valuess.clear(); - - if (!conditions.isEmpty()) { - // Add a filter to the left side. - final PartialDruidQuery leftPartialQuery = left.getPartialDruidQuery(); - final Filter whereFilter = leftPartialQuery.getWhereFilter(); - final Filter newWhereFilter; - - if (whereFilter != null) { - newWhereFilter = whereFilter.copy( - whereFilter.getTraitSet(), - whereFilter.getInput(), - RexUtil.flatten( - getCluster().getRexBuilder(), - makeAnd(ImmutableList.of(whereFilter.getCondition(), makeOr(conditions))) - ) - ); - } else { - newWhereFilter = LogicalFilter.create( - leftPartialQuery.getScan(), - makeOr(conditions) // already in flattened form - ); - } - - PartialDruidQuery newPartialQuery = PartialDruidQuery.create(leftPartialQuery.getScan()) - .withWhereFilter(newWhereFilter) - .withSelectProject(leftPartialQuery.getSelectProject()); - - if (leftPartialQuery.getAggregate() != null) { - newPartialQuery = newPartialQuery.withAggregate(leftPartialQuery.getAggregate()); - } - - if (leftPartialQuery.getHavingFilter() != null) { - newPartialQuery = newPartialQuery.withHavingFilter(leftPartialQuery.getHavingFilter()); - } - - if (leftPartialQuery.getAggregateProject() != null) { - newPartialQuery = newPartialQuery.withAggregateProject(leftPartialQuery.getAggregateProject()); - } - - if (leftPartialQuery.getSort() != null) { - newPartialQuery = newPartialQuery.withSort(leftPartialQuery.getSort()); - } - - if (leftPartialQuery.getSortProject() != null) { - newPartialQuery = newPartialQuery.withSortProject(leftPartialQuery.getSortProject()); - } - - return left.withPartialQuery(newPartialQuery); - } else { - return null; - } - } - - private RexNode makeAnd(final List conditions) - { - if (conditions.size() == 1) { - return Iterables.getOnlyElement(conditions); - } else { - return getCluster().getRexBuilder().makeCall(SqlStdOperatorTable.AND, conditions); - } - } - - private RexNode makeOr(final List conditions) - { - if (conditions.size() == 1) { - return Iterables.getOnlyElement(conditions); - } else { - return getCluster().getRexBuilder().makeCall(SqlStdOperatorTable.OR, conditions); - } - } -} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionRel.java index 2fe3d2ca027..0c2ebc8c289 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionRel.java @@ -115,7 +115,6 @@ public class DruidUnionRel extends DruidRel throw new UnsupportedOperationException(); } - @Nullable @Override public DruidQuery toDruidQuery(final boolean finalizeAggregations) { @@ -195,7 +194,7 @@ public class DruidUnionRel extends DruidRel @Override public RelOptCost computeSelfCost(final RelOptPlanner planner, final RelMetadataQuery mq) { - return planner.getCostFactory().makeCost(rels.stream().mapToDouble(mq::getRowCount).sum(), 0, 0); + return planner.getCostFactory().makeCost(CostEstimates.COST_BASE, 0, 0); } public int getLimit() diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java index 9605075c09c..73abd6977d6 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java @@ -32,6 +32,7 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexUtil; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.tools.RelBuilder; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.DataSource; @@ -396,6 +397,61 @@ public class PartialDruidQuery } } + /** + * Estimates the per-row cost of running this query. + */ + public double estimateCost() + { + double cost = CostEstimates.COST_BASE; + + if (getSelectProject() != null) { + for (final RexNode rexNode : getSelectProject().getChildExps()) { + if (rexNode.isA(SqlKind.INPUT_REF)) { + cost += CostEstimates.COST_COLUMN_READ; + } else { + cost += CostEstimates.COST_EXPRESSION; + } + } + } + + if (getWhereFilter() != null) { + // We assume filters are free and have a selectivity of CostEstimates.MULTIPLIER_FILTER. They aren't actually + // free, but we want to encourage filters, so let's go with it. + cost *= CostEstimates.MULTIPLIER_FILTER; + } + + if (getAggregate() != null) { + if (getSelectProject() == null) { + // No projection before aggregation, that means the aggregate operator is reading things directly. + // Account for the costs. + cost += CostEstimates.COST_COLUMN_READ * getAggregate().getGroupSet().size(); + } + + cost += CostEstimates.COST_DIMENSION * getAggregate().getGroupSet().size(); + cost += CostEstimates.COST_AGGREGATION * getAggregate().getAggCallList().size(); + } + + if (getSort() != null) { + if (!getSort().collation.getFieldCollations().isEmpty()) { + cost *= CostEstimates.MULTIPLIER_ORDER_BY; + } + + if (getSort().fetch != null) { + cost *= CostEstimates.MULTIPLIER_LIMIT; + } + } + + if (getAggregateProject() != null) { + cost += CostEstimates.COST_EXPRESSION * getAggregateProject().getChildExps().size(); + } + + if (getSortProject() != null) { + cost += CostEstimates.COST_EXPRESSION * getSortProject().getChildExps().size(); + } + + return cost; + } + private void validateStage(final Stage stage) { if (!canAccept(stage)) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java index 44a4705d082..be52552df9b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList; import org.apache.calcite.rel.core.Project; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.math.expr.ExprType; @@ -258,7 +259,12 @@ public class Projection for (int i = 0; i < expressions.size(); i++) { final DruidExpression expression = expressions.get(i); - if (expression.isDirectColumnAccess()) { + + final SqlTypeName sqlTypeName = project.getRowType().getFieldList().get(i).getType().getSqlTypeName(); + if (expression.isDirectColumnAccess() + && inputRowSignature.getColumnType(expression.getDirectColumn()).orElse(null) + == Calcites.getValueTypeForSqlTypeName(sqlTypeName)) { + // Refer to column directly when it's a direct access with matching type. rowOrder.add(expression.getDirectColumn()); } else { final VirtualColumn virtualColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidJoinRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidJoinRule.java index a7ba42e3ebd..58c65624403 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidJoinRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidJoinRule.java @@ -31,10 +31,8 @@ import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.util.ImmutableBitSet; -import org.apache.druid.query.DataSource; import org.apache.druid.sql.calcite.rel.DruidJoinQueryRel; import org.apache.druid.sql.calcite.rel.DruidRel; -import org.apache.druid.sql.calcite.rel.DruidRels; import java.util.ArrayList; import java.util.List; @@ -64,21 +62,12 @@ public class DruidJoinRule extends RelOptRule public boolean matches(RelOptRuleCall call) { final Join join = call.rel(0); - final DruidRel left = call.rel(1); final DruidRel right = call.rel(2); // 1) Condition must be handleable. - // 2) Left must be a scan or a join. - // 3) If left is not a join, it must be concrete. - // 4) Right must be a scan (and *cannot* be a join; we want to generate left-heavy trees). - // 5) Right must be global. - return - canHandleCondition(join.getCondition(), join.getLeft().getRowType()) - && DruidRels.isScanOrMapping(left, true) - && DruidRels.isScanOrMapping(right, false) - && (left instanceof DruidJoinQueryRel - || DruidRels.dataSourceIfLeafRel(left).filter(DataSource::isConcrete).isPresent()) - && DruidRels.dataSourceIfLeafRel(right).filter(DataSource::isGlobal).isPresent(); + // 2) Right cannot be a join; we want to generate left-heavy trees. + return canHandleCondition(join.getCondition(), join.getLeft().getRowType()) + && !(right instanceof DruidJoinQueryRel); } @Override @@ -86,11 +75,10 @@ public class DruidJoinRule extends RelOptRule { final Join join = call.rel(0); final DruidRel left = call.rel(1); + final DruidRel right = call.rel(2); // Preconditions were already verified in "matches". - call.transformTo( - DruidJoinQueryRel.create(join, left.getQueryMaker()) - ); + call.transformTo(DruidJoinQueryRel.create(join, left, right)); } /** diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidSemiJoinRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidSemiJoinRule.java deleted file mode 100644 index 97ea3539fdd..00000000000 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidSemiJoinRule.java +++ /dev/null @@ -1,179 +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.sql.calcite.rule; - -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import org.apache.calcite.plan.RelOptRule; -import org.apache.calcite.plan.RelOptRuleCall; -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.rel.core.Join; -import org.apache.calcite.rel.core.JoinInfo; -import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rex.RexInputRef; -import org.apache.calcite.rex.RexNode; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.tools.RelBuilder; -import org.apache.calcite.util.ImmutableBitSet; -import org.apache.druid.sql.calcite.planner.PlannerConfig; -import org.apache.druid.sql.calcite.rel.DruidRel; -import org.apache.druid.sql.calcite.rel.DruidSemiJoin; -import org.apache.druid.sql.calcite.rel.PartialDruidQuery; - -/** - * Planner rule adapted from Calcite 1.11.0's SemiJoinRule. - * - * This rule identifies a JOIN where the right-hand side is being used like a filter. Requirements are: - * - * 1) Right-hand side is grouping on the join key - * 2) No fields from the right-hand side are selected - * 3) Join is INNER (right-hand side acting as filter) or LEFT (right-hand side can be ignored) - * - * This is used instead of Calcite's built in rule because that rule's un-doing of aggregation is unproductive (we'd - * just want to add it back again). Also, this rule operates on DruidRels. - */ -public class DruidSemiJoinRule extends RelOptRule -{ - private static final Predicate IS_LEFT_OR_INNER = - join -> { - final JoinRelType joinType = join.getJoinType(); - return joinType == JoinRelType.LEFT || joinType == JoinRelType.INNER; - }; - - private static final Predicate IS_GROUP_BY = druidRel -> - druidRel.getPartialDruidQuery() != null && druidRel.getPartialDruidQuery().getAggregate() != null; - - private static final DruidSemiJoinRule INSTANCE = new DruidSemiJoinRule(); - - private DruidSemiJoinRule() - { - super( - operand( - Project.class, - operand( - Join.class, - null, - IS_LEFT_OR_INNER, - some( - operand( - DruidRel.class, - null, - Predicates.and(DruidRules.CAN_BUILD_ON, Predicates.not(IS_GROUP_BY)), - any() - ), - operand(DruidRel.class, null, IS_GROUP_BY, any()) - ) - ) - ) - ); - } - - public static DruidSemiJoinRule instance() - { - return INSTANCE; - } - - @Override - public void onMatch(RelOptRuleCall call) - { - final Project project = call.rel(0); - final Join join = call.rel(1); - final DruidRel left = call.rel(2); - final DruidRel right = call.rel(3); - - final ImmutableBitSet bits = - RelOptUtil.InputFinder.bits(project.getProjects(), null); - final ImmutableBitSet rightBits = - ImmutableBitSet.range( - left.getRowType().getFieldCount(), - join.getRowType().getFieldCount() - ); - - if (bits.intersects(rightBits)) { - return; - } - - final JoinInfo joinInfo = join.analyzeCondition(); - - // Rule requires that aggregate key to be the same as the join key. - // By the way, neither a super-set nor a sub-set would work. - - if (!joinInfo.isEqui() || - joinInfo.rightSet().cardinality() != right.getPartialDruidQuery().getAggregate().getGroupCount()) { - return; - } - - final PartialDruidQuery rightQuery = right.getPartialDruidQuery(); - final Project rightProject = rightQuery.getSortProject() != null ? - rightQuery.getSortProject() : - rightQuery.getAggregateProject(); - int i = 0; - for (int joinRef : joinInfo.rightSet()) { - final int aggregateRef; - - if (rightProject == null) { - aggregateRef = joinRef; - } else { - final RexNode projectExp = rightProject.getChildExps().get(joinRef); - if (projectExp.isA(SqlKind.INPUT_REF)) { - aggregateRef = ((RexInputRef) projectExp).getIndex(); - } else { - // Project expression is not part of the grouping key. - return; - } - } - - if (aggregateRef != i++) { - return; - } - } - - final RelBuilder relBuilder = call.builder(); - - if (join.getJoinType() == JoinRelType.LEFT) { - // Join can be eliminated since the right-hand side cannot have any effect (nothing is being selected, - // and LEFT means even if there is no match, a left-hand row will still be included). - relBuilder.push(left); - } else { - final DruidSemiJoin druidSemiJoin = DruidSemiJoin.create( - left, - right, - joinInfo.leftKeys, - joinInfo.rightKeys, - left.getPlannerContext() - ); - - // Check maxQueryCount. - final PlannerConfig plannerConfig = left.getPlannerContext().getPlannerConfig(); - if (plannerConfig.getMaxQueryCount() > 0 && druidSemiJoin.getQueryCount() > plannerConfig.getMaxQueryCount()) { - return; - } - - relBuilder.push(druidSemiJoin); - } - - call.transformTo( - relBuilder - .project(project.getProjects(), project.getRowType().getFieldNames()) - .build() - ); - } -} 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 279976f6cdc..bf961356026 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 @@ -142,22 +142,6 @@ public class BaseCalciteQueryTest extends CalciteTestBase return false; } }; - public static final PlannerConfig PLANNER_CONFIG_SINGLE_NESTING_ONLY = new PlannerConfig() - { - @Override - public int getMaxQueryCount() - { - return 2; - } - }; - public static final PlannerConfig PLANNER_CONFIG_NO_SUBQUERIES = new PlannerConfig() - { - @Override - public int getMaxQueryCount() - { - return 1; - } - }; public static final PlannerConfig PLANNER_CONFIG_LOS_ANGELES = new PlannerConfig() { @Override @@ -166,14 +150,6 @@ public class BaseCalciteQueryTest extends CalciteTestBase return DateTimes.inferTzFromString("America/Los_Angeles"); } }; - public static final PlannerConfig PLANNER_CONFIG_SEMI_JOIN_ROWS_LIMIT = new PlannerConfig() - { - @Override - public int getMaxSemiJoinRowsInMemory() - { - return 2; - } - }; public static final String DUMMY_SQL_ID = "dummy"; public static final String LOS_ANGELES = "America/Los_Angeles"; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java index 87e8d2947cb..ff32ac19b99 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java @@ -602,14 +602,13 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest and( bound("l1", "3", null, true, false, null, StringComparators.NUMERIC), selector("f1", useDefault ? "0.0" : null, null) - ) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) .build() ) : ImmutableList.of(), - useDefault ? ImmutableList.of() : ImmutableList.of(new Object[]{0L}), + ImmutableList.of(), ImmutableList.of(new SqlParameter(SqlType.BIGINT, 3L), new SqlParameter(SqlType.VARCHAR, "wat")) ); } 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 b34b3ddbf20..bf4aae47684 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 @@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.calcite.runtime.CalciteContextException; import org.apache.calcite.tools.ValidationException; import org.apache.druid.common.config.NullHandling; @@ -28,11 +29,13 @@ 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.JodaUtils; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Druids; import org.apache.druid.query.LookupDataSource; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.TableDataSource; @@ -103,7 +106,6 @@ import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -2906,7 +2908,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .dimension(new DefaultDimensionSpec("dim1", "d0")) - .postAggregators(expressionPostAgg("p0", "substring(\"d0\", 1, -1)")) + .postAggregators(expressionPostAgg("s0", "substring(\"d0\", 1, -1)")) .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) .threshold(10) .context(QUERY_CONTEXT_DEFAULT) @@ -3608,6 +3610,56 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testSelfJoin() throws Exception + { + // Cannot vectorize due to virtual columns. + cannotVectorize(); + + testQuery( + "SELECT COUNT(*) FROM druid.foo x, druid.foo y\n", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns( + ImmutableList.of( + "__time", + "cnt", + "dim1", + "dim2", + "dim3", + "m1", + "m2", + "unique_dim1" + ) + ) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + "1", + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{36L} + ) + ); + } + @Test public void testUnplannableQueries() { @@ -3616,12 +3668,19 @@ public class CalciteQueryTest extends BaseCalciteQueryTest // It's also here so when we do support these features, we can have "real" tests for these queries. final List queries = ImmutableList.of( - "SELECT dim1 FROM druid.foo ORDER BY dim1", // SELECT query with order by non-__time - "SELECT COUNT(*) FROM druid.foo x, druid.foo y", // Self-join - "SELECT DISTINCT dim2 FROM druid.foo ORDER BY dim2 LIMIT 2 OFFSET 5", // DISTINCT with OFFSET - "SELECT COUNT(*) FROM foo WHERE dim1 NOT IN (SELECT dim1 FROM foo WHERE dim2 = 'a')", // NOT IN subquery - "EXPLAIN PLAN FOR SELECT COUNT(*) FROM foo WHERE dim1 IN (SELECT dim1 FROM foo WHERE dim2 = 'a')\n" - + "AND dim1 IN (SELECT dim1 FROM foo WHERE m2 > 2)" // AND of two IN subqueries + // SELECT query with order by non-__time. + "SELECT dim1 FROM druid.foo ORDER BY dim1", + + // DISTINCT with OFFSET. + "SELECT DISTINCT dim2 FROM druid.foo ORDER BY dim2 LIMIT 2 OFFSET 5", + + // JOIN condition with not-equals (<>). + "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + + "FROM foo INNER JOIN lookup.lookyloo l ON foo.dim2 <> l.k", + + // JOIN condition with a function of both sides. + "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + + "FROM foo INNER JOIN lookup.lookyloo l ON CHARACTER_LENGTH(foo.dim2 || l.k) > 3\n" ); for (final String query : queries) { @@ -3630,19 +3689,101 @@ public class CalciteQueryTest extends BaseCalciteQueryTest } @Test - public void testUnplannableExactCountDistinctQueries() + public void testTwoExactCountDistincts() throws Exception { - // All of these queries are unplannable in exact COUNT DISTINCT mode. - - final List queries = ImmutableList.of( - "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo", // two COUNT DISTINCTs, same query - "SELECT dim1, COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo GROUP BY dim1", // two COUNT DISTINCTs - "SELECT COUNT(distinct unique_dim1) FROM druid.foo" // COUNT DISTINCT on sketch cannot be exact + testQuery( + PLANNER_CONFIG_NO_HLL, + "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo", + CalciteTests.REGULAR_USER_AUTH_RESULT, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("dim1", "d0", ValueType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + not(selector("d0", null, null)) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ValueType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + not(selector("d0", null, null)) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + "1", + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("a0", "j0.a0") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L} + ) ); + } - for (final String query : queries) { - assertQueryIsUnplannable(PLANNER_CONFIG_NO_HLL, query); - } + @Test + public void testUnplannableTwoExactCountDistincts() + { + // Requires GROUPING SETS + GROUPING to be translated by AggregateExpandDistinctAggregatesRule. + + assertQueryIsUnplannable( + PLANNER_CONFIG_NO_HLL, + "SELECT dim2, COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo GROUP BY dim2" + ); + } + + @Test + public void testUnplannableExactCountDistinctOnSketch() + { + // COUNT DISTINCT on a sketch cannot be exact. + + assertQueryIsUnplannable( + PLANNER_CONFIG_NO_HLL, + "SELECT COUNT(distinct unique_dim1) FROM druid.foo" + ); } @Test @@ -4841,10 +4982,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testCountStarWithNotOfDegenerateFilter() throws Exception { + // This query is evaluated in the planner (no native queries are issued) due to the degenerate filter. + testQuery( "SELECT COUNT(*) FROM druid.foo WHERE dim2 = 'a' and not (dim1 > 'a' OR dim1 < 'b')", ImmutableList.of(), - ImmutableList.of(new Object[]{0L}) + ImmutableList.of() ); } @@ -5536,7 +5679,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setVirtualColumns( expressionVirtualColumn("v0", "strlen(\"dim1\")", ValueType.LONG), // The two layers of CASTs here are unusual, they should really be collapsed into one - expressionVirtualColumn("v1", "CAST(CAST(strlen(\"dim1\"), 'STRING'), 'LONG')", ValueType.LONG) + expressionVirtualColumn( + "v1", + "CAST(CAST(strlen(\"dim1\"), 'STRING'), 'LONG')", + ValueType.LONG + ) ) .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) .setDimFilter( @@ -5876,8 +6023,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions(dimensions( - new DefaultDimensionSpec("dim1", "d0"), - new DefaultDimensionSpec("dim2", "d1") + new DefaultDimensionSpec("dim2", "d0"), + new DefaultDimensionSpec("dim1", "d1") )) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) @@ -5886,12 +6033,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("d1", "_d0"))) + .setDimensions(dimensions(new DefaultDimensionSpec("d0", "_d0"))) .setAggregatorSpecs(aggregators( new LongSumAggregatorFactory("_a0", "a0"), new FilteredAggregatorFactory( new CountAggregatorFactory("_a1"), - not(selector("d0", null, null)) + not(selector("d1", null, null)) ) )) .setContext(QUERY_CONTEXT_DEFAULT) @@ -6022,8 +6169,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions(dimensions( - new DefaultDimensionSpec("dim1", "d0"), - new DefaultDimensionSpec("m2", "d1", ValueType.DOUBLE) + new DefaultDimensionSpec("m2", "d0", ValueType.DOUBLE), + new DefaultDimensionSpec("dim1", "d1") )) .setDimFilter(new SelectorDimFilter("m1", "5.0", null)) .setAggregatorSpecs(aggregators(new LongMaxAggregatorFactory("a0", "__time"))) @@ -6041,7 +6188,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ) .setDimensions(dimensions( new DefaultDimensionSpec("v0", "_d0", ValueType.LONG), - new DefaultDimensionSpec("d0", "_d1", ValueType.STRING) + new DefaultDimensionSpec("d1", "_d1", ValueType.STRING) )) .setAggregatorSpecs( aggregators( @@ -6049,7 +6196,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ? new CountAggregatorFactory("_a0") : new FilteredAggregatorFactory( new CountAggregatorFactory("_a0"), - not(selector("d1", null, null)) + not(selector("d0", null, null)) ) ) ) @@ -6110,30 +6257,21 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setContext(QUERY_CONTEXT_DEFAULT) .build() ), - NullHandling.replaceWithDefault() ? - ImmutableList.of( - new Object[]{6L, 3L} - ) : - ImmutableList.of( - new Object[]{6L, 4L} - ) + NullHandling.replaceWithDefault() + ? ImmutableList.of(new Object[]{6L, 3L}) + : ImmutableList.of(new Object[]{6L, 4L}) ); } @Test - public void testExplainDoubleNestedGroupBy() throws Exception + public void testDoubleNestedGroupBy2() throws Exception { - // Skip vectorization since otherwise the "context" will change for each subtest. - skipVectorize(); - - final String explanation = - "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"cnt\",\"expression\":null},{\"type\":\"count\",\"name\":\"a1\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"}}], signature=[{a0:LONG, a1:LONG}])\n" - + " DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"longSum\",\"name\":\"a0\",\"fieldName\":\"cnt\",\"expression\":null}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false}], signature=[{d0:STRING, a0:LONG}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d1\",\"outputType\":\"STRING\"}],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false}], signature=[{d0:STRING, d1:STRING, a0:LONG}])\n"; + // This test fails when AggregateMergeRule is added to Rules.ABSTRACT_RELATIONAL_RULES. So, we don't add that + // rule for now. Possible bug in the rule. testQuery( - "EXPLAIN PLAN FOR SELECT SUM(cnt), COUNT(*) FROM (\n" - + " SELECT dim2, SUM(t1.cnt) cnt FROM (\n" + "SELECT MAX(cnt) FROM (\n" + + " SELECT dim2, MAX(t1.cnt) cnt FROM (\n" + " SELECT\n" + " dim1,\n" + " dim2,\n" @@ -6143,10 +6281,37 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + " ) t1\n" + " GROUP BY dim2\n" + ") t2", - ImmutableList.of(), ImmutableList.of( - new Object[]{explanation} - ) + GroupByQuery.builder() + .setDataSource( + GroupByQuery.builder() + .setDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + new DefaultDimensionSpec("dim1", "d0"), + new DefaultDimensionSpec("dim2", "d1") + ) + .setAggregatorSpecs(new CountAggregatorFactory("a0")) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("d1", "_d0")) + .setAggregatorSpecs(new LongMaxAggregatorFactory("_a0", "a0")) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(new LongMaxAggregatorFactory("a0", "_a0")) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of(new Object[]{1L}) ); } @@ -6154,12 +6319,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest public void testExactCountDistinctUsingSubquery() throws Exception { testQuery( - PLANNER_CONFIG_SINGLE_NESTING_ONLY, // Sanity check; this query should work with a single level of nesting. "SELECT\n" + " SUM(cnt),\n" + " COUNT(*)\n" + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2)", - CalciteTests.REGULAR_USER_AUTH_RESULT, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -6345,9 +6508,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testTopNFilterJoin() throws Exception { - DimFilter filter = NullHandling.replaceWithDefault() ? - in("dim2", Arrays.asList(null, "a"), null) - : selector("dim2", "a", null); + // Cannot vectorize JOIN operator. + cannotVectorize(); + // Filters on top N values of some dimension by using an inner join. testQuery( "SELECT t1.dim1, SUM(t1.cnt)\n" @@ -6364,21 +6527,32 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + "GROUP BY t1.dim1\n" + "ORDER BY 1\n", ImmutableList.of( - new TopNQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .dimension(new DefaultDimensionSpec("dim2", "d0")) - .aggregators(new LongSumAggregatorFactory("a0", "cnt")) - .metric(new NumericTopNMetricSpec("a0")) - .threshold(2) - .context(QUERY_CONTEXT_DEFAULT) - .build(), GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + new TopNQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .dimension(new DefaultDimensionSpec("dim2", "d0")) + .aggregators(new LongSumAggregatorFactory("a0", "cnt")) + .metric("a0") + .threshold(2) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromColumn("dim2"), + DruidExpression.fromColumn("j0.d0") + ), + JoinType.INNER + ) + ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(filter) .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setLimitSpec( @@ -6396,14 +6570,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setContext(QUERY_CONTEXT_DEFAULT) .build() ), - NullHandling.replaceWithDefault() ? - ImmutableList.of( - new Object[]{"", 1L}, - new Object[]{"1", 1L}, - new Object[]{"10.1", 1L}, - new Object[]{"2", 1L}, - new Object[]{"abc", 1L} - ) : ImmutableList.of( new Object[]{"", 1L}, new Object[]{"1", 1L} @@ -6412,9 +6578,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest } @Test - @Ignore // Doesn't work public void testTopNFilterJoinWithProjection() throws Exception { + // Cannot vectorize JOIN operator. + cannotVectorize(); + // Filters on top N values of some dimension by using an inner join. Also projects the outer dimension. testQuery( @@ -6431,49 +6599,55 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + ") t2 ON (t1.dim2 = t2.dim2)\n" + "GROUP BY SUBSTRING(t1.dim1, 1, 10)", ImmutableList.of( - new TopNQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .dimension(new DefaultDimensionSpec("dim2", "d0")) - .aggregators(new LongSumAggregatorFactory("a0", "cnt")) - .metric(new NumericTopNMetricSpec("a0")) - .threshold(2) - .context(QUERY_CONTEXT_DEFAULT) - .build(), GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimFilter(in("dim2", ImmutableList.of("", "a"), null)) - .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setLimitSpec( - new DefaultLimitSpec( - ImmutableList.of( - new OrderByColumnSpec( - "d0", - OrderByColumnSpec.Direction.ASCENDING, - StringComparators.LEXICOGRAPHIC - ) + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + new TopNQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .dimension(new DefaultDimensionSpec("dim2", "d0")) + .aggregators(new LongSumAggregatorFactory("a0", "cnt")) + .metric("a0") + .threshold(2) + .context(QUERY_CONTEXT_DEFAULT) + .build() ), - Integer.MAX_VALUE + "j0.", + equalsCondition( + DruidExpression.fromColumn("dim2"), + DruidExpression.fromColumn("j0.d0") + ), + JoinType.INNER ) ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new ExtractionDimensionSpec( + "dim1", + "d0", + ValueType.STRING, + new SubstringDimExtractionFn(0, 10) + ) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), ImmutableList.of( - new Object[]{"", 1L}, - new Object[]{"1", 1L}, - new Object[]{"10.1", 1L}, - new Object[]{"2", 1L}, - new Object[]{"abc", 1L} + new Object[]{NULL_STRING, 1L}, + new Object[]{"1", 1L} ) ); } @Test + @Ignore("Stopped working after the ability to join on subqueries was added to DruidJoinRule") public void testRemovableLeftJoin() throws Exception { // LEFT JOIN where the right-hand side can be ignored. @@ -6541,30 +6715,42 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + " ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n" + ")", ImmutableList.of( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "", null))) - .setDimensions(dimensions(new ExtractionDimensionSpec( - "dim1", - "d0", - new SubstringDimExtractionFn(0, 1) - ))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build(), GroupByQuery.builder() .setDataSource( new QueryDataSource( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter(not(selector("dim1", "", null))) + .setDimensions( + dimensions( + new ExtractionDimensionSpec( + "dim1", + "d0", + new SubstringDimExtractionFn(0, 1) + ) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromExpression("substring(\"dim2\", 0, 1)"), + DruidExpression.fromColumn("j0.d0") + ), + JoinType.INNER + ) + ) .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01"))) .setGranularity(Granularities.ALL) - .setDimFilter(in( - "dim2", - ImmutableList.of("1", "2", "a", "d"), - new SubstringDimExtractionFn(0, 1) - )) .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -6572,9 +6758,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setAggregatorSpecs(aggregators( - new CountAggregatorFactory("a0") - )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -6585,12 +6769,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest } @Test - public void testMaxSemiJoinRowsInMemory() throws Exception + public void testMaxSubqueryRows() throws Exception { expectedException.expect(ResourceLimitExceededException.class); - expectedException.expectMessage("maxSemiJoinRowsInMemory[2] exceeded"); + expectedException.expectMessage("Subquery generated results beyond maximum[2]"); + testQuery( - PLANNER_CONFIG_SEMI_JOIN_ROWS_LIMIT, + PLANNER_CONFIG_DEFAULT, + ImmutableMap.of(QueryContexts.MAX_SUBQUERY_ROWS_KEY, 2), "SELECT COUNT(*)\n" + "FROM druid.foo\n" + "WHERE SUBSTRING(dim2, 1, 1) IN (\n" @@ -6610,7 +6796,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest final String explanation = "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"}}], signature=[{a0:LONG}])\n" - + " DruidSemiJoin(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false}], leftExpressions=[[SUBSTRING($3, 1, 1)]], rightKeys=[[0]])\n" + + " DruidJoinQueryRel(condition=[=(SUBSTRING($3, 1, 1), $8)], joinType=[INNER], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null,\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n"; testQuery( @@ -7064,7 +7251,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest // Cannot vectorize due to extractionFn in dimension spec. cannotVectorize(); - String nullValue = NullHandling.replaceWithDefault() ? "" : null; testQuery( "SELECT DISTINCT\n" + " REGEXP_EXTRACT(dim1, '^.'),\n" @@ -7101,7 +7287,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .build() ), ImmutableList.of( - new Object[]{nullValue, nullValue}, + new Object[]{NULL_STRING, NULL_STRING}, new Object[]{"1", "1"}, new Object[]{"2", "2"}, new Object[]{"a", "a"}, @@ -7113,7 +7299,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testGroupBySortPushDown() throws Exception { - String nullValue = NullHandling.replaceWithDefault() ? "" : null; testQuery( "SELECT dim2, dim1, SUM(cnt) FROM druid.foo GROUP BY dim2, dim1 ORDER BY dim1 LIMIT 4", ImmutableList.of( @@ -7146,7 +7331,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ImmutableList.of( new Object[]{"a", "", 1L}, new Object[]{"a", "1", 1L}, - new Object[]{nullValue, "10.1", 1L}, + new Object[]{NULL_STRING, "10.1", 1L}, new Object[]{"", "2", 1L} ) ); @@ -7155,7 +7340,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testGroupByLimitPushDownWithHavingOnLong() throws Exception { - String nullValue = NullHandling.replaceWithDefault() ? "" : null; testQuery( "SELECT dim1, dim2, SUM(cnt) AS thecnt " + "FROM druid.foo " @@ -7526,50 +7710,57 @@ public class CalciteQueryTest extends BaseCalciteQueryTest testQuery( "SELECT COUNT(*) FROM druid.foo4\n" - + "WHERE EXTRACT(YEAR FROM __time) = 2000\n" - + "AND EXTRACT(MICROSECOND FROM __time) = 946723\n" - + "AND EXTRACT(MILLISECOND FROM __time) = 695\n" - + "AND EXTRACT(ISODOW FROM __time) = 6\n" - + "AND EXTRACT(ISOYEAR FROM __time) = 2000\n" - + "AND EXTRACT(DECADE FROM __time) = 200\n" - + "AND EXTRACT(CENTURY FROM __time) = 20\n" - + "AND EXTRACT(MILLENNIUM FROM __time) = 2\n", - + + "WHERE EXTRACT(YEAR FROM __time) = 2000\n" + + "AND EXTRACT(MICROSECOND FROM __time) = 946723\n" + + "AND EXTRACT(MILLISECOND FROM __time) = 695\n" + + "AND EXTRACT(ISODOW FROM __time) = 6\n" + + "AND EXTRACT(ISOYEAR FROM __time) = 2000\n" + + "AND EXTRACT(DECADE FROM __time) = 200\n" + + "AND EXTRACT(CENTURY FROM __time) = 20\n" + + "AND EXTRACT(MILLENNIUM FROM __time) = 2\n", TIMESERIES_CONTEXT_DEFAULT, ImmutableList.of( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE4) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .virtualColumns( - expressionVirtualColumn("v0", "timestamp_extract(\"__time\",'YEAR','UTC')", ValueType.LONG), - expressionVirtualColumn("v1", "timestamp_extract(\"__time\",'MICROSECOND','UTC')", ValueType.LONG), - expressionVirtualColumn("v2", "timestamp_extract(\"__time\",'MILLISECOND','UTC')", ValueType.LONG), - expressionVirtualColumn("v3", "timestamp_extract(\"__time\",'ISODOW','UTC')", ValueType.LONG), - expressionVirtualColumn("v4", "timestamp_extract(\"__time\",'ISOYEAR','UTC')", ValueType.LONG), - expressionVirtualColumn("v5", "timestamp_extract(\"__time\",'DECADE','UTC')", ValueType.LONG), - expressionVirtualColumn("v6", "timestamp_extract(\"__time\",'CENTURY','UTC')", ValueType.LONG), - expressionVirtualColumn("v7", "timestamp_extract(\"__time\",'MILLENNIUM','UTC')", ValueType.LONG) - ) - .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .filters( - and( - selector("v0", "2000", null), - selector("v1", "946723", null), - selector("v2", "695", null), - selector("v3", "6", null), - selector("v4", "2000", null), - selector("v5", "200", null), - selector("v6", "20", null), - selector("v7", "2", null) - ) - ) - .context(TIMESERIES_CONTEXT_DEFAULT) - .build() - ), + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE4) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .virtualColumns( + expressionVirtualColumn("v0", "timestamp_extract(\"__time\",'YEAR','UTC')", ValueType.LONG), + expressionVirtualColumn( + "v1", + "timestamp_extract(\"__time\",'MICROSECOND','UTC')", + ValueType.LONG + ), + expressionVirtualColumn( + "v2", + "timestamp_extract(\"__time\",'MILLISECOND','UTC')", + ValueType.LONG + ), + expressionVirtualColumn("v3", "timestamp_extract(\"__time\",'ISODOW','UTC')", ValueType.LONG), + expressionVirtualColumn("v4", "timestamp_extract(\"__time\",'ISOYEAR','UTC')", ValueType.LONG), + expressionVirtualColumn("v5", "timestamp_extract(\"__time\",'DECADE','UTC')", ValueType.LONG), + expressionVirtualColumn("v6", "timestamp_extract(\"__time\",'CENTURY','UTC')", ValueType.LONG), + expressionVirtualColumn("v7", "timestamp_extract(\"__time\",'MILLENNIUM','UTC')", ValueType.LONG) + ) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .filters( + and( + selector("v0", "2000", null), + selector("v1", "946723", null), + selector("v2", "695", null), + selector("v3", "6", null), + selector("v4", "2000", null), + selector("v5", "200", null), + selector("v6", "20", null), + selector("v7", "2", null) + ) + ) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), ImmutableList.of( - new Object[]{1L} - ) + new Object[]{1L} + ) ); } @@ -7599,9 +7790,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest cannotVectorize(); testQuery( - PLANNER_CONFIG_NO_SUBQUERIES, // Sanity check; this simple query should work with subqueries disabled. "SELECT floor(CAST(dim1 AS float)), COUNT(*) FROM druid.foo GROUP BY floor(CAST(dim1 AS float))", - CalciteTests.REGULAR_USER_AUTH_RESULT, ImmutableList.of( GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) @@ -7790,7 +7979,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest // Cannot vectorize due to extraction dimension specs. cannotVectorize(); - String nullValue = NullHandling.replaceWithDefault() ? "" : null; final RegisteredLookupExtractionFn extractionFn = new RegisteredLookupExtractionFn( null, "lookyloo", @@ -7835,7 +8023,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .build() ), ImmutableList.of( - new Object[]{nullValue, 5L}, + new Object[]{NULL_STRING, 5L}, new Object[]{"xabc", 1L} ) ); @@ -7878,6 +8066,44 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testFilterAndGroupByLookupUsingJoinOperatorBackwards() throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + // Like "testFilterAndGroupByLookupUsingJoinOperator", but with the table and lookup reversed. + testQuery( + "SELECT lookyloo.v, COUNT(*)\n" + + "FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k\n" + + "WHERE lookyloo.v <> 'xa'\n" + + "GROUP BY lookyloo.v", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new LookupDataSource("lookyloo"), + "j0.", + equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.k")), + JoinType.LEFT + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimFilter(not(selector("j0.v", "xa", null))) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("j0.v", "d0"))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{NULL_STRING, 3L}, + new Object[]{"xabc", 1L} + ) + ); + } + @Test public void testFilterAndGroupByLookupUsingJoinOperator() throws Exception { @@ -7916,39 +8142,49 @@ public class CalciteQueryTest extends BaseCalciteQueryTest } @Test - public void testFilterAndGroupByLookupUsingJoinOperatorBackwards() throws Exception + public void testFilterAndGroupByLookupUsingPostAggregationJoinOperator() throws Exception { - // Cannot vectorize JOIN operator. - cannotVectorize(); - - // Like "testFilterAndGroupByLookupUsingJoinOperator", but with the table and lookup reversed. testQuery( - "SELECT lookyloo.v, COUNT(*)\n" - + "FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k\n" - + "WHERE lookyloo.v <> 'xa'\n" - + "GROUP BY lookyloo.v", + "SELECT base.dim2, lookyloo.v, base.cnt FROM (\n" + + " SELECT dim2, COUNT(*) cnt FROM foo GROUP BY dim2\n" + + ") base\n" + + "LEFT JOIN lookup.lookyloo ON base.dim2 = lookyloo.k\n" + + "WHERE lookyloo.v <> 'xa' OR lookyloo.v IS NULL", ImmutableList.of( - GroupByQuery.builder() - .setDataSource( - join( - new TableDataSource(CalciteTests.DATASOURCE1), - new LookupDataSource("lookyloo"), - "j0.", - equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.k")), - JoinType.LEFT - ) - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(not(selector("j0.v", "xa", null))) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("j0.v", "d0"))) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + newScanQueryBuilder() + .dataSource( + join( + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + new LookupDataSource("lookyloo"), + "j0.", + equalsCondition(DruidExpression.fromColumn("d0"), DruidExpression.fromColumn("j0.k")), + JoinType.LEFT + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(or(not(selector("j0.v", "xa", null)), selector("j0.v", null, null))) + .columns("a0", "d0", "j0.v") + .context(QUERY_CONTEXT_DEFAULT) + .build() ), - ImmutableList.of( - new Object[]{NULL_STRING, 3L}, - new Object[]{"xabc", 1L} + NullHandling.sqlCompatible() + ? ImmutableList.of( + new Object[]{NULL_STRING, NULL_STRING, 2L}, + new Object[]{"", NULL_STRING, 1L}, + new Object[]{"abc", "xabc", 1L} + ) : ImmutableList.of( + new Object[]{NULL_STRING, NULL_STRING, 3L}, + new Object[]{"abc", "xabc", 1L} ) ); } @@ -8058,6 +8294,283 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testInnerJoinQueryOfLookup() throws Exception + { + // Cannot vectorize the subquery. + cannotVectorize(); + + testQuery( + "SELECT dim1, dim2, t1.v, t1.v\n" + + "FROM foo\n" + + "INNER JOIN \n" + + " (SELECT SUBSTRING(k, 1, 1) k, LATEST(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1\n" + + " ON foo.dim2 = t1.k", + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource(new LookupDataSource("lookyloo")) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + new ExtractionDimensionSpec( + "k", + "d0", + new SubstringDimExtractionFn(0, 1) + ) + ) + .setAggregatorSpecs(new StringLastAggregatorFactory("a0", "v", 10)) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.d0")), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim1", "dim2", "j0.a0") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"", "a", "xabc", "xabc"}, + new Object[]{"1", "a", "xabc", "xabc"} + ) + ); + } + + @Test + public void testInnerJoinQueryOfLookupRemovable() throws Exception + { + // Like "testInnerJoinQueryOfLookup", but the subquery is removable. + + testQuery( + "SELECT dim1, dim2, t1.sk\n" + + "FROM foo\n" + + "INNER JOIN \n" + + " (SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1\n" + + " ON foo.dim2 = t1.k", + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new LookupDataSource("lookyloo"), + "j0.", + equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.k")), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("v0", "substring(\"j0.v\", 0, 3)", ValueType.STRING)) + .columns("dim1", "dim2", "v0") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"", "a", "xa"}, + new Object[]{"1", "a", "xa"}, + new Object[]{"def", "abc", "xab"} + ) + ); + } + + @Test + public void testInnerJoinCastLeft() throws Exception + { + // foo.m1 is FLOAT, l.k is STRING. + + testQuery( + "SELECT foo.m1, l.k, l.v\n" + + "FROM foo\n" + + "INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k\n", + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new LookupDataSource("lookyloo"), + "j0.", + equalsCondition( + DruidExpression.fromExpression("CAST(\"m1\", 'STRING')"), + DruidExpression.fromColumn("j0.k") + ), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("j0.k", "j0.v", "m1") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of() + ); + } + + @Test + public void testInnerJoinCastRight() throws Exception + { + // foo.m1 is FLOAT, l.k is STRING. + + testQuery( + "SELECT foo.m1, l.k, l.v\n" + + "FROM foo\n" + + "INNER JOIN lookup.lookyloo l ON foo.m1 = CAST(l.k AS FLOAT)\n", + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(new LookupDataSource("lookyloo")) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + expressionVirtualColumn("v0", "CAST(\"k\", 'DOUBLE')", ValueType.FLOAT) + ) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .columns("k", "v", "v0") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("m1"), DruidExpression.fromColumn("j0.v0")), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("j0.k", "j0.v", "m1") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{6f, "6", "x6"} + ) + ); + } + + @Test + public void testInnerJoinMismatchedTypes() throws Exception + { + // foo.m1 is FLOAT, l.k is STRING. Comparing them generates a CAST. + + testQuery( + "SELECT foo.m1, l.k, l.v\n" + + "FROM foo\n" + + "INNER JOIN lookup.lookyloo l ON foo.m1 = l.k\n", + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(new LookupDataSource("lookyloo")) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + expressionVirtualColumn("v0", "CAST(\"k\", 'DOUBLE')", ValueType.FLOAT) + ) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .columns("k", "v", "v0") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("m1"), DruidExpression.fromColumn("j0.v0")), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("j0.k", "j0.v", "m1") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{6f, "6", "x6"} + ) + ); + } + + @Test + public void testInnerJoinLeftFunction() throws Exception + { + testQuery( + "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + + "FROM foo\n" + + "INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k\n", + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new LookupDataSource("lookyloo"), + "j0.", + equalsCondition( + DruidExpression.fromExpression("substring(\"dim2\", 0, 1)"), + DruidExpression.fromColumn("j0.k") + ), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim1", "dim2", "j0.k", "j0.v") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"", "a", "a", "xa"}, + new Object[]{"1", "a", "a", "xa"}, + new Object[]{"def", "abc", "a", "xa"} + ) + ); + } + + @Test + public void testInnerJoinRightFunction() throws Exception + { + testQuery( + "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + + "FROM foo\n" + + "INNER JOIN lookup.lookyloo l ON foo.dim2 = SUBSTRING(l.k, 1, 2)\n", + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(new LookupDataSource("lookyloo")) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + expressionVirtualColumn("v0", "substring(\"k\", 0, 2)", ValueType.STRING) + ) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .columns("k", "v", "v0") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.v0")), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim1", "dim2", "j0.k", "j0.v") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"", "a", "a", "xa"}, + new Object[]{"1", "a", "a", "xa"} + ) + ); + } + @Test public void testLeftJoinLookupOntoLookupUsingJoinOperator() throws Exception { @@ -8209,7 +8722,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ImmutableList.of( new Object[]{"abc", "abc", "xabc"}, new Object[]{NULL_STRING, "a", "xa"}, - new Object[]{NULL_STRING, "nosuchkey", "mysteryvalue"} + new Object[]{NULL_STRING, "nosuchkey", "mysteryvalue"}, + new Object[]{NULL_STRING, "6", "x6"} ) ); } @@ -8246,7 +8760,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest new Object[]{"def", 5f, 1L, NULL_STRING, NULL_STRING}, new Object[]{"abc", 6f, 1L, "abc", "xabc"}, new Object[]{NULL_STRING, NULL_FLOAT, NULL_LONG, "a", "xa"}, - new Object[]{NULL_STRING, NULL_FLOAT, NULL_LONG, "nosuchkey", "mysteryvalue"} + new Object[]{NULL_STRING, NULL_FLOAT, NULL_LONG, "nosuchkey", "mysteryvalue"}, + new Object[]{NULL_STRING, NULL_FLOAT, NULL_LONG, "6", "x6"} ) ); } @@ -8331,6 +8846,61 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testSelectStarFromLookup() throws Exception + { + testQuery( + "SELECT * FROM lookup.lookyloo", + ImmutableList.of( + newScanQueryBuilder() + .dataSource(new LookupDataSource("lookyloo")) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("k", "v") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"a", "xa"}, + new Object[]{"abc", "xabc"}, + new Object[]{"nosuchkey", "mysteryvalue"}, + new Object[]{"6", "x6"} + ) + ); + } + + @Test + public void testGroupByExpressionFromLookup() throws Exception + { + // Cannot vectorize direct queries on lookup tables. + cannotVectorize(); + + testQuery( + "SELECT SUBSTRING(v, 1, 1), COUNT(*) FROM lookup.lookyloo GROUP BY 1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(new LookupDataSource("lookyloo")) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new ExtractionDimensionSpec( + "v", + "d0", + new SubstringDimExtractionFn(0, 1) + ) + ) + ) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"m", 1L}, + new Object[]{"x", 3L} + ) + ); + } + @Test public void testTimeseries() throws Exception { @@ -8364,6 +8934,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest "SELECT\n" + " SUM(cnt) FILTER(WHERE __time >= TIMESTAMP '2000-01-01 00:00:00'\n" + " AND __time < TIMESTAMP '2000-02-01 00:00:00'),\n" + + " SUM(cnt) FILTER(WHERE __time >= TIMESTAMP '2000-01-01 00:00:01'\n" + + " AND __time < TIMESTAMP '2000-02-01 00:00:00'),\n" + " SUM(cnt) FILTER(WHERE __time >= TIMESTAMP '2001-01-01 00:00:00'\n" + " AND __time < TIMESTAMP '2001-02-01 00:00:00')\n" + "FROM foo\n" @@ -8380,7 +8952,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest new LongSumAggregatorFactory("a0", "cnt"), bound( "__time", - String.valueOf(timestamp("2000-01-01")), + null, String.valueOf(timestamp("2000-02-01")), false, true, @@ -8390,6 +8962,18 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a1", "cnt"), + bound( + "__time", + String.valueOf(timestamp("2000-01-01T00:00:01")), + String.valueOf(timestamp("2000-02-01")), + false, + true, + null, + StringComparators.NUMERIC + ) + ), + new FilteredAggregatorFactory( + new LongSumAggregatorFactory("a2", "cnt"), bound( "__time", String.valueOf(timestamp("2001-01-01")), @@ -8405,7 +8989,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .build() ), ImmutableList.of( - new Object[]{3L, 3L} + new Object[]{3L, 2L, 3L} ) ); } @@ -9750,30 +10334,126 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testUsingSubqueryAsPartOfAndFilter() throws Exception { + // Cannot vectorize JOIN operator. + cannotVectorize(); + testQuery( - PLANNER_CONFIG_SINGLE_NESTING_ONLY, // Sanity check; this query should work with a single level of nesting. "SELECT dim1, dim2, COUNT(*) FROM druid.foo\n" + "WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '')\n" + "AND dim1 <> 'xxx'\n" + "group by dim1, dim2 ORDER BY dim2", - CalciteTests.REGULAR_USER_AUTH_RESULT, ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter(not(selector("dim1", "", null))) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromColumn("dim2"), + DruidExpression.fromColumn("j0.d0") + ), + JoinType.INNER + ) + ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "", null))) - .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) + .setDimFilter(not(selector("dim1", "xxx", null))) + .setDimensions( + dimensions( + new DefaultDimensionSpec("dim1", "d0"), + new DefaultDimensionSpec("dim2", "d1") + ) + ) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of(new OrderByColumnSpec("d1", OrderByColumnSpec.Direction.ASCENDING)), + Integer.MAX_VALUE + ) + ) .setContext(QUERY_CONTEXT_DEFAULT) - .build(), + .build() + ), + ImmutableList.of( + new Object[]{"def", "abc", 1L} + ) + ); + } + + @Test + public void testUsingSubqueryAsPartOfOrFilter() throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + testQuery( + "SELECT dim1, dim2, COUNT(*) FROM druid.foo\n" + + "WHERE dim1 = 'xxx' OR dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 LIKE '%bc')\n" + + "group by dim1, dim2 ORDER BY dim2", + ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) + .setDataSource( + join( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(new LikeDimFilter("dim1", "%bc", null, null)) + .granularity(Granularities.ALL) + .aggregators(new CountAggregatorFactory("a0")) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), + "j0.", + "1", + JoinType.INNER + ), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns(expressionVirtualColumn("v0", "1", ValueType.LONG)) + .setDimFilter(new LikeDimFilter("dim1", "%bc", null, null)) + .setDimensions( + dimensions( + new DefaultDimensionSpec("dim1", "d0"), + new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "_j0.", + equalsCondition( + DruidExpression.fromColumn("dim2"), + DruidExpression.fromColumn("_j0.d0") + ), + JoinType.LEFT + ) + ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimFilter( - and( - not(selector("dim1", "xxx", null)), - in("dim2", ImmutableList.of("1", "10.1", "2", "abc", "def"), null) + or( + selector("dim1", "xxx", null), + and( + not(selector("j0.a0", "0", null)), + not(selector("_j0.d1", null, null)), + not(selector("dim2", null, null)) + ) ) ) .setDimensions( @@ -9798,19 +10478,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } - @Test - public void testUsingSubqueryAsPartOfOrFilter() - { - // This query should ideally be plannable, but it's not. The "OR" means it isn't really - // a semiJoin and so the filter condition doesn't get converted. - - final String theQuery = "SELECT dim1, dim2, COUNT(*) FROM druid.foo\n" - + "WHERE dim1 = 'xxx' OR dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 LIKE '%bc')\n" - + "group by dim1, dim2 ORDER BY dim2"; - - assertQueryIsUnplannable(theQuery); - } - @Test public void testTimeExtractWithTooFewArguments() throws Exception { @@ -9827,18 +10494,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest testQuery("SELECT TIME_EXTRACT(__time) FROM druid.foo", ImmutableList.of(), ImmutableList.of()); } - @Test - public void testUsingSubqueryAsFilterForbiddenByConfig() - { - assertQueryIsUnplannable( - PLANNER_CONFIG_NO_SUBQUERIES, - "SELECT dim1, dim2, COUNT(*) FROM druid.foo " - + "WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '')" - + "AND dim1 <> 'xxx'" - + "group by dim1, dim2 ORDER BY dim2" - ); - } - @Test public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception { @@ -9854,30 +10509,40 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + " )" + " )", ImmutableList.of( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimFilter(selector("dim2", "abc", null)) - .setDimensions(dimensions( - new DefaultDimensionSpec("dim1", "d0"), - new DefaultDimensionSpec("dim2", "d1") - )) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setHavingSpec(having(selector("a0", "1", null))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build(), newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .filters(or( - selector("dim1", "def", null), - and( - selector("dim1", "def", null), - selector("dim2", "abc", null) + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter(selector("dim2", "abc", null)) + .setDimensions(dimensions( + new DefaultDimensionSpec("dim1", "d0"), + new DefaultDimensionSpec("dim2", "d1") + )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setPostAggregatorSpecs( + ImmutableList.of(expressionPostAgg("p0", "'abc'")) + ) + .setHavingSpec(having(selector("a0", "1", null))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + StringUtils.format( + "(%s && %s)", + equalsCondition(DruidExpression.fromColumn("dim1"), DruidExpression.fromColumn("j0.d0")), + equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.p0")) + ), + JoinType.INNER ) - )) - .columns("__time", "cnt", "dim1", "dim2") + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("v0", "'abc'", ValueType.STRING)) + .columns("__time", "cnt", "dim1", "v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -9891,9 +10556,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testUsingSubqueryAsFilterWithInnerSort() throws Exception { - String nullValue = NullHandling.replaceWithDefault() ? "" : null; - // Regression test for https://github.com/apache/druid/issues/4208 + testQuery( "SELECT dim1, dim2 FROM druid.foo\n" + " WHERE dim2 IN (\n" @@ -9903,17 +10567,25 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + " ORDER BY dim2 DESC\n" + " )", ImmutableList.of( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build(), newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.d0")), + JoinType.INNER + ) + ) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(in("dim2", ImmutableList.of("", "a", "abc"), null)) .columns("dim1", "dim2") .context(QUERY_CONTEXT_DEFAULT) .build() @@ -9921,11 +10593,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest NullHandling.replaceWithDefault() ? ImmutableList.of( new Object[]{"", "a"}, - new Object[]{"10.1", nullValue}, - new Object[]{"2", ""}, new Object[]{"1", "a"}, - new Object[]{"def", "abc"}, - new Object[]{"abc", nullValue} + new Object[]{"def", "abc"} ) : ImmutableList.of( new Object[]{"", "a"}, @@ -9947,26 +10616,30 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + " WHERE dim1 = 'def'\n" + " ) AND dim1 <> ''", ImmutableList.of( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) - .setDimFilter(selector("dim1", "def", null)) - .setContext(QUERY_CONTEXT_DEFAULT) - .build(), newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) + .setDimFilter(selector("dim1", "def", null)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.d0")), + JoinType.INNER + ) + ) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( expressionVirtualColumn("v0", "timestamp_extract(\"__time\",'MONTH','UTC')", ValueType.LONG) ) - .filters( - and( - not(selector("dim1", "", null)), - selector("dim2", "abc", null) - ) - ) + .filters(not(selector("dim1", "", null))) .columns("dim1", "v0") .context(QUERY_CONTEXT_DEFAULT) .build() @@ -9995,25 +10668,29 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ImmutableList.of( GroupByQuery .builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) - .setDimFilter(selector("dim1", "def", null)) - .setContext(QUERY_CONTEXT_DEFAULT) - .build(), - GroupByQuery - .builder() - .setDataSource(CalciteTests.DATASOURCE1) + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) + .setDimFilter(selector("dim1", "def", null)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.d0")), + JoinType.INNER + ) + ) .setVirtualColumns( expressionVirtualColumn("v0", "timestamp_extract(\"__time\",'MONTH','UTC')", ValueType.LONG) ) - .setDimFilter( - and( - not(selector("dim1", "", null)), - selector("dim2", "abc", null) - ) - ) + .setDimFilter(not(selector("dim1", "", null))) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) @@ -10051,10 +10728,134 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testInAggregationSubquery() throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + testQuery( + "SELECT DISTINCT __time FROM druid.foo WHERE __time IN (SELECT MAX(__time) FROM druid.foo)", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(new LongMaxAggregatorFactory("a0", "__time")) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromColumn("__time"), + DruidExpression.fromColumn("j0.a0") + ), + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ValueType.LONG))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{timestamp("2001-01-03")} + ) + ); + } + + @Test + public void testNotInAggregationSubquery() throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + testQuery( + "SELECT DISTINCT __time FROM druid.foo WHERE __time NOT IN (SELECT MAX(__time) FROM druid.foo)", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(new LongMaxAggregatorFactory("a0", "__time")) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + new CountAggregatorFactory("_a0"), + NullHandling.sqlCompatible() + ? new FilteredAggregatorFactory( + new CountAggregatorFactory("_a1"), + not(selector("a0", null, null)) + ) + : new CountAggregatorFactory("_a1") + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + "1", + JoinType.INNER + ), + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(new LongMaxAggregatorFactory("a0", "__time")) + .postAggregators(expressionPostAgg("p0", "1")) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), + "_j0.", + "(\"__time\" == \"_j0.a0\")", + JoinType.LEFT + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter( + or( + selector("j0._a0", "0", null), + and(selector("_j0.p0", null, null), expressionFilter("(\"j0._a1\" >= \"j0._a0\")")) + ) + ) + .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ValueType.LONG))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{timestamp("2000-01-01")}, + new Object[]{timestamp("2000-01-02")}, + new Object[]{timestamp("2000-01-03")}, + new Object[]{timestamp("2001-01-01")}, + new Object[]{timestamp("2001-01-02")} + ) + ); + } + @Test public void testUsingSubqueryWithExtractionFns() throws Exception { - // Cannot vectorize due to extraction dimension specs. + // Cannot vectorize JOIN operator. cannotVectorize(); testQuery( @@ -10063,24 +10864,38 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + "group by dim2", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "", null))) - .setDimensions( - dimensions(new ExtractionDimensionSpec("dim1", "d0", new SubstringDimExtractionFn(0, 1))) + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter(not(selector("dim1", "", null))) + .setDimensions( + dimensions(new ExtractionDimensionSpec( + "dim1", + "d0", + new SubstringDimExtractionFn( + 0, + 1 + ) + )) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromExpression("substring(\"dim2\", 0, 1)"), + DruidExpression.fromColumn("j0.d0") + ), + JoinType.INNER + ) ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build(), - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(in( - "dim2", - ImmutableList.of("1", "2", "a", "d"), - new SubstringDimExtractionFn(0, 1) - )) .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) @@ -10333,8 +11148,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setGranularity(Granularities.ALL) .setDimensions(dimensions( new DefaultDimensionSpec("__time", "d0", ValueType.LONG), - new DefaultDimensionSpec("dim1", "d1"), - new DefaultDimensionSpec("m2", "d2", ValueType.DOUBLE) + new DefaultDimensionSpec("m2", "d1", ValueType.DOUBLE), + new DefaultDimensionSpec("dim1", "d2") )) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -10343,7 +11158,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setGranularity(Granularities.ALL) .setDimensions(dimensions( new DefaultDimensionSpec("d0", "_d0", ValueType.LONG), - new DefaultDimensionSpec("d1", "_d1", ValueType.STRING) + new DefaultDimensionSpec("d2", "_d1", ValueType.STRING) )) .setAggregatorSpecs( aggregators( @@ -10351,7 +11166,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ? new CountAggregatorFactory("a0") : new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("d2", null, null)) + not(selector("d1", null, null)) ) ) ) @@ -10655,33 +11470,53 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + ")", CalciteTests.REGULAR_USER_AUTH_RESULT, ImmutableList.of( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Intervals.utc( - DateTimes.of("2000-01-01").getMillis(), - JodaUtils.MAX_INSTANT - ))) - .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "", null))) - .setDimensions(dimensions(new ExtractionDimensionSpec( - "dim1", - "d0", - new SubstringDimExtractionFn(0, 1) - ))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build(), Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Intervals.utc( - DateTimes.of("2000-01-01").getMillis(), - JodaUtils.MAX_INSTANT - ))) + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval( + querySegmentSpec( + Intervals.utc( + DateTimes.of("2000-01-01").getMillis(), + JodaUtils.MAX_INSTANT + ) + ) + ) + .setDimFilter( + not(selector("dim1", NullHandling.sqlCompatible() ? "" : null, null)) + ) + .setGranularity(Granularities.ALL) + .setDimensions( + new ExtractionDimensionSpec( + "dim1", + "d0", + ValueType.STRING, + new SubstringDimExtractionFn(0, 1) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromExpression("substring(\"dim2\", 0, 1)"), + DruidExpression.fromColumn("j0.d0") + ), + JoinType.INNER + ) + ) + .intervals( + querySegmentSpec( + Intervals.utc( + DateTimes.of("2000-01-01").getMillis(), + JodaUtils.MAX_INSTANT + ) + ) + ) .granularity(Granularities.ALL) - .filters(in( - "dim2", - ImmutableList.of("1", "2", "a", "d"), - new SubstringDimExtractionFn(0, 1) - )) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) .build() @@ -11484,7 +12319,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_append(\"dim3\",'foo')", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn( + "v0", + "array_append(\"dim3\",'foo')", + ValueType.STRING + )) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) @@ -11540,7 +12379,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_prepend('foo',\"dim3\")", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn( + "v0", + "array_prepend('foo',\"dim3\")", + ValueType.STRING + )) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) @@ -11593,8 +12436,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "array_to_string(array_prepend('foo',\"dim3\"),',')", ValueType.STRING), - expressionVirtualColumn("v1", "array_to_string(array_append(\"dim3\",'foo'),',')", ValueType.STRING) + expressionVirtualColumn( + "v0", + "array_to_string(array_prepend('foo',\"dim3\"),',')", + ValueType.STRING + ), + expressionVirtualColumn( + "v1", + "array_to_string(array_append(\"dim3\",'foo'),',')", + ValueType.STRING + ) ) .setDimensions( dimensions( @@ -11650,7 +12501,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_concat(\"dim3\",\"dim3\")", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn( + "v0", + "array_concat(\"dim3\",\"dim3\")", + ValueType.STRING + )) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) @@ -11763,7 +12618,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_offset_of(\"dim3\",'b')", ValueType.LONG)) + .setVirtualColumns(expressionVirtualColumn( + "v0", + "array_offset_of(\"dim3\",'b')", + ValueType.LONG + )) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "_d0", ValueType.LONG) @@ -11802,7 +12661,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_ordinal_of(\"dim3\",'b')", ValueType.LONG)) + .setVirtualColumns(expressionVirtualColumn( + "v0", + "array_ordinal_of(\"dim3\",'b')", + ValueType.LONG + )) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "_d0", ValueType.LONG) @@ -11858,7 +12721,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_to_string(\"dim3\",',')", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn( + "v0", + "array_to_string(\"dim3\",',')", + ValueType.STRING + )) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) @@ -12014,7 +12881,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } - // no existing limit + // no existing limit testQuery( PLANNER_CONFIG_DEFAULT, outerLimitContext, 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 87f421e901b..5d446c54a83 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 @@ -28,7 +28,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.Key; -import com.google.inject.Module; import org.apache.calcite.jdbc.CalciteSchema; import org.apache.calcite.schema.SchemaPlus; import org.apache.druid.client.BrokerSegmentWatcherConfig; @@ -207,7 +206,7 @@ public class CalciteTests private static final String TIMESTAMP_COLUMN = "t"; private static final Injector INJECTOR = Guice.createInjector( - (Module) binder -> { + binder -> { binder.bind(Key.get(ObjectMapper.class, Json.class)).toInstance(TestHelper.makeJsonMapper()); // This Module is just to get a LookupExtractorFactoryContainerProvider with a usable "lookyloo" lookup. @@ -217,7 +216,8 @@ public class CalciteTests ImmutableMap.of( "a", "xa", "abc", "xabc", - "nosuchkey", "mysteryvalue" + "nosuchkey", "mysteryvalue", + "6", "x6" ) ); binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupProvider); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index f9f146c3c0e..50134ffcb35 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -32,11 +32,14 @@ import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.lookup.LookupExtractorFactoryContainer; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; +import org.apache.druid.segment.InlineSegmentWrangler; +import org.apache.druid.segment.LookupSegmentWrangler; import org.apache.druid.segment.MapSegmentWrangler; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.join.InlineJoinableFactory; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.LookupJoinableFactory; @@ -107,7 +110,12 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, C ), QueryStackTests.createLocalQuerySegmentWalker( conglomerate, - new MapSegmentWrangler(ImmutableMap.of()), + new MapSegmentWrangler( + ImmutableMap., SegmentWrangler>builder() + .put(InlineDataSource.class, new InlineSegmentWrangler()) + .put(LookupDataSource.class, new LookupSegmentWrangler(lookupProvider)) + .build() + ), joinableFactoryToUse ), conglomerate,