From cccf13ea8244d43e1ce4cceb9089edcad162c8c3 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 12 Jan 2024 00:06:31 -0800 Subject: [PATCH] Reverse, pull up lookups in the SQL planner. (#15626) * Reverse, pull up lookups in the SQL planner. Adds two new rules: 1) ReverseLookupRule, which eliminates calls to LOOKUP by doing reverse lookups. 2) AggregatePullUpLookupRule, which pulls up calls to LOOKUP above GROUP BY, when the lookup is injective. Adds configs `sqlReverseLookup` and `sqlPullUpLookup` to control whether these rules fire. Both are enabled by default. To minimize the chance of performance problems due to many keys mapping to the same value, ReverseLookupRule refrains from reversing a lookup if there are more keys than `inSubQueryThreshold`. The rationale for using this setting is that reversal works by generating an IN, and the `inSubQueryThreshold` describes the largest IN the user wants the planner to create. * Add additional line. * Style. * Remove commented-out lines. * Fix tests. * Add test. * Fix doc link. * Fix docs. * Add one more test. * Fix tests. * Logic, test updates. * - Make FilterDecomposeConcatRule more flexible. - Make CalciteRulesManager apply reduction rules til fixpoint. * Additional tests, simplify code. --- benchmarks/pom.xml | 5 +- .../benchmark/lookup/LookupBenchmarkUtil.java | 99 + .../lookup/LookupExtractorBenchmark.java | 121 + .../lookup/SqlReverseLookupBenchmark.java | 156 ++ .../druid/benchmark/query/SqlBenchmark.java | 138 +- .../extensions-core/lookups-cached-global.md | 2 +- docs/querying/dimensionspecs.md | 2 +- docs/querying/lookups.md | 173 +- docs/querying/query-context.md | 10 +- docs/querying/sql-query-context.md | 5 + .../druid/query/filter/InDimFilter.java | 25 +- .../generator/GeneratorSchemaInfo.java | 13 + .../segment/generator/DataGeneratorTest.java | 1 - .../generator/GeneratorSchemaInfoTest.java | 45 + .../LookupEnabledTestExprMacroTable.java | 118 +- .../CastedLiteralOperandTypeCheckers.java | 24 - .../expression/AliasedOperatorConversion.java | 2 - .../MultiValueStringOperatorConversions.java | 13 +- .../QueryLookupOperatorConversion.java | 19 +- .../calcite/planner/CalciteRulesManager.java | 92 +- .../calcite/planner/DruidOperatorTable.java | 4 +- .../sql/calcite/planner/DruidTypeSystem.java | 2 - .../sql/calcite/planner/PlannerContext.java | 86 +- .../calcite/planner/PlannerLookupCache.java | 62 + .../rule/AggregatePullUpLookupRule.java | 125 + .../rule/FilterDecomposeConcatRule.java | 15 +- .../sql/calcite/rule/ReverseLookupRule.java | 678 ++++++ .../CalciteLookupFunctionQueryTest.java | 2083 +++++++++++++++++ .../CalciteMultiValueStringQueryTest.java | 29 +- .../druid/sql/calcite/CalciteQueryTest.java | 457 +--- .../rule/FilterDecomposeConcatRuleTest.java | 19 + .../sql/calcite/util/LookylooModule.java | 63 +- .../sql/calcite/util/TestLookupProvider.java | 58 + 33 files changed, 4166 insertions(+), 578 deletions(-) create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/lookup/LookupBenchmarkUtil.java create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/lookup/LookupExtractorBenchmark.java create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/lookup/SqlReverseLookupBenchmark.java create mode 100644 processing/src/test/java/org/apache/druid/segment/generator/GeneratorSchemaInfoTest.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerLookupCache.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/AggregatePullUpLookupRule.java create mode 100644 sql/src/main/java/org/apache/druid/sql/calcite/rule/ReverseLookupRule.java create mode 100644 sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java create mode 100644 sql/src/test/java/org/apache/druid/sql/calcite/util/TestLookupProvider.java diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index 416d18c7f44..78db822cadf 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -50,7 +50,10 @@ org.easymock easymock - 4.0.2 + + + com.google.inject.extensions + guice-multibindings org.apache.druid diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/LookupBenchmarkUtil.java b/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/LookupBenchmarkUtil.java new file mode 100644 index 00000000000..780e58296d1 --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/LookupBenchmarkUtil.java @@ -0,0 +1,99 @@ +/* + * 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.benchmark.lookup; + +import com.google.common.collect.ImmutableMap; +import it.unimi.dsi.fastutil.objects.Object2ObjectOpenHashMap; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.query.extraction.MapLookupExtractor; +import org.apache.druid.query.lookup.LookupExtractor; + +import java.util.HashMap; +import java.util.Map; +import java.util.stream.IntStream; + +/** + * Utility functions for lookup benchmarks. + */ +public class LookupBenchmarkUtil +{ + public enum LookupType + { + HASHMAP { + @Override + public LookupExtractor build(Iterable> keyValuePairs) + { + final Map map = new HashMap<>(); + for (final Pair keyValuePair : keyValuePairs) { + map.put(keyValuePair.lhs, keyValuePair.rhs); + } + return new MapLookupExtractor(map, false); + } + }, + GUAVA { + @Override + public LookupExtractor build(Iterable> keyValuePairs) + { + final ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); + for (final Pair keyValuePair : keyValuePairs) { + mapBuilder.put(keyValuePair.lhs, keyValuePair.rhs); + } + return new MapLookupExtractor(mapBuilder.build(), false); + } + }, + FASTUTIL { + @Override + public LookupExtractor build(Iterable> keyValuePairs) + { + final Map map = new Object2ObjectOpenHashMap<>(); + for (final Pair keyValuePair : keyValuePairs) { + map.put(keyValuePair.lhs, keyValuePair.rhs); + } + return new MapLookupExtractor(map, false); + } + }; + + public abstract LookupExtractor build(Iterable> keyValuePairs); + } + + private LookupBenchmarkUtil() + { + // No instantiation. + } + + /** + * Create a {@link LookupExtractor} for benchmarking. Keys are numbers from 0 (inclusive) to numKeys (exclusive), + * as strings. Values are numbers from 0 (inclusive) to numValues (exclusive), cycled if numValues < numKeys. + */ + public static LookupExtractor makeLookupExtractor(final LookupType lookupType, final int numKeys, final int numValues) + { + if (numValues > numKeys) { + throw new IAE("numValues[%s] > numKeys[%s]", numValues, numKeys); + } + + final Iterable> keys = + () -> IntStream.range(0, numKeys) + .mapToObj(i -> Pair.of(String.valueOf(i), String.valueOf(i % numValues))) + .iterator(); + + return lookupType.build(keys); + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/LookupExtractorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/LookupExtractorBenchmark.java new file mode 100644 index 00000000000..ddc57dfc0d3 --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/LookupExtractorBenchmark.java @@ -0,0 +1,121 @@ +/* + * 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.benchmark.lookup; + +import com.google.common.base.Preconditions; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.lookup.LookupExtractor; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** + * Benchmark for various important operations on {@link LookupExtractor}. + */ +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 2) +@Measurement(iterations = 5) +public class LookupExtractorBenchmark +{ + static { + NullHandling.initializeForTests(); + } + + /** + * Type of lookup to benchmark. All are members of enum {@link LookupBenchmarkUtil.LookupType}. + */ + @Param({"hashmap", "guava", "fastutil"}) + private String lookupType; + + /** + * Number of keys in the lookup table. + */ + @Param({"1000000"}) + private int numKeys; + + /** + * Average number of keys that map to each value. + */ + @Param({"1", "1000"}) + private int keysPerValue; + + private LookupExtractor lookup; + private Set oneThousandValues; + + @Setup(Level.Trial) + public void setup() + { + final int numValues = Math.max(1, numKeys / keysPerValue); + lookup = LookupBenchmarkUtil.makeLookupExtractor( + LookupBenchmarkUtil.LookupType.valueOf(StringUtils.toUpperCase(lookupType)), + numKeys, + numValues + ); + + Preconditions.checkArgument(lookup.keySet().size() == numKeys); + + // Set of values to unapply for the benchmark lookupUnapplyOneThousand. + oneThousandValues = new HashSet<>(); + for (int i = 0; i < 1000; i++) { + oneThousandValues.add(String.valueOf(i)); + } + } + + @Benchmark + @BenchmarkMode(Mode.Throughput) + @OutputTimeUnit(TimeUnit.SECONDS) + public void lookupApply(Blackhole blackhole) + { + blackhole.consume(lookup.apply("0")); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void lookupUnapplyOne(Blackhole blackhole) + { + blackhole.consume(lookup.unapplyAll(Collections.singleton("0"))); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void lookupUnapplyOneThousand(Blackhole blackhole) + { + blackhole.consume(lookup.unapplyAll(oneThousandValues)); + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/SqlReverseLookupBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/SqlReverseLookupBenchmark.java new file mode 100644 index 00000000000..c2131c252c8 --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/lookup/SqlReverseLookupBenchmark.java @@ -0,0 +1,156 @@ +/* + * 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.benchmark.lookup; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.benchmark.query.SqlBenchmark; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; +import org.apache.druid.segment.generator.SegmentGenerator; +import org.apache.druid.sql.calcite.planner.DruidPlanner; +import org.apache.druid.sql.calcite.planner.PlannerFactory; +import org.apache.druid.sql.calcite.planner.PlannerResult; +import org.apache.druid.sql.calcite.rule.ReverseLookupRule; +import org.apache.druid.sql.calcite.run.SqlEngine; +import org.apache.druid.timeline.DataSegment; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import javax.annotation.Nullable; +import java.util.concurrent.TimeUnit; + +/** + * Benchmark for SQL planning involving {@link ReverseLookupRule}. + */ +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 2) +@Measurement(iterations = 5) +public class SqlReverseLookupBenchmark +{ + static { + NullHandling.initializeForTests(); + } + + /** + * Type of lookup to benchmark. All are members of enum {@link LookupBenchmarkUtil.LookupType}. + */ + @Param({"hashmap"}) + private String lookupType; + + /** + * Number of keys in the lookup table. + */ + @Param({"5000000"}) + private int numKeys; + + /** + * Average number of keys that map to each value. + */ + @Param({"100000", "200000", "400000", "800000", "1600000"}) + private int keysPerValue; + + private SqlEngine engine; + private LookupExtractor lookup; + + @Nullable + private PlannerFactory plannerFactory; + private final Closer closer = Closer.create(); + + @Setup(Level.Trial) + public void setup() + { + final int numValues = Math.max(1, numKeys / keysPerValue); + lookup = LookupBenchmarkUtil.makeLookupExtractor( + LookupBenchmarkUtil.LookupType.valueOf(StringUtils.toUpperCase(lookupType)), + numKeys, + numValues + ); + + final GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); + final DataSegment dataSegment = schemaInfo.makeSegmentDescriptor("foo"); + final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); + + final QueryableIndex index = + segmentGenerator.generate(dataSegment, schemaInfo, IndexSpec.DEFAULT, Granularities.NONE, 1); + + final Pair sqlSystem = SqlBenchmark.createSqlSystem( + ImmutableMap.of(dataSegment, index), + ImmutableMap.of("benchmark-lookup", lookup), + null, + closer + ); + + plannerFactory = sqlSystem.lhs; + engine = sqlSystem.rhs; + } + + @TearDown(Level.Trial) + public void tearDown() throws Exception + { + closer.close(); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void planEquals(Blackhole blackhole) + { + final String sql = + "SELECT COUNT(*) FROM foo WHERE LOOKUP(dimZipf, 'benchmark-lookup', 'N/A') = '0'"; + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, ImmutableMap.of())) { + final PlannerResult plannerResult = planner.plan(); + blackhole.consume(plannerResult); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MILLISECONDS) + public void planNotEquals(Blackhole blackhole) + { + final String sql = + "SELECT COUNT(*) FROM foo WHERE LOOKUP(dimZipf, 'benchmark-lookup', 'N/A') <> '0'"; + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, ImmutableMap.of())) { + final PlannerResult plannerResult = planner.plan(); + blackhole.consume(plannerResult); + } + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index f153a4ff28b..fbdc6bff7fa 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -19,16 +19,28 @@ package org.apache.druid.benchmark.query; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.multibindings.MapBinder; import org.apache.druid.common.config.NullHandling; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.guice.ExpressionModule; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.SegmentWranglerModule; +import org.apache.druid.guice.StartupInjectorBuilder; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchApproxCountDistinctSqlAggregator; @@ -36,6 +48,7 @@ import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchApproxCo import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchApproxQuantileSqlAggregator; import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchObjectSqlAggregator; import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchApproxCountDistinctSqlAggregator; +import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -45,11 +58,13 @@ import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; +import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.sql.calcite.aggregation.ApproxCountDistinctSqlAggregator; +import org.apache.druid.sql.calcite.aggregation.SqlAggregationModule; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.aggregation.builtin.CountSqlAggregator; import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; @@ -64,8 +79,10 @@ import org.apache.druid.sql.calcite.planner.PlannerResult; import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.schema.DruidSchemaCatalog; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.calcite.util.LookylooModule; +import org.apache.druid.sql.calcite.util.QueryFrameworkUtils; +import org.apache.druid.sql.calcite.util.testoperator.CalciteTestOperatorModule; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -82,6 +99,7 @@ import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; import javax.annotation.Nullable; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -461,66 +479,112 @@ public class SqlBenchmark public void setup() { final GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); - - final DataSegment dataSegment = DataSegment.builder() - .dataSource("foo") - .interval(schemaInfo.getDataInterval()) - .version("1") - .shardSpec(new LinearShardSpec(0)) - .size(0) - .build(); - - final PlannerConfig plannerConfig = new PlannerConfig(); - + final DataSegment dataSegment = schemaInfo.makeSegmentDescriptor("foo"); final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); + log.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment); - StringEncodingStrategy encodingStrategy; - if (stringEncoding.startsWith("front-coded")) { - String[] split = stringEncoding.split("-"); - int bucketSize = Integer.parseInt(split[2]); - encodingStrategy = new StringEncodingStrategy.FrontCoded(bucketSize, FrontCodedIndexed.V1); - } else { - encodingStrategy = new StringEncodingStrategy.Utf8(); - } + final QueryableIndex index = segmentGenerator.generate( dataSegment, schemaInfo, - IndexSpec.builder().withStringDictionaryEncoding(encodingStrategy).build(), + IndexSpec.builder().withStringDictionaryEncoding(getStringEncodingStrategy()).build(), Granularities.NONE, rowsPerSegment ); - final QueryRunnerFactoryConglomerate conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(closer); + final Pair sqlSystem = createSqlSystem( + ImmutableMap.of(dataSegment, index), + Collections.emptyMap(), + null, + closer + ); + plannerFactory = sqlSystem.lhs; + engine = sqlSystem.rhs; + } + + private StringEncodingStrategy getStringEncodingStrategy() + { + if (stringEncoding.startsWith("front-coded")) { + String[] split = stringEncoding.split("-"); + int bucketSize = Integer.parseInt(split[2]); + return new StringEncodingStrategy.FrontCoded(bucketSize, FrontCodedIndexed.V1); + } else { + return new StringEncodingStrategy.Utf8(); + } + } + + public static Pair createSqlSystem( + final Map segmentMap, + final Map lookupMap, + @Nullable final String storageType, + final Closer closer + ) + { + final QueryRunnerFactoryConglomerate conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate(closer); final SpecificSegmentsQuerySegmentWalker walker = new SpecificSegmentsQuerySegmentWalker(conglomerate); - addSegmentToWalker(walker, dataSegment, index); - closer.register(walker); + final PlannerConfig plannerConfig = new PlannerConfig(); + + for (final Map.Entry segmentEntry : segmentMap.entrySet()) { + addSegmentToWalker(walker, segmentEntry.getKey(), segmentEntry.getValue(), storageType); + } + + // Child injector that adds additional lookups. + final Injector injector = new StartupInjectorBuilder() + .withEmptyProperties() + .add( + new ExpressionModule(), + new SegmentWranglerModule(), + new LookylooModule(), + new SqlAggregationModule(), + new CalciteTestOperatorModule(), + binder -> { + for (Map.Entry entry : lookupMap.entrySet()) { + MapBinder.newMapBinder(binder, String.class, LookupExtractor.class) + .addBinding(entry.getKey()) + .toProvider(entry::getValue) + .in(LazySingleton.class); + } + } + ) + .build(); final DruidSchemaCatalog rootSchema = - CalciteTests.createMockRootSchema(conglomerate, walker, plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER); - engine = CalciteTests.createMockSqlEngine(walker, conglomerate); - plannerFactory = new PlannerFactory( + QueryFrameworkUtils.createMockRootSchema( + injector, + conglomerate, + walker, + plannerConfig, + AuthTestUtils.TEST_AUTHORIZER_MAPPER + ); + + final SqlEngine engine = CalciteTests.createMockSqlEngine(walker, conglomerate); + + final PlannerFactory plannerFactory = new PlannerFactory( rootSchema, - createOperatorTable(), - CalciteTests.createExprMacroTable(), + createOperatorTable(injector), + injector.getInstance(ExprMacroTable.class), plannerConfig, AuthTestUtils.TEST_AUTHORIZER_MAPPER, - CalciteTests.getJsonMapper(), + injector.getInstance(Key.get(ObjectMapper.class, Json.class)), CalciteTests.DRUID_SCHEMA_NAME, new CalciteRulesManager(ImmutableSet.of()), - CalciteTests.createJoinableFactoryWrapper(), + new JoinableFactoryWrapper(QueryFrameworkUtils.createDefaultJoinableFactory(injector)), CatalogResolver.NULL_RESOLVER, new AuthConfig() ); + + return Pair.of(plannerFactory, engine); } - private void addSegmentToWalker( + private static void addSegmentToWalker( final SpecificSegmentsQuerySegmentWalker walker, final DataSegment descriptor, - final QueryableIndex index + final QueryableIndex index, + @Nullable final String storageType ) { - if (STORAGE_MMAP.equals(storageType)) { + if (storageType == null || STORAGE_MMAP.equals(storageType)) { walker.add(descriptor, new QueryableIndexSegment(index, descriptor.getId())); } else if (STORAGE_FRAME_ROW.equals(storageType)) { walker.add( @@ -540,14 +604,16 @@ public class SqlBenchmark descriptor.getId() ) ); + } else { + throw new IAE("Invalid storageType[%s]", storageType); } } - private static DruidOperatorTable createOperatorTable() + private static DruidOperatorTable createOperatorTable(final Injector injector) { try { final Set extractionOperators = new HashSet<>(); - extractionOperators.add(CalciteTests.INJECTOR.getInstance(QueryLookupOperatorConversion.class)); + extractionOperators.add(injector.getInstance(QueryLookupOperatorConversion.class)); final ApproxCountDistinctSqlAggregator countDistinctSqlAggregator = new ApproxCountDistinctSqlAggregator(new HllSketchApproxCountDistinctSqlAggregator()); final Set aggregators = new HashSet<>( diff --git a/docs/development/extensions-core/lookups-cached-global.md b/docs/development/extensions-core/lookups-cached-global.md index 13883c7aa79..15e1469de4a 100644 --- a/docs/development/extensions-core/lookups-cached-global.md +++ b/docs/development/extensions-core/lookups-cached-global.md @@ -84,7 +84,7 @@ The parameters are as follows |--------|-----------|--------|-------| |`extractionNamespace`|Specifies how to populate the local cache. See below|Yes|-| |`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`0` (do not wait)| -|`injective`|If the underlying map is [injective](../../querying/lookups.md#query-execution) (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`| +|`injective`|If the underlying map is [injective](../../querying/lookups.md#query-rewrites) (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`| If `firstCacheTimeout` is set to a non-zero value, it should be less than `druid.manager.lookups.hostUpdateTimeout`. If `firstCacheTimeout` is NOT set, then management is essentially asynchronous and does not know if a lookup succeeded or failed in starting. In such a case logs from the processes using lookups should be monitored for repeated failures. diff --git a/docs/querying/dimensionspecs.md b/docs/querying/dimensionspecs.md index d336d746d45..cda6d289799 100644 --- a/docs/querying/dimensionspecs.md +++ b/docs/querying/dimensionspecs.md @@ -376,7 +376,7 @@ be treated as missing. It is illegal to set `retainMissingValue = true` and also specify a `replaceMissingValueWith`. A property of `injective` can override the lookup's own sense of whether or not it is -[injective](lookups.md#query-execution). If left unspecified, Druid will use the registered cluster-wide lookup +[injective](lookups.md#injective-lookups). If left unspecified, Druid will use the registered cluster-wide lookup configuration. A property `optimize` can be supplied to allow optimization of lookup based extraction filter (by default `optimize = true`). diff --git a/docs/querying/lookups.md b/docs/querying/lookups.md index 10aa6690ec2..aac7abd4cd6 100644 --- a/docs/querying/lookups.md +++ b/docs/querying/lookups.md @@ -31,8 +31,8 @@ refers to a dimension value to match, and a "value" refers to its replacement. S It is worth noting that lookups support not just use cases where keys map one-to-one to unique values, such as country code and country name, but also support use cases where multiple IDs map to the same value, e.g. multiple app-ids -mapping to a single account manager. When lookups are one-to-one, Druid is able to apply additional optimizations at -query time; see [Query execution](#query-execution) below for more details. +mapping to a single account manager. When lookups are one-to-one, Druid is able to apply additional +[query rewrites](#query-rewrites); see below for more details. Lookups do not have history. They always use the current data. This means that if the chief account manager for a particular app-id changes, and you issue a query with a lookup to store the app-id to account manager relationship, @@ -62,14 +62,13 @@ SELECT FROM sales GROUP BY 1 ``` -The lookup function also accepts the 3rd argument called `replaceMissingValueWith` as a constant string. If your value is missing a lookup for the queried key, the lookup function returns the result value from `replaceMissingValueWith` -For example: -``` -LOOKUP(store, 'store_to_country', 'NA') -``` -If value is missing from `store_to_country` lookup for given key 'store' then it will return `NA`. -They can also be queried using the [JOIN operator](datasource.md#join): +The `LOOKUP` function also accepts a third argument called `replaceMissingValueWith` as a constant string. If the lookup +does not contain a value for the provided key, then the `LOOKUP` function returns this `replaceMissingValueWith` value +rather than `NULL`, just like `COALESCE`. For example, `LOOKUP(store, 'store_to_country', 'NA')` is equivalent to +`COALESCE(LOOKUP(store, 'store_to_country'), 'NA')`. + +Lookups can be queried using the [JOIN operator](datasource.md#join): ```sql SELECT @@ -81,45 +80,145 @@ FROM GROUP BY 1 ``` +:::info +The `LOOKUP` function has automatic [query rewrites](#query-rewrites) available that the `JOIN` approach does not, +including [reverse lookups](#reverse-lookup) and [pulling up through `GROUP BY`](#pull-up). If these rewrites are +important for you, consider using the `LOOKUP` function instead of `JOIN`. +::: + In native queries, lookups can be queried with [dimension specs or extraction functions](dimensionspecs.md). -Query Execution ---------------- -When executing an aggregation query involving lookup functions, like the SQL [`LOOKUP` function](sql-scalar.md#string-functions), -Druid can decide to apply them while scanning and aggregating rows, or to apply them after aggregation is complete. It -is more efficient to apply lookups after aggregation is complete, so Druid will do this if it can. Druid decides this -by checking if the lookup is marked as "injective" or not. In general, you should set this property for any lookup that -is naturally one-to-one, to allow Druid to run your queries as fast as possible. +Query Rewrites +-------------- +Druid can perform two automatic query rewrites when using the `LOOKUP` function: [reverse lookups](#reverse-lookup) and +[pulling up through `GROUP BY`](#pull-up). These rewrites and their requirements are described in the following +sections. -Injective lookups should include _all_ possible keys that may show up in your dataset, and should also map all keys to -_unique values_. This matters because non-injective lookups may map different keys to the same value, which must be -accounted for during aggregation, lest query results contain two result values that should have been aggregated into -one. +### Reverse lookup -This lookup is injective (assuming it contains all possible keys from your data): +When `LOOKUP` function calls appear in the `WHERE` clause of a query, Druid reverses them [when possible](#table). +For example, if the lookup table `sku_to_name` contains the mapping `'WB00013' => 'WhizBang Sprocket'`, then Druid +automatically rewrites this query: -``` -1 -> Foo -2 -> Bar -3 -> Billy +```sql +SELECT + LOOKUP(sku, 'sku_to_name') AS name, + SUM(revenue) +FROM sales +WHERE LOOKUP(sku, 'sku_to_name') = 'WhizBang Sprocket' +GROUP BY LOOKUP(sku, 'sku_to_name') ``` -But this one is not, since both "2" and "3" map to the same value: +Into this: -``` -1 -> Foo -2 -> Bar -3 -> Bar +```sql +SELECT + LOOKUP(sku, 'sku_to_name') AS name, + SUM(revenue) +FROM sales +WHERE sku = 'WB00013' +GROUP BY LOOKUP(sku, 'sku_to_name') ``` -To tell Druid that your lookup is injective, you must specify `"injective" : true` in the lookup configuration. Druid -will not detect this automatically. +The difference is that in the latter case, data servers do not need to apply the `LOOKUP` function while filtering, and +can make more efficient use of indexes for `sku`. -:::info - Currently, the injective lookup optimization is not triggered when lookups are inputs to a - [join datasource](datasource.md#join). It is only used when lookup functions are used directly, without the join - operator. -::: +The following table contains examples of when it is possible to reverse calls to `LOOKUP` while in +Druid's default null handling mode. The list of examples is illustrative, albeit not exhaustive. + +|SQL|Reversible?| +|---|-----------| +|`LOOKUP(sku, 'sku_to_name') = 'WhizBang Sprocket'`|Yes| +|`LOOKUP(sku, 'sku_to_name') IS NOT DISTINCT FROM 'WhizBang Sprocket'`|Yes, for non-null literals| +|`LOOKUP(sku, 'sku_to_name') <> 'WhizBang Sprocket'`|No, unless `sku_to_name` is [injective](#injective-lookups)| +|`LOOKUP(sku, 'sku_to_name') IS DISTINCT FROM 'WhizBang Sprocket'`|Yes, for non-null literals| +|`LOOKUP(sku, 'sku_to_name') = 'WhizBang Sprocket' IS NOT TRUE`|Yes| +|`LOOKUP(sku, 'sku_to_name') IN ('WhizBang Sprocket', 'WhizBang Chain')`|Yes| +|`LOOKUP(sku, 'sku_to_name') NOT IN ('WhizBang Sprocket', 'WhizBang Chain')`|No, unless `sku_to_name` is [injective](#injective-lookups)| +|`LOOKUP(sku, 'sku_to_name') IN ('WhizBang Sprocket', 'WhizBang Chain') IS NOT TRUE`|Yes| +|`LOOKUP(sku, 'sku_to_name') IS NULL`|No| +|`LOOKUP(sku, 'sku_to_name') IS NOT NULL`|No| +|`LOOKUP(UPPER(sku), 'sku_to_name') = 'WhizBang Sprocket'`|Yes, to `UPPER(sku) = [key for 'WhizBang Sprocket']` (the `UPPER` function remains)| +|`COALESCE(LOOKUP(sku, 'sku_to_name'), 'N/A') = 'WhizBang Sprocket'`|Yes, but see next item for `= 'N/A'`| +|`COALESCE(LOOKUP(sku, 'sku_to_name'), 'N/A') = 'N/A'`|No, unless `sku_to_name` is [injective](#injective-lookups), which allows Druid to ignore the `COALESCE`| +|`COALESCE(LOOKUP(sku, 'sku_to_name'), 'N/A') = 'WhizBang Sprocket' IS NOT TRUE`|Yes| +|`COALESCE(LOOKUP(sku, 'sku_to_name'), 'N/A') <> 'WhizBang Sprocket'`|Yes, but see next item for `<> 'N/A'`| +|`COALESCE(LOOKUP(sku, 'sku_to_name'), 'N/A') <> 'N/A'`|No, unless `sku_to_name` is [injective](#injective-lookups), which allows Druid to ignore the `COALESCE`| +|`COALESCE(LOOKUP(sku, 'sku_to_name'), sku) = 'WhizBang Sprocket'`|No, `COALESCE` is only reversible when the second argument is a constant| +|`LOWER(LOOKUP(sku, 'sku_to_name')) = 'whizbang sprocket'`|No, functions other than `COALESCE` are not reversible| +|`MV_CONTAINS(LOOKUP(sku, 'sku_to_name'), 'WhizBang Sprocket')`|Yes| +|`NOT MV_CONTAINS(LOOKUP(sku, 'sku_to_name'), 'WhizBang Sprocket')`|No, unless `sku_to_name` is [injective](#injective-lookups)| +|`MV_OVERLAP(LOOKUP(sku, 'sku_to_name'), ARRAY['WhizBang Sprocket'])`|Yes| +|`NOT MV_OVERLAP(LOOKUP(sku, 'sku_to_name'), ARRAY['WhizBang Sprocket'])`|No, unless `sku_to_name` is [injective](#injective-lookups)| + +You can see the difference in the native query that is generated during SQL planning, which you +can retrieve with [`EXPLAIN PLAN FOR`](sql.md#explain-plan). When a lookup is reversed in this way, the `lookup` +function disappears and is replaced by a simpler filter, typically of type `equals` or `in`. + +Lookups are not reversed if the number of matching keys exceeds the [`inSubQueryThreshold`](sql-query-context.md) for +the query. + +This rewrite adds some planning time that may become noticeable for larger lookups, especially if many keys map to the +same value. You can see the impact on planning time in the `sqlQuery/planningTimeMs` metric. You can also measure the +time taken by `EXPLAIN PLAN FOR`, which plans the query but does not execute it. + +This rewrite can be disabled by setting [`sqlReverseLookup: false`](sql-query-context.md) in your query context. + +### Pull up + +Lookups marked as [_injective_](#injective-lookups) can be pulled up through a `GROUP BY`. For example, if the lookup +`sku_to_name` is injective, Druid automatically rewrites this query: + +```sql +SELECT + LOOKUP(sku, 'sku_to_name') AS name, + SUM(revenue) +FROM sales +GROUP BY LOOKUP(sku, 'sku_to_name') +``` + +Into this: + +```sql +SELECT + LOOKUP(sku, 'sku_to_name') AS name, + SUM(revenue) +FROM sales +GROUP BY sku +``` + +The difference is that the `LOOKUP` function is not applied until after the `GROUP BY` is finished, which speeds up +the `GROUP BY`. + +You can see the difference in the native query that is generated during SQL planning, which you +can retrieve with [`EXPLAIN PLAN FOR`](sql.md#explain-plan). When a lookup is pulled up in this way, the `lookup` +function call typically moves from the `virtualColumns` or `dimensions` section of a native query into the +`postAggregations`. + +This rewrite can be disabled by setting [`sqlPullUpLookup: false`](sql-query-context.md) in your query context. + +### Injective lookups + +Injective lookups are eligible for the largest set of query rewrites. Injective lookups must satisfy the following +"one-to-one lookup" properties: + +- All values in the lookup table must be unique. That is, no two keys can map to the same value. +- The lookup table must have a key-value pair defined for every input that the `LOOKUP` function call may + encounter. For example, when calling `LOOKUP(sku, 'sku_to_name')`, the `sku_to_name` lookup table must have a key + for all possible `sku`. +- In SQL-compatible null handling mode (when `druid.generic.useDefaultValueForNull = false`, the default) injective + lookup tables are not required to have keys for `null`, since `LOOKUP` of `null` is always `null` itself. +- When `druid.generic.useDefaultValueForNull = true`, a `LOOKUP` of `null` retrieves the value mapped to the + empty-string key (`""`). In this mode, injective lookup tables must have an empty-string key if the `LOOKUP` + function may encounter null input values. + +To determine whether a lookup is injective, Druid relies on an `injective` property that you can set in the +[lookup definition](../development/extensions-core/lookups-cached-global.md). In general, you should set +`injective: true` for any lookup that satisfies the required properties, to allow Druid to run your queries as fast as +possible. + +Druid does not verify whether lookups satisfy these required properties. Druid may return incorrect query results +if you set `injective: true` for a lookup table that is not actually a one-to-one lookup. Dynamic Configuration --------------------- diff --git a/docs/querying/query-context.md b/docs/querying/query-context.md index 0fd35d40320..98d9c9c6aa7 100644 --- a/docs/querying/query-context.md +++ b/docs/querying/query-context.md @@ -35,7 +35,8 @@ Note that setting query context will override both the default value and the run ## General parameters -Unless otherwise noted, the following parameters apply to all query types. +Unless otherwise noted, the following parameters apply to all query types, and to both native and SQL queries. +See [SQL query context](sql-query-context.md) for other query context parameters that are specific to Druid SQL planning. |Parameter |Default | Description | |-------------------|----------------------------------------|----------------------| @@ -62,15 +63,8 @@ Unless otherwise noted, the following parameters apply to all query types. |`parallelMergeSmallBatchRows`|`druid.processing.merge.task.smallBatchNumRows`|Size of result batches to operate on in ForkJoinPool merge tasks for parallel result merging on the Broker. See [Broker configuration](../configuration/index.md#broker) for more details.| |`useFilterCNF`|`false`| If true, Druid will attempt to convert the query filter to Conjunctive Normal Form (CNF). During query processing, columns can be pre-filtered by intersecting the bitmap indexes of all values that match the eligible filters, often greatly reducing the raw number of rows which need to be scanned. But this effect only happens for the top level filter, or individual clauses of a top level 'and' filter. As such, filters in CNF potentially have a higher chance to utilize a large amount of bitmap indexes on string columns during pre-filtering. However, this setting should be used with great caution, as it can sometimes have a negative effect on performance, and in some cases, the act of computing CNF of a filter can be expensive. We recommend hand tuning your filters to produce an optimal form if possible, or at least verifying through experimentation that using this parameter actually improves your query performance with no ill-effects.| |`secondaryPartitionPruning`|`true`|Enable secondary partition pruning on the Broker. The Broker will always prune unnecessary segments from the input scan based on a filter on time intervals, but if the data is further partitioned with hash or range partitioning, this option will enable additional pruning based on a filter on secondary partition dimensions.| -|`enableJoinLeftTableScanDirect`|`false`|This flag applies to queries which have joins. For joins, where left child is a simple scan with a filter, by default, druid will run the scan as a query and the join the results to the right child on broker. Setting this flag to true overrides that behavior and druid will attempt to push the join to data servers instead. Please note that the flag could be applicable to queries even if there is no explicit join. since queries can internally translated into a join by the SQL planner.| |`debug`| `false` | Flag indicating whether to enable debugging outputs for the query. When set to false, no additional logs will be produced (logs produced will be entirely dependent on your logging level). When set to true, the following addition logs will be produced:
- Log the stack trace of the exception (if any) produced by the query | |`setProcessingThreadNames`|`true`| Whether processing thread names will be set to `queryType_dataSource_intervals` while processing a query. This aids in interpreting thread dumps, and is on by default. Query overhead can be reduced slightly by setting this to `false`. This has a tiny effect in most scenarios, but can be meaningful in high-QPS, low-per-segment-processing-time scenarios. | -|`maxNumericInFilters`|`-1`|Max limit for the amount of numeric values that can be compared for a string type dimension when the entire SQL WHERE clause of a query translates only to an [OR](../querying/filters.md#or) of [Bound filter](../querying/filters.md#bound-filter). By default, Druid does not restrict the amount of of numeric Bound Filters on String columns, although this situation may block other queries from running. Set this parameter to a smaller value to prevent Druid from running queries that have prohibitively long segment processing times. The optimal limit requires some trial and error; we recommend starting with 100. Users who submit a query that exceeds the limit of `maxNumericInFilters` should instead rewrite their queries to use strings in the `WHERE` clause instead of numbers. For example, `WHERE someString IN (‘123’, ‘456’)`. This value cannot exceed the set system configuration `druid.sql.planner.maxNumericInFilters`. This value is ignored if `druid.sql.planner.maxNumericInFilters` is not set explicitly.| -|`inSubQueryThreshold`|`20`| Threshold for minimum number of values in an IN clause to convert the query to a JOIN operation on an inlined table rather than a predicate. A threshold of 0 forces usage of an inline table in all cases; a threshold of [Integer.MAX_VALUE] forces usage of OR in all cases. | - -## Druid SQL parameters - -See [SQL query context](sql-query-context.md) for query context parameters specific to Druid SQL queries. ## Parameters by query type diff --git a/docs/querying/sql-query-context.md b/docs/querying/sql-query-context.md index dc192db1718..e72f1df0e47 100644 --- a/docs/querying/sql-query-context.md +++ b/docs/querying/sql-query-context.md @@ -47,6 +47,11 @@ Configure Druid SQL query planning using the parameters in the table below. |`useNativeQueryExplain`|If true, `EXPLAIN PLAN FOR` will return the explain plan as a JSON representation of equivalent native query(s), else it will return the original version of explain plan generated by Calcite.

This property is provided for backwards compatibility. It is not recommended to use this parameter unless you were depending on the older behavior.|`druid.sql.planner.useNativeQueryExplain` on the Broker (default: true)| |`sqlFinalizeOuterSketches`|If false (default behavior in Druid 25.0.0 and later), `DS_HLL`, `DS_THETA`, and `DS_QUANTILES_SKETCH` return sketches in query results, as documented. If true (default behavior in Druid 24.0.1 and earlier), sketches from these functions are finalized when they appear in query results.

This property is provided for backwards compatibility with behavior in Druid 24.0.1 and earlier. It is not recommended to use this parameter unless you were depending on the older behavior. Instead, use a function that does not return a sketch, such as `APPROX_COUNT_DISTINCT_DS_HLL`, `APPROX_COUNT_DISTINCT_DS_THETA`, `APPROX_QUANTILE_DS`, `DS_THETA_ESTIMATE`, or `DS_GET_QUANTILE`.|`druid.query.default.context.sqlFinalizeOuterSketches` on the Broker (default: false)| |`sqlUseBoundAndSelectors`|If false (default behavior if `druid.generic.useDefaultValueForNull=false` in Druid 27.0.0 and later), the SQL planner will use [equality](./filters.md#equality-filter), [null](./filters.md#null-filter), and [range](./filters.md#range-filter) filters instead of [selector](./filters.md#selector-filter) and [bounds](./filters.md#bound-filter). This value must be set to `false` for correct behavior for filtering `ARRAY` typed values. | Defaults to same value as `druid.generic.useDefaultValueForNull`, which is `false`| +|`sqlReverseLookup`|Whether to consider the [reverse-lookup rewrite](lookups.md#reverse-lookup) of the `LOOKUP` function during SQL planning.|true| +|`sqlPullUpLookup`|Whether to consider the [pull-up rewrite](lookups.md#pull-up) of the `LOOKUP` function during SQL planning.|true| +|`enableJoinLeftTableScanDirect`|`false`|This flag applies to queries which have joins. For joins, where left child is a simple scan with a filter, by default, druid will run the scan as a query and the join the results to the right child on broker. Setting this flag to true overrides that behavior and druid will attempt to push the join to data servers instead. Please note that the flag could be applicable to queries even if there is no explicit join. since queries can internally translated into a join by the SQL planner.| +|`maxNumericInFilters`|`-1`|Max limit for the amount of numeric values that can be compared for a string type dimension when the entire SQL WHERE clause of a query translates only to an [OR](../querying/filters.md#or) of [Bound filter](../querying/filters.md#bound-filter). By default, Druid does not restrict the amount of of numeric Bound Filters on String columns, although this situation may block other queries from running. Set this parameter to a smaller value to prevent Druid from running queries that have prohibitively long segment processing times. The optimal limit requires some trial and error; we recommend starting with 100. Users who submit a query that exceeds the limit of `maxNumericInFilters` should instead rewrite their queries to use strings in the `WHERE` clause instead of numbers. For example, `WHERE someString IN (‘123’, ‘456’)`. This value cannot exceed the set system configuration `druid.sql.planner.maxNumericInFilters`. This value is ignored if `druid.sql.planner.maxNumericInFilters` is not set explicitly.| +|`inSubQueryThreshold`|`20`| Threshold for minimum number of values in an IN clause to convert the query to a JOIN operation on an inlined table rather than a predicate. A threshold of 0 forces usage of an inline table in all cases; a threshold of [Integer.MAX_VALUE] forces usage of OR in all cases. | ## Setting the query context The query context parameters can be specified as a "context" object in the [JSON API](../api-reference/sql-api.md) or as a [JDBC connection properties object](../api-reference/sql-jdbc.md). diff --git a/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java index ebd23eb6e37..035fd18eeab 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java @@ -31,7 +31,6 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ForwardingSortedSet; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.collect.Iterators; import com.google.common.collect.Range; import com.google.common.collect.RangeSet; import com.google.common.collect.Sets; @@ -377,7 +376,7 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter final DimFilterToStringBuilder builder = new DimFilterToStringBuilder(); return builder.appendDimension(dimension, extractionFn) .append(" IN (") - .append(Joiner.on(", ").join(Iterables.transform(values, StringUtils::nullToEmptyNonDruidDataString))) + .append(Joiner.on(", ").join(Iterables.transform(values, String::valueOf))) .append(")") .appendFilterTuning(filterTuning) .build(); @@ -435,8 +434,21 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter * @param inFilter in filter * @param mayIncludeUnknown same as the argument to {@link #optimize(boolean)} */ + public static ValuesSet optimizeLookup(final InDimFilter inFilter, final boolean mayIncludeUnknown) + { + return optimizeLookup(inFilter, mayIncludeUnknown, Integer.MAX_VALUE); + } + + /** + * If the provided "in" filter uses a {@link LookupExtractionFn} that can be reversed, then return the matching + * set of keys as a {@link ValuesSet}. Otherwise return null. + * + * @param inFilter in filter + * @param mayIncludeUnknown same as the argument to {@link #optimize(boolean)} + * @param maxSize maximum number of values in the returned filter + */ @Nullable - static ValuesSet optimizeLookup(final InDimFilter inFilter, final boolean mayIncludeUnknown) + public static ValuesSet optimizeLookup(final InDimFilter inFilter, final boolean mayIncludeUnknown, final int maxSize) { final LookupExtractionFn exFn; @@ -507,7 +519,12 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter return null; } - Iterators.addAll(unapplied, keysIterator); + while (keysIterator.hasNext()) { + unapplied.add(keysIterator.next()); + if (unapplied.size() > maxSize) { + return null; + } + } // In SQL-compatible null handling mode, lookup of null is always "replaceMissingValueWith", regardless of contents // of the lookup. So, if we're matching against "replaceMissingValueWith", we need to include null in the diff --git a/processing/src/main/java/org/apache/druid/segment/generator/GeneratorSchemaInfo.java b/processing/src/main/java/org/apache/druid/segment/generator/GeneratorSchemaInfo.java index cb1591ab897..603e7dd0d82 100644 --- a/processing/src/main/java/org/apache/druid/segment/generator/GeneratorSchemaInfo.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/GeneratorSchemaInfo.java @@ -22,6 +22,8 @@ package org.apache.druid.segment.generator; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; import org.joda.time.Interval; import java.util.List; @@ -82,6 +84,17 @@ public class GeneratorSchemaInfo return withRollup; } + public DataSegment makeSegmentDescriptor(final String dataSource) + { + return DataSegment.builder() + .dataSource(dataSource) + .interval(getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(); + } + @Override public String toString() { diff --git a/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java b/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java index 3d62fcb22ec..0cb5f25ef73 100644 --- a/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java @@ -50,7 +50,6 @@ public class DataGeneratorTest extends InitializedNullHandlingTest { private static final Logger log = new Logger(DataGeneratorTest.class); - @Test public void testSequential() { diff --git a/processing/src/test/java/org/apache/druid/segment/generator/GeneratorSchemaInfoTest.java b/processing/src/test/java/org/apache/druid/segment/generator/GeneratorSchemaInfoTest.java new file mode 100644 index 00000000000..f9f16975fed --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/generator/GeneratorSchemaInfoTest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.generator; + +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.timeline.DataSegment; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; + +public class GeneratorSchemaInfoTest +{ + @Test + public void testMakeSegmentDescriptor() + { + final GeneratorSchemaInfo schemaInfo = new GeneratorSchemaInfo( + Collections.emptyList(), + Collections.emptyList(), + Intervals.ETERNITY, + false + ); + + final DataSegment dataSegment = schemaInfo.makeSegmentDescriptor("foo"); + Assert.assertEquals("foo", dataSegment.getDataSource()); + Assert.assertEquals(Intervals.ETERNITY, dataSegment.getInterval()); + } +} diff --git a/server/src/test/java/org/apache/druid/query/expression/LookupEnabledTestExprMacroTable.java b/server/src/test/java/org/apache/druid/query/expression/LookupEnabledTestExprMacroTable.java index c5c399e8b4c..fe40b9c5b73 100644 --- a/server/src/test/java/org/apache/druid/query/expression/LookupEnabledTestExprMacroTable.java +++ b/server/src/test/java/org/apache/druid/query/expression/LookupEnabledTestExprMacroTable.java @@ -45,7 +45,7 @@ import java.util.Set; public class LookupEnabledTestExprMacroTable extends ExprMacroTable { public static final ExprMacroTable INSTANCE = new LookupEnabledTestExprMacroTable(); - private static final String LOOKYLOO = "lookyloo"; + public static final String LOOKYLOO = "lookyloo"; private LookupEnabledTestExprMacroTable() { @@ -58,7 +58,14 @@ public class LookupEnabledTestExprMacroTable extends ExprMacroTable Iterables.concat( TestExprMacroTable.INSTANCE.getMacros(), Collections.singletonList( - new LookupExprMacro(createTestLookupProvider(theLookup)) + new LookupExprMacro( + createTestLookupProvider( + ImmutableMap.of( + LOOKYLOO, + new MapLookupExtractor(theLookup, false) + ) + ) + ) ) ) ); @@ -68,55 +75,8 @@ public class LookupEnabledTestExprMacroTable extends ExprMacroTable * Returns a {@link LookupExtractorFactoryContainerProvider} that has one lookup, "lookyloo". Public so other tests * can use this helper method directly. */ - public static LookupExtractorFactoryContainerProvider createTestLookupProvider(final Map theLookup) + public static LookupExtractorFactoryContainerProvider createTestLookupProvider(final Map lookups) { - final LookupExtractorFactoryContainer container = new LookupExtractorFactoryContainer( - "v0", - new LookupExtractorFactory() - { - @Override - public boolean start() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean close() - { - throw new UnsupportedOperationException(); - } - - @Override - public boolean replaces(@Nullable final LookupExtractorFactory other) - { - throw new UnsupportedOperationException(); - } - - @Override - public LookupIntrospectHandler getIntrospectHandler() - { - throw new UnsupportedOperationException(); - } - - @Override - public void awaitInitialization() - { - } - - @Override - public boolean isInitialized() - { - return true; - } - - @Override - public LookupExtractor get() - { - return new MapLookupExtractor(theLookup, false); - } - } - ); - return new LookupExtractorFactoryContainerProvider() { @Override @@ -128,12 +88,66 @@ public class LookupEnabledTestExprMacroTable extends ExprMacroTable @Override public Optional get(String lookupName) { - if (LOOKYLOO.equals(lookupName)) { - return Optional.of(container); + final LookupExtractor theLookup = lookups.get(lookupName); + if (theLookup != null) { + return Optional.of(new TestLookupContainer(theLookup)); } else { return Optional.empty(); } } }; } + + public static class TestLookupContainer extends LookupExtractorFactoryContainer + { + public TestLookupContainer(final LookupExtractor theLookup) + { + super( + "v0", + new LookupExtractorFactory() + { + @Override + public boolean start() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean close() + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean replaces(@Nullable final LookupExtractorFactory other) + { + throw new UnsupportedOperationException(); + } + + @Override + public LookupIntrospectHandler getIntrospectHandler() + { + throw new UnsupportedOperationException(); + } + + @Override + public void awaitInitialization() + { + } + + @Override + public boolean isInitialized() + { + return true; + } + + @Override + public LookupExtractor get() + { + return theLookup; + } + } + ); + } + } } diff --git a/sql/src/main/java/org/apache/calcite/sql/type/CastedLiteralOperandTypeCheckers.java b/sql/src/main/java/org/apache/calcite/sql/type/CastedLiteralOperandTypeCheckers.java index 0155f7ba6f4..9d8c111d074 100644 --- a/sql/src/main/java/org/apache/calcite/sql/type/CastedLiteralOperandTypeCheckers.java +++ b/sql/src/main/java/org/apache/calcite/sql/type/CastedLiteralOperandTypeCheckers.java @@ -105,30 +105,6 @@ public class CastedLiteralOperandTypeCheckers } }; - public static boolean isLiteral(SqlNode node, boolean allowCast) - { - assert node != null; - if (node instanceof SqlLiteral) { - return true; - } - if (!allowCast) { - return false; - } - switch (node.getKind()) { - case CAST: - // "CAST(e AS type)" is literal if "e" is literal - return isLiteral(((SqlCall) node).operand(0), true); - case MAP_VALUE_CONSTRUCTOR: - case ARRAY_VALUE_CONSTRUCTOR: - return ((SqlCall) node).getOperandList().stream() - .allMatch(o -> isLiteral(o, true)); - case DEFAULT: - return true; // DEFAULT is always NULL - default: - return false; - } - } - /** * Fetches primitive literals from the casts, including NULL literal. * It throws if the entered node isn't a primitive literal, which can be cast multiple times. diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/AliasedOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/AliasedOperatorConversion.java index 47e364f196d..9e42ba411c0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/AliasedOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/AliasedOperatorConversion.java @@ -35,7 +35,6 @@ import javax.annotation.Nullable; public class AliasedOperatorConversion implements SqlOperatorConversion { private final SqlOperatorConversion baseConversion; - private final String name; private final SqlOperator operator; public AliasedOperatorConversion(final SqlOperatorConversion baseConversion, final String name) @@ -47,7 +46,6 @@ public class AliasedOperatorConversion implements SqlOperatorConversion final SqlFunction baseFunction = (SqlFunction) baseConversion.calciteOperator(); this.baseConversion = baseConversion; - this.name = name; this.operator = new SqlFunction( name, baseFunction.getKind(), diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringOperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringOperatorConversions.java index 0ad0f545670..5a32b06c544 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringOperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringOperatorConversions.java @@ -54,6 +54,9 @@ import java.util.List; */ public class MultiValueStringOperatorConversions { + public static final SqlOperatorConversion CONTAINS = new Contains(); + public static final SqlOperatorConversion OVERLAP = new Overlap(); + public static class Append extends ArrayAppendOperatorConversion { private static final SqlFunction SQL_FUNCTION = OperatorConversions @@ -132,7 +135,10 @@ public class MultiValueStringOperatorConversions } } - public static class Contains extends ArrayContainsOperatorConversion + /** + * Private: use singleton {@link #CONTAINS}. + */ + private static class Contains extends ArrayContainsOperatorConversion { private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("MV_CONTAINS") @@ -300,7 +306,10 @@ public class MultiValueStringOperatorConversions } } - public static class Overlap extends AliasedOperatorConversion + /** + * Private: use singleton {@link #OVERLAP}. + */ + private static class Overlap extends AliasedOperatorConversion { public Overlap() { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java index 108bc7eb472..c21d14ced64 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java @@ -35,6 +35,7 @@ import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.OperatorConversions; import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rule.ReverseLookupRule; import java.util.List; @@ -89,7 +90,14 @@ public class QueryLookupOperatorConversion implements SqlOperatorConversion false, replaceMissingValueWith, null, - true + // If SplitLookupRule is disabled, then enable optimization at the extractionFn level, since a + // similar optimization may be done by the native query toolchests. We'd like to ensure that if + // people upgrade to a version where this rule was added, and then disable the rule due to some + // problem with it, they still get any optimization that the native layer was able to do. + // + // Note that we don't check plannerContext.isReverseLookup(), because the native layer doesn't + // optimize filters on RegisteredLookupExtractionFn anyway. + !plannerContext.isSplitLookup() ) ); } else { @@ -99,6 +107,15 @@ public class QueryLookupOperatorConversion implements SqlOperatorConversion ); } + /** + * For {@link org.apache.druid.sql.calcite.rule.DruidRules} to be able to instantiate + * {@link ReverseLookupRule}. + */ + public LookupExtractorFactoryContainerProvider getLookupExtractorFactoryContainerProvider() + { + return lookupExtractorFactoryContainerProvider; + } + private String getReplaceMissingValueWith( final List inputExpressions, final PlannerContext plannerContext diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java index 78d8ca7d098..b2ba05b3049 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java @@ -48,6 +48,7 @@ import org.apache.calcite.tools.Programs; import org.apache.calcite.tools.RelBuilder; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.sql.calcite.external.ExternalTableScanRule; +import org.apache.druid.sql.calcite.rule.AggregatePullUpLookupRule; import org.apache.druid.sql.calcite.rule.CaseToCoalesceRule; import org.apache.druid.sql.calcite.rule.CoalesceLookupRule; import org.apache.druid.sql.calcite.rule.DruidLogicalValuesRule; @@ -60,12 +61,12 @@ import org.apache.druid.sql.calcite.rule.FilterDecomposeConcatRule; import org.apache.druid.sql.calcite.rule.FilterJoinExcludePushToChildRule; import org.apache.druid.sql.calcite.rule.FlattenConcatRule; import org.apache.druid.sql.calcite.rule.ProjectAggregatePruneUnusedCallRule; +import org.apache.druid.sql.calcite.rule.ReverseLookupRule; import org.apache.druid.sql.calcite.rule.SortCollapseRule; import org.apache.druid.sql.calcite.rule.logical.DruidLogicalRules; import org.apache.druid.sql.calcite.run.EngineFeature; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import java.util.Set; @@ -140,7 +141,6 @@ public class CalciteRulesManager CoreRules.FILTER_VALUES_MERGE, CoreRules.PROJECT_FILTER_VALUES_MERGE, CoreRules.PROJECT_VALUES_MERGE, - CoreRules.SORT_PROJECT_TRANSPOSE, CoreRules.AGGREGATE_VALUES ); @@ -274,45 +274,83 @@ public class CalciteRulesManager prePrograms.add(buildReductionProgram(plannerContext, isDruid)); prePrograms.add(new LoggingProgram("Finished expression reduction program", isDebug)); + if (isDruid) { + prePrograms.add(buildPreVolcanoManipulationProgram(plannerContext)); + prePrograms.add(new LoggingProgram("Finished pre-Volcano manipulation program", isDebug)); + } + return Programs.sequence(prePrograms.toArray(new Program[0])); } + /** + * Program to perform manipulations on the logical tree prior to starting the cost-based planner. Mainly this + * helps the cost-based planner finish faster, and helps the decoupled planner generate the same plans as the + * classic planner. + */ + private Program buildPreVolcanoManipulationProgram(final PlannerContext plannerContext) + { + final HepProgramBuilder builder = HepProgram.builder(); + builder.addMatchLimit(CalciteRulesManager.HEP_DEFAULT_MATCH_LIMIT); + + // Apply FILTER_INTO_JOIN early, if using a join algorithm that requires subqueries anyway. + if (plannerContext.getJoinAlgorithm().requiresSubquery()) { + builder.addRuleInstance(CoreRules.FILTER_INTO_JOIN); + } + + // Apply SORT_PROJECT_TRANSPOSE to match the expected order of "sort" and "sortProject" in PartialDruidQuery. + builder.addRuleInstance(CoreRules.SORT_PROJECT_TRANSPOSE); + + return Programs.of(builder.build(), true, DefaultRelMetadataProvider.INSTANCE); + } + /** * Builds an expression reduction program using {@link #REDUCTION_RULES} (built-in to Calcite) plus some * Druid-specific rules. */ private Program buildReductionProgram(final PlannerContext plannerContext, final boolean isDruid) { - final List hepRules = new ArrayList<>(); + final HepProgramBuilder builder = HepProgram.builder(); + builder.addMatchLimit(CalciteRulesManager.HEP_DEFAULT_MATCH_LIMIT); if (isDruid) { - // Must run before REDUCTION_RULES, since otherwise ReduceExpressionsRule#pushPredicateIntoCase may + // COALESCE rules must run before REDUCTION_RULES, since otherwise ReduceExpressionsRule#pushPredicateIntoCase may // make it impossible to convert to COALESCE. - hepRules.add(new CaseToCoalesceRule()); - hepRules.add(new CoalesceLookupRule()); + builder.addRuleInstance(new CaseToCoalesceRule()); + builder.addRuleInstance(new CoalesceLookupRule()); + } + // Remaining rules run as a single group until fixpoint. + builder.addGroupBegin(); + + if (isDruid) { // Flatten calls to CONCAT, which happen easily with the || operator since it only accepts two arguments. - hepRules.add(new FlattenConcatRule()); + builder.addRuleInstance(new FlattenConcatRule()); // Decompose filters on COALESCE to promote more usage of indexes. - hepRules.add(new FilterDecomposeCoalesceRule()); + builder.addRuleInstance(new FilterDecomposeCoalesceRule()); + + // Decompose filters on CONCAT to promote more usage of indexes. + builder.addRuleInstance(new FilterDecomposeConcatRule()); + + // Include rule to split injective LOOKUP across a GROUP BY. + if (plannerContext.isSplitLookup()) { + builder.addRuleInstance(new AggregatePullUpLookupRule(plannerContext)); + } + + // Include rule to reduce certain LOOKUP expressions that appear in filters. + if (plannerContext.isReverseLookup()) { + builder.addRuleInstance(new ReverseLookupRule(plannerContext)); + } } // Calcite's builtin reduction rules. - hepRules.addAll(REDUCTION_RULES); - - if (isDruid) { - // Decompose filters on CONCAT to promote more usage of indexes. Runs after REDUCTION_RULES because - // this rule benefits from reduction of effectively-literal calls to actual literals. - hepRules.add(new FilterDecomposeConcatRule()); + for (final RelOptRule rule : REDUCTION_RULES) { + builder.addRuleInstance(rule); } - // Apply CoreRules#FILTER_INTO_JOIN early to avoid exploring less optimal plans. - if (isDruid && plannerContext.getJoinAlgorithm().requiresSubquery()) { - hepRules.add(CoreRules.FILTER_INTO_JOIN); - } + builder.addGroupEnd(); - return buildHepProgram(hepRules); + return Programs.of(builder.build(), true, DefaultRelMetadataProvider.INSTANCE); } private static class LoggingProgram implements Program @@ -413,22 +451,6 @@ public class CalciteRulesManager return rules.build(); } - /** - * Build a {@link HepProgram} to apply rules mechanically as part of {@link #buildPreProgram}. Rules are applied - * one-by-one. - * - * @param rules rules to apply - */ - private static Program buildHepProgram(final Collection rules) - { - final HepProgramBuilder builder = HepProgram.builder(); - builder.addMatchLimit(CalciteRulesManager.HEP_DEFAULT_MATCH_LIMIT); - for (RelOptRule rule : rules) { - builder.addRuleInstance(rule); - } - return Programs.of(builder.build(), true, DefaultRelMetadataProvider.INSTANCE); - } - /** * Based on Calcite's Programs.DecorrelateProgram and Programs.TrimFieldsProgram, which are private and only * accessible through Programs.standard (which we don't want, since it also adds Enumerable rules). diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java index f877a107210..145886686f4 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java @@ -267,8 +267,8 @@ public class DruidOperatorTable implements SqlOperatorTable .add(new MultiValueStringOperatorConversions.Append()) .add(new MultiValueStringOperatorConversions.Prepend()) .add(new MultiValueStringOperatorConversions.Concat()) - .add(new MultiValueStringOperatorConversions.Contains()) - .add(new MultiValueStringOperatorConversions.Overlap()) + .add(MultiValueStringOperatorConversions.CONTAINS) + .add(MultiValueStringOperatorConversions.OVERLAP) .add(new MultiValueStringOperatorConversions.Length()) .add(new MultiValueStringOperatorConversions.Offset()) .add(new MultiValueStringOperatorConversions.Ordinal()) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidTypeSystem.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidTypeSystem.java index dcba20ee6c4..521291c3244 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidTypeSystem.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidTypeSystem.java @@ -35,8 +35,6 @@ public class DruidTypeSystem implements RelDataTypeSystem */ public static final int DEFAULT_TIMESTAMP_PRECISION = 3; - public static final String VARIANT_TYPE_NAME = "VARIANT"; - private DruidTypeSystem() { // Singleton. diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 29f22d46c57..625546a4d21 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -37,11 +37,18 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContext; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; +import org.apache.druid.query.lookup.RegisteredLookupExtractionFn; import org.apache.druid.segment.join.JoinableFactoryWrapper; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.QueryLookupOperatorConversion; import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; +import org.apache.druid.sql.calcite.rule.AggregatePullUpLookupRule; +import org.apache.druid.sql.calcite.rule.ReverseLookupRule; import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.run.QueryMaker; import org.apache.druid.sql.calcite.run.SqlEngine; @@ -82,9 +89,23 @@ public class PlannerContext */ public static final String CTX_ENABLE_WINDOW_FNS = "enableWindowing"; + /** + * Context key for {@link PlannerContext#isUseBoundsAndSelectors()}. + */ public static final String CTX_SQL_USE_BOUNDS_AND_SELECTORS = "sqlUseBoundAndSelectors"; public static final boolean DEFAULT_SQL_USE_BOUNDS_AND_SELECTORS = NullHandling.replaceWithDefault(); + /** + * Context key for {@link PlannerContext#isSplitLookup()}. + */ + public static final String CTX_SQL_PULL_UP_LOOKUP = "sqlPullUpLookup"; + public static final boolean DEFAULT_SQL_PULL_UP_LOOKUP = true; + + /** + * Context key for {@link PlannerContext#isReverseLookup()}. + */ + public static final String CTX_SQL_REVERSE_LOOKUP = "sqlReverseLookup"; + public static final boolean DEFAULT_SQL_REVERSE_LOOKUP = true; // DataContext keys public static final String DATA_CTX_AUTHENTICATION_RESULT = "authenticationResult"; @@ -99,6 +120,8 @@ public class PlannerContext private final String sqlQueryId; private final boolean stringifyArrays; private final boolean useBoundsAndSelectors; + private final boolean splitLookup; + private final boolean reverseLookup; private final CopyOnWriteArrayList nativeQueryIds = new CopyOnWriteArrayList<>(); private final PlannerHook hook; // bindings for dynamic parameters to bind during planning @@ -116,6 +139,7 @@ public class PlannerContext private VirtualColumnRegistry joinExpressionVirtualColumnRegistry; // set of attributes for a SQL statement used in the EXPLAIN PLAN output private ExplainAttributes explainAttributes; + private PlannerLookupCache lookupCache; private PlannerContext( final PlannerToolbox plannerToolbox, @@ -124,6 +148,8 @@ public class PlannerContext final DateTime localNow, final boolean stringifyArrays, final boolean useBoundsAndSelectors, + final boolean splitLookup, + final boolean reverseLookup, final SqlEngine engine, final Map queryContext, final PlannerHook hook @@ -138,6 +164,8 @@ public class PlannerContext this.localNow = Preconditions.checkNotNull(localNow, "localNow"); this.stringifyArrays = stringifyArrays; this.useBoundsAndSelectors = useBoundsAndSelectors; + this.splitLookup = splitLookup; + this.reverseLookup = reverseLookup; this.hook = hook == null ? NoOpPlannerHook.INSTANCE : hook; String sqlQueryId = (String) this.queryContext.get(QueryContexts.CTX_SQL_QUERY_ID); @@ -160,11 +188,15 @@ public class PlannerContext final DateTimeZone timeZone; final boolean stringifyArrays; final boolean useBoundsAndSelectors; + final boolean splitLookup; + final boolean reverseLookup; final Object stringifyParam = queryContext.get(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS); final Object tsParam = queryContext.get(CTX_SQL_CURRENT_TIMESTAMP); final Object tzParam = queryContext.get(CTX_SQL_TIME_ZONE); final Object useBoundsAndSelectorsParam = queryContext.get(CTX_SQL_USE_BOUNDS_AND_SELECTORS); + final Object splitLookupParam = queryContext.get(CTX_SQL_PULL_UP_LOOKUP); + final Object reverseLookupParam = queryContext.get(CTX_SQL_REVERSE_LOOKUP); if (tsParam != null) { utcNow = new DateTime(tsParam, DateTimeZone.UTC); @@ -190,6 +222,18 @@ public class PlannerContext useBoundsAndSelectors = DEFAULT_SQL_USE_BOUNDS_AND_SELECTORS; } + if (splitLookupParam != null) { + splitLookup = Numbers.parseBoolean(splitLookupParam); + } else { + splitLookup = DEFAULT_SQL_PULL_UP_LOOKUP; + } + + if (reverseLookupParam != null) { + reverseLookup = Numbers.parseBoolean(reverseLookupParam); + } else { + reverseLookup = DEFAULT_SQL_REVERSE_LOOKUP; + } + return new PlannerContext( plannerToolbox, sql, @@ -197,6 +241,8 @@ public class PlannerContext utcNow.withZone(timeZone), stringifyArrays, useBoundsAndSelectors, + splitLookup, + reverseLookup, engine, queryContext, hook @@ -255,7 +301,6 @@ public class PlannerContext } - /** * Equivalent to {@link ExpressionParser#parse(String)} on {@link #getExpressionParser()}. */ @@ -332,6 +377,25 @@ public class PlannerContext return useBoundsAndSelectors; } + /** + * Whether we should use {@link AggregatePullUpLookupRule} to split LOOKUP functions on injective lookups when they + * are dimensions in aggregations, and whether we should set the "optimize" flag on + * {@link RegisteredLookupExtractionFn}. + */ + public boolean isSplitLookup() + { + return splitLookup; + } + + /** + * Whether we should use {@link ReverseLookupRule} to reduce the LOOKUP function, and whether we should set the + * "optimize" flag on {@link RegisteredLookupExtractionFn}. + */ + public boolean isReverseLookup() + { + return reverseLookup; + } + public List getParameters() { return parameters; @@ -553,4 +617,24 @@ public class PlannerContext this.explainAttributes = explainAttributes; } + /** + * Retrieve a named {@link LookupExtractor}. + */ + public LookupExtractor getLookup(final String lookupName) + { + if (lookupCache == null) { + final SqlOperatorConversion lookupOperatorConversion = + plannerToolbox.operatorTable().lookupOperatorConversion(QueryLookupOperatorConversion.SQL_FUNCTION); + + if (lookupOperatorConversion != null) { + final LookupExtractorFactoryContainerProvider lookupProvider = + ((QueryLookupOperatorConversion) lookupOperatorConversion).getLookupExtractorFactoryContainerProvider(); + lookupCache = new PlannerLookupCache(lookupProvider); + } else { + lookupCache = new PlannerLookupCache(null); + } + } + + return lookupCache.getLookup(lookupName); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerLookupCache.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerLookupCache.java new file mode 100644 index 00000000000..df583ba12e1 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerLookupCache.java @@ -0,0 +1,62 @@ +/* + * 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.planner; + +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.query.lookup.LookupExtractorFactoryContainer; +import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * Caches {@link LookupExtractor} within a {@link PlannerContext}. + */ +public class PlannerLookupCache +{ + @Nullable + private final LookupExtractorFactoryContainerProvider lookupProvider; + private final Map map; + + PlannerLookupCache(@Nullable final LookupExtractorFactoryContainerProvider lookupProvider) + { + this.lookupProvider = lookupProvider; + this.map = new HashMap<>(); + } + + @Nullable + LookupExtractor getLookup(final String lookupName) + { + if (lookupProvider == null) { + return null; + } + + return map.computeIfAbsent( + lookupName, + name -> { + final Optional maybeContainer = lookupProvider.get(name); + return maybeContainer.map(container -> container.getLookupExtractorFactory().get()) + .orElse(null); + } + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/AggregatePullUpLookupRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/AggregatePullUpLookupRule.java new file mode 100644 index 00000000000..38bab545a37 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/AggregatePullUpLookupRule.java @@ -0,0 +1,125 @@ +/* + * 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 org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.sql.calcite.expression.builtin.QueryLookupOperatorConversion; +import org.apache.druid.sql.calcite.planner.PlannerContext; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +/** + * Rule that pulls {@link QueryLookupOperatorConversion#SQL_FUNCTION} up through an {@link Aggregate}. + */ +public class AggregatePullUpLookupRule extends RelOptRule +{ + private final PlannerContext plannerContext; + + public AggregatePullUpLookupRule(final PlannerContext plannerContext) + { + super(operand(Aggregate.class, operand(Project.class, any()))); + this.plannerContext = plannerContext; + } + + @Override + public void onMatch(RelOptRuleCall call) + { + final Aggregate aggregate = call.rel(0); + final Project project = call.rel(1); + final RexBuilder rexBuilder = aggregate.getCluster().getRexBuilder(); + final Set aggCallInputs = RelOptUtil.getAllFields2(ImmutableBitSet.of(), aggregate.getAggCallList()); + + RexNode[] topProjects = null; // Projects pulled up on top of the Aggregate + RexNode[] bottomProjects = null; // Projects that stay on the bottom of the Aggregate + boolean matched = false; // Whether we found a LOOKUP call to pull up + + int dimensionIndex = 0; + for (Iterator iterator = aggregate.getGroupSet().iterator(); iterator.hasNext(); dimensionIndex++) { + int projectIndex = iterator.next(); + final RexNode projectExpr = project.getProjects().get(projectIndex); + + if (ReverseLookupRule.isLookupCall(projectExpr) && !aggCallInputs.contains(projectIndex)) { + final RexCall lookupCall = (RexCall) projectExpr; + final String lookupName = RexLiteral.stringValue(lookupCall.getOperands().get(1)); + final LookupExtractor lookup = plannerContext.getLookup(lookupName); + + if (lookup != null && lookup.isOneToOne()) { + if (!matched) { + matched = true; + bottomProjects = new RexNode[project.getProjects().size()]; + topProjects = new RexNode[aggregate.getRowType().getFieldCount()]; + } + + // Rewrite LOOKUP("x", 'lookupName) => "x" underneath the Aggregate. + bottomProjects[projectIndex] = lookupCall.getOperands().get(0); + + // Pull LOOKUP above the Aggregate. + final List newLookupOperands = new ArrayList<>(lookupCall.getOperands()); + final RelDataType dimensionType = aggregate.getRowType().getFieldList().get(dimensionIndex).getType(); + newLookupOperands.set(0, rexBuilder.makeInputRef(dimensionType, dimensionIndex)); + topProjects[dimensionIndex] = lookupCall.clone(dimensionType, newLookupOperands); + } + } + } + + if (matched) { + // Fill in any missing bottomProjects. + for (int i = 0; i < bottomProjects.length; i++) { + if (bottomProjects[i] == null) { + bottomProjects[i] = project.getProjects().get(i); + } + } + + // Fill in any missing topProjects. + for (int i = 0; i < topProjects.length; i++) { + if (topProjects[i] == null) { + topProjects[i] = rexBuilder.makeInputRef(aggregate.getRowType().getFieldList().get(i).getType(), i); + } + } + + final RelBuilder relBuilder = call.builder(); + call.transformTo(relBuilder + .push(project.getInput()) + .project(bottomProjects) + .aggregate( + relBuilder.groupKey(aggregate.getGroupSet(), aggregate.getGroupSets()), + aggregate.getAggCallList() + ) + .project(topProjects) + .build()); + call.getPlanner().prune(aggregate); + } + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/FilterDecomposeConcatRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/FilterDecomposeConcatRule.java index 1a28392a5b4..30b9b8eb962 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/FilterDecomposeConcatRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/FilterDecomposeConcatRule.java @@ -19,8 +19,8 @@ package org.apache.druid.sql.calcite.rule; -import com.google.common.collect.HashMultiset; import com.google.common.collect.Iterables; +import com.google.common.collect.LinkedHashMultiset; import com.google.common.collect.Multiset; import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptRuleCall; @@ -162,7 +162,10 @@ public class FilterDecomposeConcatRule extends RelOptRule implements Substitutio // be unambiguously reversed.) final StringBuilder regexBuilder = new StringBuilder(); final List nonLiterals = new ArrayList<>(); - final Multiset literalCounter = HashMultiset.create(); + + // Order is important in literalCounter, since we look for later literals only after the first occurrences of + // earlier literals. So, use LinkedHashMultiset to preserve order. + final Multiset literalCounter = LinkedHashMultiset.create(); boolean expectLiteral = false; // If true, next operand must be a literal. for (int i = 0; i < concatCall.getOperands().size(); i++) { final RexNode operand = concatCall.getOperands().get(i); @@ -187,13 +190,19 @@ public class FilterDecomposeConcatRule extends RelOptRule implements Substitutio } // Verify, using literalCounter, that each literal appears in the matchValue the correct number of times. + int checkPos = 0; for (Multiset.Entry entry : literalCounter.entrySet()) { - final int occurrences = countOccurrences(matchValue, entry.getElement()); + final int occurrences = countOccurrences(matchValue.substring(checkPos), entry.getElement()); if (occurrences > entry.getCount()) { // If occurrences > entry.getCount(), the match is ambiguous; consider concat(x, 'x', y) = '2x3x4' return null; } else if (occurrences < entry.getCount()) { return impossibleMatch(nonLiterals, rexBuilder); + } else { + // Literal N + 1 can be ignored if it appears before literal N, because it can't possibly match. Consider the + // case where [CONCAT(a, ' (', b, 'x', ')') = 'xxx (2x4)']. This is unambiguous, because 'x' only appears once + // after the first ' ('. + checkPos = matchValue.indexOf(entry.getElement()) + 1; } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/ReverseLookupRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/ReverseLookupRule.java new file mode 100644 index 00000000000..3deacaee355 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/ReverseLookupRule.java @@ -0,0 +1,678 @@ +/* + * 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.collect.Iterables; +import com.google.common.collect.Lists; +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.calcite.rel.rules.SubstitutionRule; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.rex.RexUtil; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.query.QueryContext; +import org.apache.druid.query.extraction.MapLookupExtractor; +import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.lookup.LookupExtractionFn; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.sql.calcite.expression.builtin.MultiValueStringOperatorConversions; +import org.apache.druid.sql.calcite.expression.builtin.QueryLookupOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.SearchOperatorConversion; +import org.apache.druid.sql.calcite.filtration.CollectComparisons; +import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.utils.CollectionUtils; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +/** + * Eliminates calls to {@link QueryLookupOperatorConversion#SQL_FUNCTION} by doing reverse-lookups. + * + * Considers reversing {@link SqlStdOperatorTable#IS_NULL} and operators that match + * {@link #isBinaryComparison(RexNode)}. However, reversal is not done in all cases, such as when it would require + * embedding the entire lookup into the query, or when the number of keys exceeds + * {@link QueryContext#getInSubQueryThreshold()}. + * + * The heart of the class, where the reversal actually happens, is + * {@link ReverseLookupShuttle.CollectReverseLookups#reverseLookup}. The rest of the rule is mainly about grouping + * together as many LOOKUP calls as possible prior to attempting to reverse, since reversing a lookup may require + * iteration of the lookup. We don't want to do that more often than necessary. + */ +public class ReverseLookupRule extends RelOptRule implements SubstitutionRule +{ + /** + * Context parameter for tests, to allow us to confirm that this rule doesn't do too many calls to + * {@link InDimFilter#optimize}. This is important because certain lookup implementations, most + * prominently {@link MapLookupExtractor}, do a full iteration of the map for each call to + * {@link LookupExtractor#unapplyAll}, which may be called by {@link InDimFilter#optimize}. + */ + public static final String CTX_MAX_OPTIMIZE_COUNT = "maxOptimizeCountForDruidReverseLookupRule"; + + /** + * Context parameter for tests, to allow us to force the case where we avoid creating a bunch of ORs. + */ + public static final String CTX_MAX_IN_SIZE = "maxInSizeForDruidReverseLookupRule"; + + private final PlannerContext plannerContext; + + public ReverseLookupRule(final PlannerContext plannerContext) + { + super(operand(LogicalFilter.class, any())); + this.plannerContext = plannerContext; + } + + @Override + public void onMatch(RelOptRuleCall call) + { + final Filter filter = call.rel(0); + + final int maxOptimizeCount = plannerContext.queryContext().getInt(CTX_MAX_OPTIMIZE_COUNT, Integer.MAX_VALUE); + final int maxInSize = + plannerContext.queryContext().getInt(CTX_MAX_IN_SIZE, plannerContext.queryContext().getInSubQueryThreshold()); + final ReverseLookupShuttle reverseLookupShuttle = new ReverseLookupShuttle( + plannerContext, + filter.getCluster().getRexBuilder(), + maxOptimizeCount, + maxInSize + ); + final RexNode newCondition = filter.getCondition().accept(reverseLookupShuttle); + + //noinspection ObjectEquality + if (newCondition != filter.getCondition()) { + call.transformTo(call.builder() + .push(filter.getInput()) + .filter(newCondition).build()); + call.getPlanner().prune(filter); + } + } + + static class ReverseLookupShuttle extends RexShuttle + { + private final PlannerContext plannerContext; + private final RexBuilder rexBuilder; + + /** + * Maximum number of calls to {@link InDimFilter#optimizeLookup(InDimFilter, boolean, int)}. If this limit + * is exceeded, we throw an error. Used in tests. + */ + private final int maxOptimizeCount; + + /** + * Maximum number of reversed keys to consider passing to {@link SearchOperatorConversion#makeIn}. If this limit + * is exceeded, we don't rewrite the LOOKUP call. This mainly comes up when lots of keys map to the same value. + */ + private final int maxInSize; + + /** + * Tracks LOOKUP nodes that we have considered for reversal when they appear as children of AND or OR, so we don't + * revisit them as we continue down the tree if reverse-lookup turned out to be impossible. + */ + private final Set consideredAsChild = new HashSet<>(); + + /** + * Flipped by each call to {@link #visitNot(RexCall)}. See {@link NullHandling#useThreeValueLogic()}. + */ + private boolean includeUnknown = false; + + /** + * Tracker towards the limit {@link #maxOptimizeCount}. + */ + private int optimizeCount = 0; + + public ReverseLookupShuttle( + final PlannerContext plannerContext, + final RexBuilder rexBuilder, + final int maxOptimizeCount, + final int maxInSize + ) + { + this.plannerContext = plannerContext; + this.rexBuilder = rexBuilder; + this.maxOptimizeCount = maxOptimizeCount; + this.maxInSize = maxInSize; + } + + @Override + public RexNode visitCall(RexCall call) + { + if (call.getKind() == SqlKind.NOT) { + return visitNot(call); + } else if (call.getKind() == SqlKind.AND) { + return visitAnd(call); + } else if (call.getKind() == SqlKind.OR) { + return visitOr(call); + } else if (call.isA(SqlKind.SEARCH)) { + return visitSearch(call); + } else if ((call.isA(SqlKind.IS_NULL) || isBinaryComparison(call)) && !consideredAsChild.contains(call)) { + return visitComparison(call); + } else { + return super.visitCall(call); + } + } + + /** + * When we encounter NOT, flip {@link #includeUnknown} to ensure filters are optimized correctly. + */ + private RexNode visitNot(final RexCall call) + { + includeUnknown = NullHandling.useThreeValueLogic() && !includeUnknown; + final RexNode retVal = super.visitCall(call); + includeUnknown = NullHandling.useThreeValueLogic() && !includeUnknown; + return retVal; + } + + /** + * When we encounter OR, collect and reverse all LOOKUP calls that appear as children. + */ + private RexNode visitOr(final RexCall call) + { + consideredAsChild.addAll(call.getOperands()); + + final List newOperands = + new CollectReverseLookups(call.getOperands(), rexBuilder).collect(); + + //noinspection ObjectEquality + if (newOperands != call.getOperands()) { + return RexUtil.composeDisjunction(rexBuilder, newOperands); + } else { + return super.visitCall(call); + } + } + + /** + * When we encounter AND, transform [!a && !b && c && d] to [!(a || b) && c && c], then run + * similar logic as {@link #visitOr(RexCall)} on the [(a || b)] part. + */ + private RexNode visitAnd(final RexCall call) + { + // Transform [!a && !b && c && d] to [!(a || b) && c && c] + final List notOrs = new ArrayList<>(); + final List remainder = new ArrayList<>(); + + for (final RexNode operand : call.getOperands()) { + if (operand.isA(SqlKind.NOT)) { + // Need to add the node beneath the NOT to consideredAsChild, because that's the one that would be visited + // by a later visitCall. This is unlike NOT_EQUALS and IS_NOT_NULL, where the NOT_EQUALS and IS_NOT_NULL + // nodes are visited directly. + final RexNode nodeBeneathNot = Iterables.getOnlyElement(((RexCall) operand).getOperands()); + consideredAsChild.add(nodeBeneathNot); + notOrs.add(nodeBeneathNot); + } else if (operand.isA(SqlKind.NOT_EQUALS)) { + consideredAsChild.add(operand); + notOrs.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, ((RexCall) operand).getOperands())); + } else if (operand.isA(SqlKind.IS_NOT_NULL)) { + consideredAsChild.add(operand); + notOrs.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, ((RexCall) operand).getOperands())); + } else { + remainder.add(operand); + } + } + + if (!notOrs.isEmpty()) { + includeUnknown = !includeUnknown; + final List newNotOrs = + new CollectReverseLookups(notOrs, rexBuilder).collect(); + includeUnknown = !includeUnknown; + + //noinspection ObjectEquality + if (newNotOrs != notOrs) { + RexNode retVal = + rexBuilder.makeCall(SqlStdOperatorTable.NOT, RexUtil.composeDisjunction(rexBuilder, newNotOrs)); + + if (!remainder.isEmpty()) { + remainder.add(retVal); + retVal = rexBuilder.makeCall(SqlStdOperatorTable.AND, remainder); + } + + return retVal; + } + } + + return super.visitCall(call); + } + + /** + * When we encounter SEARCH, expand it using {@link SearchOperatorConversion#expandSearch(RexCall, RexBuilder)} + * and continue processing what lies beneath. + */ + private RexNode visitSearch(final RexCall call) + { + final RexNode expanded = SearchOperatorConversion.expandSearch(call, rexBuilder); + + if (expanded instanceof RexCall) { + final RexNode converted = visitCall((RexCall) expanded); + //noinspection ObjectEquality + if (converted != expanded) { + return converted; + } + } + + // If no change, return the original SEARCH call, not the expanded one. + return call; + } + + /** + * When we encounter a comparison, check if it is of the form {@code LOOKUP(...) = }, and if so, attempt + * to do a reverse lookup to eliminate the {@code LOOKUP} call. + */ + private RexNode visitComparison(final RexCall call) + { + return CollectionUtils.getOnlyElement( + new CollectReverseLookups(Collections.singletonList(call), rexBuilder).collect(), + ret -> new ISE("Expected to collect single node, got[%s]", ret) + ); + } + + /** + * Collect and reverse a set of lookups that appear as children to OR. + */ + private class CollectReverseLookups + extends CollectComparisons + { + private final RexBuilder rexBuilder; + + private CollectReverseLookups( + final List orExprs, + final RexBuilder rexBuilder + ) + { + super(orExprs); + this.rexBuilder = rexBuilder; + } + + @Nullable + @Override + protected Pair> getCollectibleComparison(RexNode expr) + { + final RexCall asLookupComparison = getAsLookupComparison(expr); + if (asLookupComparison != null) { + return Pair.of(asLookupComparison, Collections.emptyList()); + } else { + return null; + } + } + + @Nullable + @Override + protected ReverseLookupKey getCollectionKey(RexCall call) + { + final RexCall lookupCall = (RexCall) call.getOperands().get(0); + final List lookupOperands = lookupCall.getOperands(); + final RexNode argument = lookupOperands.get(0); + final String lookupName = RexLiteral.stringValue(lookupOperands.get(1)); + final String replaceMissingValueWith; + + if (lookupOperands.size() >= 3) { + replaceMissingValueWith = NullHandling.emptyToNullIfNeeded(RexLiteral.stringValue(lookupOperands.get(2))); + } else { + replaceMissingValueWith = null; + } + + final LookupExtractor lookup = plannerContext.getLookup(lookupName); + + if (lookup == null) { + return null; + } + + if (!lookup.isOneToOne()) { + // For non-injective lookups, we can't reverse comparisons to the "replaceMissingValueWith" value. + // If we detect these, ignore them by returning a null collection key. This isn't strictly required for + // correctness, because InDimFilter#optimize won't optimize this case, and so our "reverseLookup" won't do + // anything. But it's helpful to encourage grouping together filters that *can* be reversed when they + // are peers to an irreversible one, and all are children of OR and AND. + final boolean isComparisonAgainstReplaceMissingValueWith; + + if (replaceMissingValueWith == null) { + // Optimization: avoid calling getMatchValues(call) when it's IS NULL. + // Equivalent to the "else" case logically, but involves less work. + isComparisonAgainstReplaceMissingValueWith = call.isA(SqlKind.IS_NULL); + } else { + isComparisonAgainstReplaceMissingValueWith = getMatchValues(call).contains(replaceMissingValueWith); + } + + if (isComparisonAgainstReplaceMissingValueWith) { + return null; + } + } + + final boolean multiValue = + call.getOperator().equals(MultiValueStringOperatorConversions.CONTAINS.calciteOperator()) + || call.getOperator().equals(MultiValueStringOperatorConversions.OVERLAP.calciteOperator()); + final boolean negate = call.getKind() == SqlKind.NOT_EQUALS; + return new ReverseLookupKey(argument, lookupName, replaceMissingValueWith, multiValue, negate); + } + + @Override + protected Set getMatchValues(RexCall call) + { + if (call.isA(SqlKind.IS_NULL)) { + return Collections.singleton(null); + } else { + // Compute the set of values that this comparison operator matches. + // Note that MV_CONTAINS and MV_OVERLAP match nulls, but other comparison operators do not. + // See "isBinaryComparison" for the set of operators we might encounter here. + final RexNode matchLiteral = call.getOperands().get(1); + final boolean matchNulls = + call.getOperator().equals(MultiValueStringOperatorConversions.CONTAINS.calciteOperator()) + || call.getOperator().equals(MultiValueStringOperatorConversions.OVERLAP.calciteOperator()); + return toStringSet(matchLiteral, matchNulls); + } + } + + @Nullable + @Override + protected RexNode makeCollectedComparison(ReverseLookupKey reverseLookupKey, InDimFilter.ValuesSet matchValues) + { + final LookupExtractor lookupExtractor = plannerContext.getLookup(reverseLookupKey.lookupName); + + if (lookupExtractor != null) { + final Set reversedMatchValues = reverseLookup( + lookupExtractor, + reverseLookupKey.replaceMissingValueWith, + matchValues, + includeUnknown ^ reverseLookupKey.negate + ); + + if (reversedMatchValues != null) { + return makeMatchCondition(reverseLookupKey, reversedMatchValues, rexBuilder); + } + } + + return null; + } + + @Override + protected RexNode makeAnd(List exprs) + { + throw new UnsupportedOperationException(); + } + + + /** + * Return expr as a lookup comparison, where the lookup operator is the left-hand side, a literal is + * on the right-hand side, and the comparison is either {@link SqlStdOperatorTable#IS_NULL} or one of + * the operators recognized by {@link #isBinaryComparison(RexNode)}. + * + * Returns null if expr does not actually match the above pattern. + */ + @Nullable + private RexCall getAsLookupComparison(final RexNode expr) + { + if (expr.isA(SqlKind.IS_NULL) && isLookupCall(((RexCall) expr).getOperands().get(0))) { + return (RexCall) expr; + } + + if (!isBinaryComparison(expr)) { + return null; + } + + final RexCall call = (RexCall) expr; + + RexNode lookupCall = call.getOperands().get(0); + RexNode literal = call.getOperands().get(1); + + // Possibly swap arguments. + if (literal instanceof RexCall && Calcites.isLiteral(lookupCall, true, true)) { + // MV_CONTAINS doesn't allow swapping of arguments, since they aren't commutative. + // See "isBinaryComparison" for the set of operators we might encounter here. + if (call.getOperator().equals(MultiValueStringOperatorConversions.CONTAINS.calciteOperator())) { + return null; + } + + // Swap lookupCall, literal. + RexNode tmp = lookupCall; + lookupCall = literal; + literal = tmp; + } + + lookupCall = RexUtil.removeNullabilityCast(rexBuilder.getTypeFactory(), lookupCall); + literal = RexUtil.removeNullabilityCast(rexBuilder.getTypeFactory(), literal); + + // Check that the call is of the form: LOOKUP(...) + if (isLookupCall(lookupCall) && Calcites.isLiteral(literal, true, true)) { + return (RexCall) rexBuilder.makeCall(call.getOperator(), lookupCall, literal); + } else { + return null; + } + } + + /** + * Perform a reverse lookup, leveraging logic from {@link InDimFilter#optimize(boolean)}. + * + * @param lookupExtractor lookup object + * @param replaceMissingValueWith third argument to LOOKUP function-- missing values are replaced with this. + * By default, this is null. + * @param matchValues values to reverse-lookup + * @param mayIncludeUnknown whether the reverse-lookup should be done in a context where unknown matches + * may be considered 'true' rather than 'false'. + * + * @return reverse lookup of "matchValues", or null if a reverse lookup can't be performed + */ + @Nullable + private Set reverseLookup( + final LookupExtractor lookupExtractor, + @Nullable final String replaceMissingValueWith, + final InDimFilter.ValuesSet matchValues, + final boolean mayIncludeUnknown + ) + { + optimizeCount++; + if (optimizeCount > maxOptimizeCount) { + throw new ISE("Too many optimize calls[%s]", optimizeCount); + } + + final InDimFilter filterToOptimize = new InDimFilter( + "__dummy__", + matchValues, + new LookupExtractionFn(lookupExtractor, false, replaceMissingValueWith, null, true) + ); + + return InDimFilter.optimizeLookup( + filterToOptimize, + mayIncludeUnknown && NullHandling.useThreeValueLogic(), + maxInSize + ); + } + + private RexNode makeMatchCondition( + final ReverseLookupKey reverseLookupKey, + final Set reversedMatchValues, + final RexBuilder rexBuilder + ) + { + if (reversedMatchValues.isEmpty()) { + return rexBuilder.makeLiteral(reverseLookupKey.negate); + } else if (reverseLookupKey.multiValue) { + // Use MV_CONTAINS or MV_OVERLAP. + RexNode condition; + if (reversedMatchValues.size() == 1) { + condition = rexBuilder.makeCall( + MultiValueStringOperatorConversions.CONTAINS.calciteOperator(), + reverseLookupKey.arg, + Iterables.getOnlyElement(stringsToRexNodes(reversedMatchValues, rexBuilder)) + ); + } else { + condition = rexBuilder.makeCall( + MultiValueStringOperatorConversions.OVERLAP.calciteOperator(), + reverseLookupKey.arg, + rexBuilder.makeCall( + SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, + stringsToRexNodes(reversedMatchValues, rexBuilder) + ) + ); + } + + if (reverseLookupKey.negate) { + condition = rexBuilder.makeCall(SqlStdOperatorTable.NOT, condition); + } + + return condition; + } else { + return SearchOperatorConversion.makeIn( + reverseLookupKey.arg, + stringsToRexNodes(reversedMatchValues, rexBuilder), + reverseLookupKey.negate, + rexBuilder + ); + } + } + } + } + + /** + * Return a list of {@link RexNode} corresponding to the provided strings. + */ + private static List stringsToRexNodes(final Iterable strings, final RexBuilder rexBuilder) + { + return Lists.newArrayList( + Iterables.transform( + strings, + s -> { + if (s == null) { + return rexBuilder.makeNullLiteral(rexBuilder.getTypeFactory().createSqlType(SqlTypeName.VARCHAR)); + } else { + return rexBuilder.makeLiteral(s); + } + } + ) + ); + } + + /** + * Whether a call is a binary (two-argument) comparison of the kind that + * {@link ReverseLookupShuttle#visitComparison(RexCall)} can possibly handle. + */ + private static boolean isBinaryComparison(final RexNode rexNode) + { + if (rexNode instanceof RexCall) { + final RexCall call = (RexCall) rexNode; + return call.getKind() == SqlKind.EQUALS + || call.getKind() == SqlKind.NOT_EQUALS + || call.getOperator().equals(MultiValueStringOperatorConversions.CONTAINS.calciteOperator()) + || call.getOperator().equals(MultiValueStringOperatorConversions.OVERLAP.calciteOperator()); + } else { + return false; + } + } + + /** + * Whether a given expression is a LOOKUP call. + */ + static boolean isLookupCall(final RexNode expr) + { + return expr.isA(SqlKind.OTHER_FUNCTION) + && ((RexCall) expr).getOperator().equals(QueryLookupOperatorConversion.SQL_FUNCTION); + } + + /** + * Convert a literal that we are doing an equality (or MV contains, overlaps) operation with into a set of values + * to match. Returns null if the literal is not a string, string array, or null type. + */ + @Nullable + private static Set toStringSet(final RexNode literal, final boolean matchNulls) + { + if (RexUtil.isNullLiteral(literal, true)) { + return matchNulls ? Collections.singleton(null) : Collections.emptySet(); + } else if (SqlTypeFamily.STRING.contains(literal.getType())) { + final String s = RexLiteral.stringValue(literal); + return s != null || matchNulls ? Collections.singleton(s) : Collections.emptySet(); + } else if (literal.getType().getSqlTypeName() == SqlTypeName.ARRAY + && SqlTypeFamily.STRING.contains(literal.getType().getComponentType())) { + final Set elements = new HashSet<>(); + for (final RexNode element : ((RexCall) literal).getOperands()) { + final String s = RexLiteral.stringValue(element); + if (s != null || matchNulls) { + elements.add(s); + } + } + return elements; + } else { + return null; + } + } + + /** + * Collection key for {@link ReverseLookupShuttle.CollectReverseLookups}. + */ + private static class ReverseLookupKey + { + private final RexNode arg; + private final String lookupName; + private final String replaceMissingValueWith; + private final boolean multiValue; + private final boolean negate; + + private ReverseLookupKey( + final RexNode arg, + final String lookupName, + final String replaceMissingValueWith, + final boolean multiValue, + final boolean negate + ) + { + this.arg = arg; + this.lookupName = lookupName; + this.replaceMissingValueWith = replaceMissingValueWith; + this.multiValue = multiValue; + this.negate = negate; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReverseLookupKey that = (ReverseLookupKey) o; + return multiValue == that.multiValue + && negate == that.negate + && Objects.equals(arg, that.arg) + && Objects.equals(lookupName, that.lookupName) + && Objects.equals(replaceMissingValueWith, that.replaceMissingValueWith); + } + + @Override + public int hashCode() + { + return Objects.hash(arg, lookupName, replaceMissingValueWith, multiValue, negate); + } + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java new file mode 100644 index 00000000000..fe8b454b4c0 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java @@ -0,0 +1,2083 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.Druids; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.FilteredAggregatorFactory; +import org.apache.druid.query.aggregation.GroupingAggregatorFactory; +import org.apache.druid.query.aggregation.LongMinAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.ExtractionDimensionSpec; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.lookup.RegisteredLookupExtractionFn; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.planner.PlannerConfig; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rule.ReverseLookupRule; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Assert; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class CalciteLookupFunctionQueryTest extends BaseCalciteQueryTest +{ + private static final Map QUERY_CONTEXT = + ImmutableMap.builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(PlannerContext.CTX_SQL_REVERSE_LOOKUP, true) + .put(ReverseLookupRule.CTX_MAX_OPTIMIZE_COUNT, 1) + .build(); + + private static final ExtractionFn EXTRACTION_FN = + new RegisteredLookupExtractionFn(null, "lookyloo", false, null, null, false); + + private static final ExtractionFn EXTRACTION_FN_121 = + new RegisteredLookupExtractionFn(null, "lookyloo121", false, null, null, false); + + @Test + public void testFilterEquals() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') = 'xabc'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension("'xabc'", equality("dim1", "abc", ColumnType.STRING)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterLookupOfFunction() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(LOWER(dim1), 'lookyloo') = 'xabc'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lower(\"dim1\")", ColumnType.STRING), + equality("v0", "abc", ColumnType.STRING) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterFunctionOfLookup() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOWER(LOOKUP(dim1, 'lookyloo')) = 'xabc'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lower(lookup(\"dim1\",'lookyloo'))", ColumnType.STRING), + equality("v0", "xabc", ColumnType.STRING) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterLookupOfConcat() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(CONCAT(dim1, 'b', dim2), 'lookyloo') = 'xabc'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension( + "'xa'", // dim1 must be 'a', and lookup of 'a' is 'xa' + and( + equality("dim1", "a", ColumnType.STRING), + equality("dim2", "c", ColumnType.STRING) + ) + ), + ImmutableList.of() + ); + } + + @Test + public void testFilterInLookupOfConcat() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(CONCAT(dim1, 'a', dim2), 'lookyloo') IN ('xa', 'xabc')"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + or( + and( + equality("dim1", "", ColumnType.STRING), + equality("dim2", "", ColumnType.STRING) + ), + and( + equality("dim1", "", ColumnType.STRING), + equality("dim2", "bc", ColumnType.STRING) + ) + ) + ), + ImmutableList.of() + ); + } + + @Test + public void testFilterConcatOfLookup() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("CONCAT(LOOKUP(dim1, 'lookyloo'), ' (', dim1, ')') = 'xabc (abc)'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension( + "'xabc'", + equality("dim1", "abc", ColumnType.STRING) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterInConcatOfLookup() + { + cannotVectorize(); + + // One optimize call is needed for each "IN" value, because this expression is decomposed into a sequence of + // [(LOOKUP(dim1, 'lookyloo') = 'xabc' AND dim1 = 'abc') OR ...]. They can't be collected and combined. + + final ImmutableMap queryContext = + ImmutableMap.builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(PlannerContext.CTX_SQL_REVERSE_LOOKUP, true) + .put(ReverseLookupRule.CTX_MAX_OPTIMIZE_COUNT, 2) + .build(); + + testQuery( + buildFilterTestSql("CONCAT(LOOKUP(dim1, 'lookyloo'), ' (', dim1, ')') IN ('xa (a)', 'xabc (abc)')"), + queryContext, + buildFilterTestExpectedQuery(in("dim1", ImmutableList.of("a", "abc"), null)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterConcatOfLookupOfConcat() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "CONCAT(LOOKUP(CONCAT(dim1, 'b', dim2), 'lookyloo'), ' (', CONCAT(dim1, 'b', dim2), ')') = 'xabc (abc)'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension( + "'xa'", // dim1 must be 'a', and lookup of 'a' is 'xa' + and( + equality("dim1", "a", ColumnType.STRING), + equality("dim2", "c", ColumnType.STRING) + ) + ), + ImmutableList.of() + ); + } + + @Test + public void testFilterInConcatOfLookupOfConcat() + { + cannotVectorize(); + + // One optimize call is needed for each "IN" value, because this expression is decomposed into a sequence of + // [(LOOKUP(dim1, 'lookyloo') = 'xabc' AND dim1 = 'abc') OR ...]. They can't be collected and combined. + + final ImmutableMap queryContext = + ImmutableMap.builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(PlannerContext.CTX_SQL_REVERSE_LOOKUP, true) + .put(ReverseLookupRule.CTX_MAX_OPTIMIZE_COUNT, 2) + .build(); + + testQuery( + buildFilterTestSql( + "CONCAT(LOOKUP(CONCAT(dim1, 'a', dim2), 'lookyloo'), ' (', CONCAT(dim1, 'a', dim2), ')')\n" + + "IN ('xa (a)', 'xabc (abc)')"), + queryContext, + buildFilterTestExpectedQuery( + or( + and( + equality("dim1", "", ColumnType.STRING), + equality("dim2", "", ColumnType.STRING) + ), + and( + equality("dim1", "", ColumnType.STRING), + equality("dim2", "bc", ColumnType.STRING) + ) + ) + ), + ImmutableList.of() + ); + } + + @Test + public void testFilterConcatOfCoalesceLookupOfConcat() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "CONCAT(COALESCE(LOOKUP(CONCAT(dim1, 'b', dim2), 'lookyloo'), 'N/A'), ' (', CONCAT(dim1, 'b', dim2), ')') = 'xabc (abc)'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension( + "'xa'", // dim1 must be 'a', and lookup of 'a' is 'xa' + and( + equality("dim1", "a", ColumnType.STRING), + equality("dim2", "c", ColumnType.STRING) + ) + ), + ImmutableList.of() + ); + } + + @Test + public void testFilterImpossibleLookupOfConcat() + { + cannotVectorize(); + + // No keys in the lookup table begin with 'key:', so this is always false. + testQuery( + buildFilterTestSql("LOOKUP('key:' || dim1, 'lookyloo') = 'xabc'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryAlwaysFalse(), + ImmutableList.of() + ); + } + + @Test + public void testFilterChainedEquals() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(LOOKUP(dim1, 'lookyloo'), 'lookyloo-chain') = 'zabc'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension("'xabc'", equality("dim1", "abc", ColumnType.STRING)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterEqualsLiteralFirst() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("'xabc' = LOOKUP(dim1, 'lookyloo')"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension("'xabc'", equality("dim1", "abc", ColumnType.STRING)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterEqualsAlwaysFalse() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') = 'nonexistent'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryAlwaysFalse(), + ImmutableList.of() + ); + } + + @Test + public void testFilterIsNotDistinctFrom() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IS NOT DISTINCT FROM 'xabc'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension("'xabc'", equality("dim1", "abc", ColumnType.STRING)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterMultipleIsNotDistinctFrom() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IS NOT DISTINCT FROM 'xabc' OR " + + "LOOKUP(dim1, 'lookyloo') IS NOT DISTINCT FROM 'x6' OR " + + "LOOKUP(dim1, 'lookyloo') IS NOT DISTINCT FROM 'nonexistent'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(in("dim1", Arrays.asList("6", "abc"), null)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterIn() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IN ('xabc', 'x6', 'nonexistent')"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(in("dim1", Arrays.asList("6", "abc"), null)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterInOverMaxSize() + { + cannotVectorize(); + + // Set CTX_MAX_IN_SIZE = 1 to stop the LOOKUP call from being reversed. + final ImmutableMap queryContext = + ImmutableMap.builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(PlannerContext.CTX_SQL_REVERSE_LOOKUP, true) + .put(ReverseLookupRule.CTX_MAX_IN_SIZE, 1) + .build(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IN ('xabc', 'x6', 'nonexistent')"), + queryContext, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + in("v0", ImmutableList.of("nonexistent", "x6", "xabc"), null) + ) + : buildFilterTestExpectedQuery( + in("dim1", ImmutableList.of("nonexistent", "x6", "xabc"), EXTRACTION_FN) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterInOrIsNull() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "LOOKUP(dim1, 'lookyloo') IN ('xabc', 'x6', 'nonexistent') OR LOOKUP(dim1, 'lookyloo') IS NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + or( + in("dim1", Arrays.asList("6", "abc"), null), + isNull("v0") + ) + ) + : buildFilterTestExpectedQuery( + or( + in("dim1", Arrays.asList("6", "abc"), null), + selector("dim1", null, EXTRACTION_FN) + ) + ), + ImmutableList.of(new Object[]{NullHandling.defaultStringValue(), 5L}, new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterInAndIsNotNull() + { + cannotVectorize(); + + // Ideally we'd be able to eliminate "AND LOOKUP(dim1, 'lookyloo') IS NOT NULL", because it's implied by + // "LOOKUP(dim1, 'lookyloo') IN ('xabc', 'x6', 'nonexistent')". We're not currently able to do that. + + testQuery( + buildFilterTestSql( + "LOOKUP(dim1, 'lookyloo') IN ('xabc', 'x6', 'nonexistent') AND LOOKUP(dim1, 'lookyloo') IS NOT NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + and( + in("dim1", ImmutableList.of("6", "abc"), null), + not(isNull("v0")) + ) + ) + : buildFilterTestExpectedQuery( + and( + in("dim1", Arrays.asList("6", "abc"), null), + not(selector("dim1", null, EXTRACTION_FN)) + ) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterInOrIsNullInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "LOOKUP(dim1, 'lookyloo121') IN ('xabc', 'x6', 'nonexistent') OR LOOKUP(dim1, 'lookyloo121') IS NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + or(isNull("dim1"), equality("dim1", "abc", ColumnType.STRING)) + ) + : buildFilterTestExpectedQueryConstantDimension( + "'xabc'", + equality("dim1", "abc", ColumnType.STRING) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterNotInAndIsNotNull() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "LOOKUP(dim1, 'lookyloo') NOT IN ('x6', 'nonexistent') AND LOOKUP(dim1, 'lookyloo') IS NOT NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + and( + not(equality("v0", "x6", ColumnType.STRING)), + not(equality("v0", "nonexistent", ColumnType.STRING)), + notNull("v0") + ) + ) + : buildFilterTestExpectedQuery( + and( + not(selector("dim1", "6", null)), + not(selector("dim1", null, EXTRACTION_FN)) + ) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterInIsNotTrueAndIsNotNull() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "(LOOKUP(dim1, 'lookyloo') IN ('xabc', 'x6', 'nonexistent')) IS NOT TRUE " + + "AND LOOKUP(dim1, 'lookyloo') IS NOT NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + and( + not(istrue(in("dim1", ImmutableList.of("6", "abc"), null))), + notNull("v0") + ) + ) + : buildFilterTestExpectedQuery( + and( + not(in("dim1", ImmutableList.of("6", "abc"), null)), + not(selector("dim1", null, EXTRACTION_FN)) + ) + ), + ImmutableList.of() + ); + } + + @Test + public void testFilterNotInAndIsNotNullInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "LOOKUP(dim1, 'lookyloo121') NOT IN ('xabc', 'xdef', 'nonexistent') " + + "AND LOOKUP(dim1, 'lookyloo121') IS NOT NULL"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? and( + not(isNull("dim1")), + not(in("dim1", ImmutableList.of("abc", "def"), null)) + ) + : not(in("dim1", ImmutableList.of("abc", "def"), null))), + ImmutableList.of(new Object[]{NULL_STRING, 4L}) + ); + } + + @Test + public void testFilterNotInOrIsNull() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "LOOKUP(dim1, 'lookyloo') NOT IN ('x6', 'nonexistent') OR LOOKUP(dim1, 'lookyloo') IS NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + or( + and( + not(equality("v0", "x6", ColumnType.STRING)), + not(equality("v0", "nonexistent", ColumnType.STRING)) + ), + isNull("v0") + ) + ) + : buildFilterTestExpectedQuery( + or( + not(selector("dim1", "6", null)), + selector("dim1", null, EXTRACTION_FN) + ) + ), + ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterInIsNotTrueOrIsNull() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "(LOOKUP(dim1, 'lookyloo') IN ('x6', 'nonexistent')) IS NOT TRUE OR LOOKUP(dim1, 'lookyloo') IS NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + or( + not(istrue(equality("dim1", "6", ColumnType.STRING))), + isNull("v0") + ) + ) + : buildFilterTestExpectedQuery( + or( + not(selector("dim1", "6", null)), + selector("dim1", null, EXTRACTION_FN) + ) + ), + ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterNotIn() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') NOT IN ('x6', 'nonexistent')"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + and(not(equality("v0", "x6", ColumnType.STRING)), not(equality("v0", "nonexistent", ColumnType.STRING))) + ) + : buildFilterTestExpectedQuery(not(equality("dim1", "6", ColumnType.STRING))), + // sql compatible mode expression filter (correctly) leaves out null values + NullHandling.sqlCompatible() + ? ImmutableList.of(new Object[]{"xabc", 1L}) + : ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterInIsNotTrue() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IN ('x6', 'nonexistent') IS NOT TRUE"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? not(istrue(equality("dim1", "6", ColumnType.STRING))) + : not(equality("dim1", "6", ColumnType.STRING)) + ), + ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterNotInInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo121') NOT IN ('xabc', 'xdef', 'nonexistent')"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(not(in("dim1", ImmutableList.of("abc", "def"), null))), + ImmutableList.of(new Object[]{NULL_STRING, 4L}) + ); + } + + @Test + public void testFilterNotInWithReplaceMissingValue() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo', 'xyzzy') NOT IN ('xabc', 'x6', 'nonexistent')"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(not(in("dim1", ImmutableList.of("6", "abc"), null))), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterInIsNotTrueWithReplaceMissingValue() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo', 'xyzzy') IN ('xabc', 'x6', 'nonexistent') IS NOT TRUE"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? not(istrue(in("dim1", ImmutableList.of("6", "abc"), null))) + : not(in("dim1", ImmutableList.of("6", "abc"), null))), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterMvContains() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("MV_CONTAINS(LOOKUP(dim1, 'lookyloo'), 'xabc')"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(equality("dim1", "abc", ColumnType.STRING)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterMvContainsNull() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("MV_CONTAINS(LOOKUP(dim1, 'lookyloo'), NULL)"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(expressionFilter("array_contains(lookup(\"dim1\",'lookyloo'),null)")), + Collections.emptyList() + ); + } + + @Test + public void testFilterMvContainsNullInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("MV_CONTAINS(LOOKUP(dim1, 'lookyloo121'), NULL)"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery(expressionFilter("array_contains(\"dim1\",null)")) + : buildFilterTestExpectedQueryAlwaysFalse(), + ImmutableList.of() + ); + } + + @Test + public void testFilterMvOverlap() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("MV_OVERLAP(lookup(dim1, 'lookyloo'), ARRAY['xabc', 'x6', 'nonexistent'])"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(in("dim1", ImmutableSet.of("6", "abc"), null)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterMvOverlapNull() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("MV_OVERLAP(lookup(dim1, 'lookyloo'), ARRAY['xabc', 'x6', 'nonexistent', NULL])"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + in("dim1", Arrays.asList(null, "nonexistent", "x6", "xabc"), EXTRACTION_FN) + ), + ImmutableList.of( + new Object[]{NullHandling.defaultStringValue(), 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterMvOverlapNullInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("MV_OVERLAP(lookup(dim1, 'lookyloo121'), ARRAY['xabc', 'x6', 'nonexistent', NULL])"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? in("dim1", Arrays.asList(null, "abc"), null) + : equality("dim1", "abc", ColumnType.STRING) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterNotMvContains() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("NOT MV_CONTAINS(lookup(dim1, 'lookyloo'), 'xabc')"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + not(equality("v0", "xabc", ColumnType.STRING)) + ) + : buildFilterTestExpectedQuery( + not(equality("dim1", "abc", ColumnType.STRING)) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of() + : ImmutableList.of(new Object[]{"", 5L}) + ); + } + + @Test + public void testFilterMvContainsIsNotTrue() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("MV_CONTAINS(lookup(dim1, 'lookyloo'), 'xabc') IS NOT TRUE"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + not(equality("v0", "xabc", ColumnType.STRING)) + ) + : buildFilterTestExpectedQuery( + not(equality("dim1", "abc", ColumnType.STRING)) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of() + : ImmutableList.of(new Object[]{"", 5L}) + ); + } + + @Test + public void testFilterNotMvContainsInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("NOT MV_CONTAINS(LOOKUP(dim1, 'lookyloo121'), 'xabc')"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(not(equality("dim1", "abc", ColumnType.STRING))), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterNotMvOverlap() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("NOT MV_OVERLAP(lookup(dim1, 'lookyloo'), ARRAY['xabc', 'x6', 'nonexistent'])"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? not(in("dim1", ImmutableList.of("nonexistent", "x6", "xabc"), EXTRACTION_FN)) + : not(in("dim1", ImmutableList.of("6", "abc"), null)) + ), + NullHandling.sqlCompatible() + ? Collections.emptyList() + : ImmutableList.of(new Object[]{NullHandling.defaultStringValue(), 5L}) + ); + } + + @Test + public void testFilterMvOverlapIsNotTrue() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("MV_OVERLAP(lookup(dim1, 'lookyloo'), ARRAY['xabc', 'x6', 'nonexistent']) IS NOT TRUE"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? not(in("dim1", ImmutableList.of("x6", "xabc", "nonexistent"), EXTRACTION_FN)) + : not(in("dim1", ImmutableList.of("6", "abc"), null)) + ), + NullHandling.sqlCompatible() + ? Collections.emptyList() + : ImmutableList.of(new Object[]{NullHandling.defaultStringValue(), 5L}) + ); + } + + @Test + public void testFilterNotMvOverlapInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("NOT MV_OVERLAP(lookup(dim1, 'lookyloo121'), ARRAY['xabc', 'x6', 'nonexistent'])"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(not(equality("dim1", "abc", ColumnType.STRING))), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterMultipleIsDistinctFrom() + { + cannotVectorize(); + + // One optimize call is needed for each "IS DISTINCT FROM", because "x IS DISTINCT FROM y" is sugar for + // "(x = y) IS NOT TRUE", and ReverseLookupRule doesn't peek into the "IS NOT TRUE" calls nested beneatth + // the "AND". + + final ImmutableMap queryContext = + ImmutableMap.builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(PlannerContext.CTX_SQL_REVERSE_LOOKUP, true) + .put(ReverseLookupRule.CTX_MAX_OPTIMIZE_COUNT, 3) + .build(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IS DISTINCT FROM 'xabc' AND " + + "LOOKUP(dim1, 'lookyloo') IS DISTINCT FROM 'x6' AND " + + "LOOKUP(dim1, 'lookyloo') IS DISTINCT FROM 'nonexistent'"), + queryContext, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? and( + not(istrue(equality("dim1", "abc", ColumnType.STRING))), + not(istrue(equality("dim1", "6", ColumnType.STRING))) + ) + : and( + not(equality("dim1", "abc", ColumnType.STRING)), + not(equality("dim1", "6", ColumnType.STRING)) + ) + ), + ImmutableList.of( + new Object[]{NULL_STRING, 5L} + ) + ); + } + + @Test + public void testFilterIsNull() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IS NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + isNull("v0") + ) + : buildFilterTestExpectedQuery(selector("dim1", null, EXTRACTION_FN)), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterIsNullInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo121') IS NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQueryConstantDimension("null", isNull("dim1")) + : buildFilterTestExpectedQueryAlwaysFalse(), + ImmutableList.of() + ); + } + + @Test + public void testFilterIsNotNull() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IS NOT NULL"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + not(isNull("v0")) + ) + : buildFilterTestExpectedQuery( + not(selector("dim1", null, EXTRACTION_FN)) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterIsNotNullInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo121') IS NOT NULL"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? not(isNull("dim1")) + : null + ), + ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterNotEquals() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') <> 'x6'"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + not(equality("v0", "x6", ColumnType.STRING)) + ) + : buildFilterTestExpectedQuery( + not(selector("dim1", "6", null)) + ), + // sql compatible mode expression filter (correctly) leaves out null values + NullHandling.sqlCompatible() + ? ImmutableList.of(new Object[]{"xabc", 1L}) + : ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterNotEqualsInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo121') <> 'xabc'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(not(equality("dim1", "abc", ColumnType.STRING))), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterEqualsIsNotTrue() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') = 'x6' IS NOT TRUE"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? not(istrue(equality("dim1", "6", ColumnType.STRING))) + : not(selector("dim1", "6", null)) + ), + ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterEqualsIsNotTrueInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo121') = 'xabc' IS NOT TRUE"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? not(istrue(equality("dim1", "abc", ColumnType.STRING))) + : not(equality("dim1", "abc", ColumnType.STRING))), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterIsDistinctFrom() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') IS DISTINCT FROM 'x6'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? not(istrue(equality("dim1", "6", ColumnType.STRING))) + : not(equality("dim1", "6", ColumnType.STRING)) + ), + ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterIsDistinctFromReplaceMissingValueWithSameLiteral() + { + cannotVectorize(); + + final RegisteredLookupExtractionFn extractionFn = + new RegisteredLookupExtractionFn(null, "lookyloo", false, "x6", null, false); + + testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo', 'x6') IS DISTINCT FROM 'x6'"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo','x6')", ColumnType.STRING), + not(istrue(equality("v0", "x6", ColumnType.STRING))) + ) + : buildFilterTestExpectedQuery( + not(selector("dim1", "x6", extractionFn)) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterNotEquals2() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("NOT (LOOKUP(dim1, 'lookyloo') = 'x6' OR cnt = 2)"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING), + and( + not(equality("v0", "x6", ColumnType.STRING)), + not(equality("cnt", 2L, ColumnType.LONG)) + ) + ) + : buildFilterTestExpectedQuery( + and( + not(equality("dim1", "6", ColumnType.STRING)), + not(equality("cnt", 2L, ColumnType.LONG)) + ) + ), + // sql compatible mode expression filter (correctly) leaves out null values + NullHandling.sqlCompatible() + ? ImmutableList.of(new Object[]{"xabc", 1L}) + : ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterEqualsIsNotTrue2() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("(LOOKUP(dim1, 'lookyloo') = 'x6' OR cnt = 2) IS NOT TRUE"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? not(istrue(or( + equality("dim1", "6", ColumnType.STRING), + equality("cnt", 2L, ColumnType.LONG) + ))) + : not(or( + equality("dim1", "6", ColumnType.STRING), + equality("cnt", 2L, ColumnType.LONG) + ))), + ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterNotEquals2Injective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("NOT (LOOKUP(dim1, 'lookyloo121') = 'xdef' OR cnt = 2)"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(and( + not(equality("dim1", "def", ColumnType.STRING)), + not(equality("cnt", 2L, ColumnType.LONG)) + )), + ImmutableList.of( + new Object[]{NULL_STRING, 4L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterCoalesceSameLiteral() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), 'x6') = 'x6'"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo','x6')", ColumnType.STRING), + equality("v0", "x6", ColumnType.STRING) + ) + : buildFilterTestExpectedQuery( + selector("dim1", "x6", new RegisteredLookupExtractionFn(null, "lookyloo", false, "x6", null, false)) + ), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterCoalesceSameLiteralInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo121'), 'x6') = 'x6'"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQueryConstantDimension("null", isNull("dim1")) + : buildFilterTestExpectedQueryAlwaysFalse(), + ImmutableList.of() + ); + } + + @Test + public void testFilterInCoalesceSameLiteral() + { + cannotVectorize(); + + final RegisteredLookupExtractionFn extractionFn = + new RegisteredLookupExtractionFn(null, "lookyloo", false, "x6", null, false); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), 'x6') IN ('xa', 'xabc', 'x6')"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo','x6')", ColumnType.STRING), + or( + in("dim1", Arrays.asList("a", "abc"), null), + equality("v0", "x6", ColumnType.STRING) + ) + ) + : buildFilterTestExpectedQuery( + or( + in("dim1", Arrays.asList("a", "abc"), null), + selector("dim1", "x6", extractionFn) + ) + ), + ImmutableList.of(new Object[]{NullHandling.defaultStringValue(), 5L}, new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterInCoalesceSameLiteralInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo121'), 'x2') IN ('xabc', 'xdef', 'x2')"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + NullHandling.sqlCompatible() + ? or(in("dim1", Arrays.asList("2", "abc", "def"), null), isNull("dim1")) + : in("dim1", Arrays.asList("2", "abc", "def"), null) + ), + ImmutableList.of(new Object[]{NullHandling.defaultStringValue(), 2L}, new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterMvContainsCoalesceSameLiteral() + { + cannotVectorize(); + + final RegisteredLookupExtractionFn extractionFn = + new RegisteredLookupExtractionFn(null, "lookyloo", false, "x6", null, false); + + testQuery( + buildFilterTestSql("MV_CONTAINS(COALESCE(LOOKUP(dim1, 'lookyloo'), 'x6'), 'x6')"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo','x6')", ColumnType.STRING), + equality("v0", "x6", ColumnType.STRING) + ) + : buildFilterTestExpectedQuery( + selector("dim1", "x6", extractionFn) + ), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterMvOverlapCoalesceSameLiteral() + { + cannotVectorize(); + + final RegisteredLookupExtractionFn extractionFn = + new RegisteredLookupExtractionFn(null, "lookyloo", false, "x6", null, false); + + testQuery( + buildFilterTestSql("MV_OVERLAP(COALESCE(LOOKUP(dim1, 'lookyloo'), 'x6'), ARRAY['xabc', 'x6', 'nonexistent'])"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(in("dim1", ImmutableList.of("xabc", "x6", "nonexistent"), extractionFn)), + ImmutableList.of(new Object[]{NULL_STRING, 5L}, new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterCoalesceSameLiteralNotEquals() + { + cannotVectorize(); + + final RegisteredLookupExtractionFn extractionFn = + new RegisteredLookupExtractionFn(null, "lookyloo", false, "x6", null, false); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), 'x6') <> 'x6'"), + QUERY_CONTEXT, + NullHandling.sqlCompatible() + ? buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo','x6')", ColumnType.STRING), + not(equality("v0", "x6", ColumnType.STRING)) + ) + : buildFilterTestExpectedQuery( + not(selector("dim1", "x6", extractionFn)) + ), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterCoalesceSameLiteralNotEqualsInjective() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo121'), 'xabc') <> 'xabc'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(not(equality("dim1", "abc", ColumnType.STRING))), + ImmutableList.of(new Object[]{NULL_STRING, 5L}) + ); + } + + @Test + public void testFilterCoalesceDifferentLiteral() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), 'xyzzy') = 'x6'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension("'x6'", equality("dim1", "6", ColumnType.STRING)), + Collections.emptyList() + ); + } + + @Test + public void testFilterCoalesceDifferentLiteralAlwaysFalse() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), 'xyzzy') = 'nonexistent'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryAlwaysFalse(), + Collections.emptyList() + ); + } + + @Test + public void testFilterCoalesceCastVarcharDifferentLiteral() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(CAST(LOOKUP(dim1, 'lookyloo') AS VARCHAR), 'xyzzy') = 'x6'"), + QUERY_CONTEXT, + buildFilterTestExpectedQueryConstantDimension("'x6'", equality("dim1", "6", ColumnType.STRING)), + Collections.emptyList() + ); + } + + @Test + public void testFilterCoalesceCastBigintDifferentLiteral() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(CAST(LOOKUP(dim1, 'lookyloo') AS BIGINT), 1) = 6"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "nvl(CAST(lookup(\"dim1\",'lookyloo'), 'LONG'),1)", ColumnType.LONG), + equality("v0", 6L, ColumnType.LONG) + ), + Collections.emptyList() + ); + } + + @Test + public void testFilterMvContainsCoalesceDifferentLiteral() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("MV_CONTAINS(COALESCE(LOOKUP(dim1, 'lookyloo'), 'xyzzy'), 'x6')"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(equality("dim1", "6", ColumnType.STRING)), + Collections.emptyList() + ); + } + + @Test + public void testFilterMvOverlapCoalesceDifferentLiteral() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql( + "MV_OVERLAP(COALESCE(LOOKUP(dim1, 'lookyloo'), 'xyzzy'), ARRAY['xabc', 'x6', 'nonexistent'])"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(in("dim1", ImmutableSet.of("6", "abc"), null)), + ImmutableList.of(new Object[]{"xabc", 1L}) + ); + } + + @Test + public void testFilterCoalesceDifferentLiteralNotEquals() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), 'xyzzy') <> 'x6'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(not(equality("dim1", "6", ColumnType.STRING))), + ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterCoalesceDifferentLiteralNotEqualsAlwaysTrue() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), 'xyzzy') <> 'nonexistent'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery(null), + ImmutableList.of( + new Object[]{NULL_STRING, 5L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterCoalesceSameColumn() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), dim1) = 'x6'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "nvl(lookup(\"dim1\",'lookyloo'),\"dim1\")", ColumnType.STRING), + equality("v0", "x6", ColumnType.STRING) + ), + Collections.emptyList() + ); + } + + @Test + public void testFilterInCoalesceSameColumn() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), dim1) IN ('xabc', '10.1')"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "nvl(lookup(\"dim1\",'lookyloo'),\"dim1\")", ColumnType.STRING), + in("v0", ImmutableList.of("10.1", "xabc"), null) + ), + ImmutableList.of( + new Object[]{NullHandling.defaultStringValue(), 1L}, + new Object[]{"xabc", 1L} + ) + ); + } + + @Test + public void testFilterCoalesceFunctionOfSameColumn() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), dim1 || '') = 'x6'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "nvl(lookup(\"dim1\",'lookyloo'),concat(\"dim1\",''))", ColumnType.STRING), + equality("v0", "x6", ColumnType.STRING) + ), + Collections.emptyList() + ); + } + + @Test + public void testFilterCoalesceDifferentColumn() + { + cannotVectorize(); + + testQuery( + buildFilterTestSql("COALESCE(LOOKUP(dim1, 'lookyloo'), dim2) = 'x6'"), + QUERY_CONTEXT, + buildFilterTestExpectedQuery( + expressionVirtualColumn("v0", "nvl(lookup(\"dim1\",'lookyloo'),\"dim2\")", ColumnType.STRING), + equality("v0", "x6", ColumnType.STRING) + ), + Collections.emptyList() + ); + } + + @Test + public void testFilterMaxUnapplyCount() + { + // Test to verify that "maxUnapplyCountForDruidReverseLookupRule" works properly. This ensures that the *other* + // tests are correctly validating that we aren't doing too many reverse lookups. + final DruidException e = Assert.assertThrows( + DruidException.class, + () -> testQuery( + buildFilterTestSql("LOOKUP(dim1, 'lookyloo') = 'xabc' OR LOOKUP(dim2, 'lookyloo') = 'x6'"), + QUERY_CONTEXT, + ImmutableList.of(), + ImmutableList.of() + ) + ); + + MatcherAssert.assertThat( + e, + ThrowableMessageMatcher.hasMessage(CoreMatchers.startsWith("Too many optimize calls[2]")) + ); + } + + @Test + public void testLookupReplaceMissingValueWith() + { + // Cannot vectorize due to extraction dimension specs. + cannotVectorize(); + + final RegisteredLookupExtractionFn extractionFn1 = + new RegisteredLookupExtractionFn(null, "lookyloo", false, "Missing_Value", null, false); + testQuery( + "SELECT\n" + + " LOOKUP(dim1, 'lookyloo', 'Missing_Value'),\n" + + " COALESCE(LOOKUP(dim1, 'lookyloo'), 'Missing_Value'), -- converted to the first form\n" + + " LOOKUP(dim1, 'lookyloo', null) as rmvNull,\n" + + " COUNT(*)\n" + + "FROM foo\n" + + "GROUP BY 1,2,3", + QUERY_CONTEXT, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new ExtractionDimensionSpec( + "dim1", + "d0", + ColumnType.STRING, + extractionFn1 + ), + new ExtractionDimensionSpec( + "dim1", + "d1", + ColumnType.STRING, + extractionFn1 + ), + new ExtractionDimensionSpec( + "dim1", + "d2", + ColumnType.STRING, + EXTRACTION_FN + ) + ) + ) + .setAggregatorSpecs(new CountAggregatorFactory("a0")) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"Missing_Value", "Missing_Value", NullHandling.defaultStringValue(), 5L}, + new Object[]{"xabc", "xabc", "xabc", 1L} + ) + ); + } + + @Test + public void testCountDistinctOfLookup() + { + // Cannot vectorize due to extraction dimension spec. + cannotVectorize(); + + testQuery( + "SELECT COUNT(DISTINCT LOOKUP(dim1, 'lookyloo')) FROM foo", + QUERY_CONTEXT, + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(aggregators( + new CardinalityAggregatorFactory( + "a0", + null, + ImmutableList.of(new ExtractionDimensionSpec("dim1", null, EXTRACTION_FN)), + false, + true + ) + )) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{NullHandling.replaceWithDefault() ? 2L : 1L} + ) + ); + } + + @Test + public void testLookupOnValueThatIsNull() + { + List expected; + if (useDefault) { + expected = ImmutableList.builder().add( + new Object[]{NULL_STRING, NULL_STRING}, + new Object[]{NULL_STRING, NULL_STRING}, + new Object[]{NULL_STRING, NULL_STRING} + ).build(); + } else { + expected = ImmutableList.builder().add( + new Object[]{NULL_STRING, NULL_STRING}, + new Object[]{NULL_STRING, NULL_STRING} + ).build(); + } + testQuery( + "SELECT dim2 ,lookup(dim2,'lookyloo') from foo where dim2 is null", + QUERY_CONTEXT, + ImmutableList.of( + new Druids.ScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + expressionVirtualColumn("v0", "null", ColumnType.STRING) + ) + .columns("v0") + .legacy(false) + .filters(isNull("dim2")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + expected + ); + } + + @Test + public void testLookupOnValueThatIsNotDistinctFromNull() + { + List expected; + if (useDefault) { + expected = ImmutableList.builder().add( + new Object[]{NULL_STRING, NULL_STRING}, + new Object[]{NULL_STRING, NULL_STRING}, + new Object[]{NULL_STRING, NULL_STRING} + ).build(); + } else { + expected = ImmutableList.builder().add( + new Object[]{NULL_STRING, NULL_STRING}, + new Object[]{NULL_STRING, NULL_STRING} + ).build(); + } + testQuery( + "SELECT dim2 ,lookup(dim2,'lookyloo') from foo where dim2 is not distinct from null", + QUERY_CONTEXT, + ImmutableList.of( + new Druids.ScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns( + expressionVirtualColumn("v0", "null", ColumnType.STRING) + ) + .columns("v0") + .legacy(false) + .filters(isNull("dim2")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + expected + ); + } + + @SqlTestFrameworkConfig(numMergeBuffers = 3) + @Test + public void testExactCountDistinct() + { + msqIncompatible(); + final String sqlQuery = "SELECT CAST(LOOKUP(dim1, 'lookyloo') AS VARCHAR), " + + "COUNT(DISTINCT foo.dim2), " + + "SUM(foo.cnt) FROM druid.foo " + + "GROUP BY 1"; + + // ExtractionDimensionSpec cannot be vectorized + cannotVectorize(); + + testQuery( + PLANNER_CONFIG_NO_HLL.withOverrides( + ImmutableMap.of( + PlannerConfig.CTX_KEY_USE_GROUPING_SET_FOR_EXACT_DISTINCT, + "true" + ) + ), + QUERY_CONTEXT, + sqlQuery, + CalciteTests.REGULAR_USER_AUTH_RESULT, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new ExtractionDimensionSpec( + "dim1", + "d0", + ColumnType.STRING, + EXTRACTION_FN + ), + new DefaultDimensionSpec("dim2", "d1", ColumnType.STRING) + )) + .setAggregatorSpecs( + aggregators( + new LongSumAggregatorFactory("a0", "cnt"), + new GroupingAggregatorFactory( + "a1", + Arrays.asList("dim1", "dim2") + ) + ) + ) + .setSubtotalsSpec( + ImmutableList.of( + ImmutableList.of("d0", "d1"), + ImmutableList.of("d0") + ) + ) + .build() + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("d0", "_d0", ColumnType.STRING)) + .setAggregatorSpecs(aggregators( + new FilteredAggregatorFactory( + new CountAggregatorFactory("_a0"), + and( + notNull("d1"), + equality("a1", 0L, ColumnType.LONG) + ) + ), + new FilteredAggregatorFactory( + new LongMinAggregatorFactory("_a1", "a0"), + equality("a1", 1L, ColumnType.LONG) + ) + )) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{NullHandling.defaultStringValue(), NullHandling.replaceWithDefault() ? 2L : 3L, 5L}, + new Object[]{"xabc", 0L, 1L} + ) + ); + } + + @Test + public void testPullUpLookup() + { + testQuery( + "SELECT LOOKUP(dim1, 'lookyloo121'), COUNT(*) FROM druid.foo GROUP BY 1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setPostAggregatorSpecs( + expressionPostAgg("p0", "lookup(\"d0\",'lookyloo121')", ColumnType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"x", 1L}, + new Object[]{"x1", 1L}, + new Object[]{"x10.1", 1L}, + new Object[]{"x2", 1L}, + new Object[]{"xabc", 1L}, + new Object[]{"xdef", 1L} + ) + ); + } + + @Test + public void testPullUpAndReverseLookup() + { + testQuery( + "SELECT LOOKUP(dim1, 'lookyloo121'), COUNT(*)\n" + + "FROM druid.foo\n" + + "WHERE LOOKUP(dim1, 'lookyloo121') IN ('xabc', 'xdef')\n" + + "GROUP BY 1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter(in("dim1", ImmutableList.of("abc", "def"), null)) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setPostAggregatorSpecs( + expressionPostAgg("p0", "lookup(\"d0\",'lookyloo121')", ColumnType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"xabc", 1L}, + new Object[]{"xdef", 1L} + ) + ); + } + + @Test + public void testDontPullUpLookupWhenUsedByAggregation() + { + cannotVectorize(); + + testQuery( + "SELECT LOOKUP(dim1, 'lookyloo121'), COUNT(LOOKUP(dim1, 'lookyloo121')) FROM druid.foo GROUP BY 1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + NullHandling.sqlCompatible() + ? new VirtualColumn[]{ + expressionVirtualColumn( + "v0", + "lookup(\"dim1\",'lookyloo121')", + ColumnType.STRING + ) + } + : new VirtualColumn[0] + ) + .setDimensions(dimensions(new ExtractionDimensionSpec("dim1", "d0", EXTRACTION_FN_121))) + .setAggregatorSpecs( + aggregators( + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + NullHandling.sqlCompatible() + ? not(isNull("v0")) + : not(selector("dim1", null, EXTRACTION_FN_121)) + ) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"x", 1L}, + new Object[]{"x1", 1L}, + new Object[]{"x10.1", 1L}, + new Object[]{"x2", 1L}, + new Object[]{"xabc", 1L}, + new Object[]{"xdef", 1L} + ) + ); + } + + @Test + public void testPullUpLookupGroupOnLookupInput() + { + testQuery( + "SELECT dim1, LOOKUP(dim1, 'lookyloo121'), COUNT(*) FROM druid.foo GROUP BY 1, 2", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setPostAggregatorSpecs( + expressionPostAgg("p0", "lookup(\"d0\",'lookyloo121')", ColumnType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"", "x", 1L}, + new Object[]{"1", "x1", 1L}, + new Object[]{"10.1", "x10.1", 1L}, + new Object[]{"2", "x2", 1L}, + new Object[]{"abc", "xabc", 1L}, + new Object[]{"def", "xdef", 1L} + ) + ); + } + + @Test + public void testPullUpLookupMoreDimensions() + { + testQuery( + "SELECT COUNT(*), dim2, dim1, LOOKUP(dim1, 'lookyloo121') FROM druid.foo GROUP BY 2, 3", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING), + new DefaultDimensionSpec("dim2", "d1", ColumnType.STRING) + )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setPostAggregatorSpecs( + expressionPostAgg("p0", "lookup(\"d0\",'lookyloo121')", ColumnType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{1L, "a", "", "x"}, + new Object[]{1L, "a", "1", "x1"}, + new Object[]{1L, NullHandling.defaultStringValue(), "10.1", "x10.1"}, + new Object[]{1L, "", "2", "x2"}, + new Object[]{1L, NullHandling.defaultStringValue(), "abc", "xabc"}, + new Object[]{1L, "abc", "def", "xdef"} + ) + ); + } + + @Test + public void testPullUpLookupOneInjectiveOneNot() + { + cannotVectorize(); + + testQuery( + "SELECT COUNT(*), LOOKUP(dim1, 'lookyloo'), LOOKUP(dim1, 'lookyloo121') FROM druid.foo GROUP BY 2, 3", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions( + new ExtractionDimensionSpec("dim1", "d0", EXTRACTION_FN), + new DefaultDimensionSpec("dim1", "d1", ColumnType.STRING) + )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setPostAggregatorSpecs( + expressionPostAgg("p0", "lookup(\"d1\",'lookyloo121')", ColumnType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{1L, NullHandling.defaultStringValue(), "x"}, + new Object[]{1L, NullHandling.defaultStringValue(), "x1"}, + new Object[]{1L, NullHandling.defaultStringValue(), "x10.1"}, + new Object[]{1L, NullHandling.defaultStringValue(), "x2"}, + new Object[]{1L, NullHandling.defaultStringValue(), "xdef"}, + new Object[]{1L, "xabc", "xabc"} + ) + ); + } + + private String buildFilterTestSql(final String conditionSql) + { + return "SELECT LOOKUP(dim1, 'lookyloo'), COUNT(*) FROM foo\n" + + "WHERE (" + conditionSql + ") AND TIME_IN_INTERVAL(__time, '2000/3000')\n" + + "GROUP BY LOOKUP(dim1, 'lookyloo')"; + } + + private List> buildFilterTestExpectedQuery( + @Nullable final VirtualColumn expectedVirtualColumn, + @Nullable final DimFilter expectedFilter + ) + { + return ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Intervals.of("2000/3000"))) + .setVirtualColumns(expectedVirtualColumn != null + ? VirtualColumns.create(expectedVirtualColumn) + : VirtualColumns.EMPTY) + .setGranularity(Granularities.ALL) + .setDimFilter(expectedFilter) + .setDimensions(new ExtractionDimensionSpec("dim1", "d0", ColumnType.STRING, EXTRACTION_FN)) + .setAggregatorSpecs(new CountAggregatorFactory("a0")) + .setContext(QUERY_CONTEXT) + .build() + ); + } + + private List> buildFilterTestExpectedQuery(@Nullable final DimFilter expectedFilter) + { + return buildFilterTestExpectedQuery(null, expectedFilter); + } + + private List> buildFilterTestExpectedQueryConstantDimension( + final String expectedConstantDimension, + @Nullable final DimFilter expectedFilter + ) + { + return ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Intervals.of("2000/3000"))) + .setVirtualColumns(expressionVirtualColumn("v0", expectedConstantDimension, ColumnType.STRING)) + .setGranularity(Granularities.ALL) + .setDimFilter(expectedFilter) + .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) + .setAggregatorSpecs(new CountAggregatorFactory("a0")) + .setContext(QUERY_CONTEXT) + .build() + ); + } + + private List> buildFilterTestExpectedQueryAlwaysFalse() + { + return ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(InlineDataSource.fromIterable( + ImmutableList.of(), + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("$f1", ColumnType.LONG) + .build() + )) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("$f1", "EXPR$0") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT) + .legacy(false) + .build() + ); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java index 97f41c425e4..e186476ef50 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java @@ -148,20 +148,18 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest // Cannot vectorize due to usage of expressions. cannotVectorize(); + // concat(dim3, '', 'foo') instad of concat(dim3, 'foo'), to disable the FilterDecomposeConcatRule rewrite testQuery( - "SELECT concat(dim3, 'foo'), SUM(cnt) FROM druid.numfoo where concat(dim3, 'foo') = 'bfoo' GROUP BY 1 ORDER BY 2 DESC", + "SELECT concat(dim3, '', 'foo'), SUM(cnt) FROM druid.numfoo where concat(dim3, '', 'foo') = 'bfoo' GROUP BY 1 ORDER BY 2 DESC", ImmutableList.of( GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "concat(\"dim3\",'foo')", ColumnType.STRING)) - .setDimensions( - dimensions( - new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING) - ) - ) - .setDimFilter(equality("dim3", "b", ColumnType.STRING)) + .setVirtualColumns( + expressionVirtualColumn("v0", "concat(\"dim3\",'','foo')", ColumnType.STRING)) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING))) + .setDimFilter(equality("v0", "bfoo", ColumnType.STRING)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setLimitSpec(new DefaultLimitSpec( ImmutableList.of(new OrderByColumnSpec( @@ -241,14 +239,15 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest @Test public void testMultiValueStringWorksLikeStringScanWithFilter() { + // concat(dim3, '', 'foo') instad of concat(dim3, 'foo'), to disable the FilterDecomposeConcatRule rewrite testQuery( - "SELECT concat(dim3, 'foo') FROM druid.numfoo where concat(dim3, 'foo') = 'bfoo'", + "SELECT concat(dim3, '', 'foo') FROM druid.numfoo where concat(dim3, '', 'foo') = 'bfoo'", ImmutableList.of( new Druids.ScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .eternityInterval() - .virtualColumns(expressionVirtualColumn("v0", "concat(\"dim3\",'foo')", ColumnType.STRING)) - .filters(equality("dim3", "b", ColumnType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "concat(\"dim3\",'','foo')", ColumnType.STRING)) + .filters(equality("v0", "bfoo", ColumnType.STRING)) .columns(ImmutableList.of("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -1111,7 +1110,11 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setPostAggregatorSpecs(expressionPostAgg("p0", "string_to_array('a,b',',')", ColumnType.STRING)) + .setPostAggregatorSpecs(expressionPostAgg( + "p0", + "string_to_array('a,b',',')", + ColumnType.STRING + )) .setDimensions(dimensions(new DefaultDimensionSpec("m1", "_d0", ColumnType.FLOAT))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2202,7 +2205,7 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest "dim3", "dim3", ColumnType.STRING, - new RegisteredLookupExtractionFn(null, "lookyloo", false, null, null, true) + new RegisteredLookupExtractionFn(null, "lookyloo", false, null, null, false) ), Collections.singleton(null), true 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 232228f5654..43e8e4f77c9 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 @@ -95,7 +95,6 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.NoopLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction; -import org.apache.druid.query.lookup.RegisteredLookupExtractionFn; import org.apache.druid.query.operator.ColumnWithDirection; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; @@ -186,6 +185,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"lookup", "lookyloo", "TABLE", "YES", "YES"}) + .add(new Object[]{"lookup", "lookyloo-chain", "TABLE", "YES", "YES"}) + .add(new Object[]{"lookup", "lookyloo121", "TABLE", "YES", "YES"}) .add(new Object[]{"sys", "segments", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"sys", "server_segments", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"sys", "servers", "SYSTEM_TABLE", "NO", "NO"}) @@ -225,6 +226,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"lookup", "lookyloo", "TABLE", "YES", "YES"}) + .add(new Object[]{"lookup", "lookyloo-chain", "TABLE", "YES", "YES"}) + .add(new Object[]{"lookup", "lookyloo121", "TABLE", "YES", "YES"}) .add(new Object[]{"sys", "segments", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"sys", "server_segments", "SYSTEM_TABLE", "NO", "NO"}) .add(new Object[]{"sys", "servers", "SYSTEM_TABLE", "NO", "NO"}) @@ -921,6 +924,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ) ); } + // This test the off-heap (buffer) version of the AnyAggregator (Double/Float/Long) against numeric columns // that have null values (when run in SQL compatible null mode) @Test @@ -2350,6 +2354,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ) ); } + @Test public void testFilterOnDouble() { @@ -2651,96 +2656,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } - @SqlTestFrameworkConfig(numMergeBuffers = 3) - @Test - public void testExactCountDistinctLookup() - { - msqIncompatible(); - final String sqlQuery = "SELECT CAST(LOOKUP(dim1, 'lookyloo') AS VARCHAR), " - + "COUNT(DISTINCT foo.dim2), " - + "SUM(foo.cnt) FROM druid.foo " - + "GROUP BY 1"; - - // ExtractionDimensionSpec cannot be vectorized - cannotVectorize(); - - testQuery( - PLANNER_CONFIG_NO_HLL.withOverrides( - ImmutableMap.of( - PlannerConfig.CTX_KEY_USE_GROUPING_SET_FOR_EXACT_DISTINCT, - "true" - ) - ), - sqlQuery, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of( - GroupByQuery.builder() - .setDataSource( - new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions( - new ExtractionDimensionSpec( - "dim1", - "d0", - ColumnType.STRING, - new RegisteredLookupExtractionFn( - null, - "lookyloo", - false, - null, - null, - true - ) - ), - new DefaultDimensionSpec("dim2", "d1", ColumnType.STRING) - )) - .setAggregatorSpecs( - aggregators( - new LongSumAggregatorFactory("a0", "cnt"), - new GroupingAggregatorFactory( - "a1", - Arrays.asList("dim1", "dim2") - ) - ) - ) - .setSubtotalsSpec( - ImmutableList.of( - ImmutableList.of("d0", "d1"), - ImmutableList.of("d0") - ) - ) - .build() - ) - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("d0", "_d0", ColumnType.STRING)) - .setAggregatorSpecs(aggregators( - new FilteredAggregatorFactory( - new CountAggregatorFactory("_a0"), - and( - notNull("d1"), - equality("a1", 0L, ColumnType.LONG) - ) - ), - new FilteredAggregatorFactory( - new LongMinAggregatorFactory("_a1", "a0"), - equality("a1", 1L, ColumnType.LONG) - ) - )) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of( - new Object[]{NullHandling.defaultStringValue(), NullHandling.replaceWithDefault() ? 2L : 3L, 5L}, - new Object[]{"xabc", 0L, 1L} - ) - ); - } - @Test public void testHavingOnFloatSum() { @@ -3269,11 +3184,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .virtualColumns( - expressionVirtualColumn( - "v0", - "case_searched((\"dim1\" == 'def'),\"dim2\",null)", - ColumnType.STRING - ) + expressionVirtualColumn("v0", "'def'", ColumnType.STRING), + expressionVirtualColumn("v1", "'abc'", ColumnType.STRING) ) .intervals(querySegmentSpec(Filtration.eternity())) .filters( @@ -3282,7 +3194,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest equality("dim2", "abc", ColumnType.STRING) ) ) - .columns("dim1", "dim2", "v0") + .columns("v0", "v1") .build() ), ImmutableList.of( @@ -5574,6 +5486,38 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testInIsNotTrueFilter() + { + testQuery( + "SELECT dim1, COUNT(*) FROM druid.foo WHERE dim1 IN ('abc', 'def', 'ghi') IS NOT TRUE GROUP BY dim1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) + .setDimFilter( + NullHandling.sqlCompatible() + ? not(istrue(in("dim1", ImmutableList.of("abc", "def", "ghi"), null))) + : not(in("dim1", ImmutableList.of("abc", "def", "ghi"), null))) + .setAggregatorSpecs( + aggregators( + new CountAggregatorFactory("a0") + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"", 1L}, + new Object[]{"1", 1L}, + new Object[]{"10.1", 1L}, + new Object[]{"2", 1L} + ) + ); + } + @Test public void testNotInOrIsNullFilter() { @@ -5640,6 +5584,46 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testInIsNotTrueAndLessThanFilter() + { + testQuery( + "SELECT dim1, COUNT(*) FROM druid.foo WHERE dim1 IN ('abc', 'def', 'ghi') IS NOT TRUE " + + "AND dim1 < 'zzz' GROUP BY dim1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) + .setDimFilter( + NullHandling.sqlCompatible() + ? and( + not(istrue(in("dim1", ImmutableList.of("abc", "def", "ghi"), null))), + range("dim1", ColumnType.STRING, null, "zzz", false, true) + ) + : and( + not(in("dim1", ImmutableList.of("abc", "def", "ghi"), null)), + range("dim1", ColumnType.STRING, null, "zzz", false, true) + ) + ) + .setAggregatorSpecs( + aggregators( + new CountAggregatorFactory("a0") + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"", 1L}, + new Object[]{"1", 1L}, + new Object[]{"10.1", 1L}, + new Object[]{"2", 1L} + ) + ); + } + @Test public void testSqlIsNullToInFilter() { @@ -8945,168 +8929,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } - @Test - public void testFilterAndGroupByLookup() - { - // Cannot vectorize due to extraction dimension specs. - cannotVectorize(); - - final RegisteredLookupExtractionFn extractionFn = new RegisteredLookupExtractionFn( - null, - "lookyloo", - false, - null, - null, - true - ); - GroupByQuery.Builder builder = - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new ExtractionDimensionSpec( - "dim1", - "d0", - ColumnType.STRING, - extractionFn - ) - ) - ) - .setAggregatorSpecs( - aggregators( - new CountAggregatorFactory("a0") - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT); - if (NullHandling.sqlCompatible()) { - builder = builder.setVirtualColumns( - expressionVirtualColumn("v0", "lookup(\"dim1\",'lookyloo')", ColumnType.STRING) - ) - .setDimFilter( - not(equality("v0", "xxx", ColumnType.STRING)) - ); - } else { - builder = builder.setDimFilter( - not(selector("dim1", "xxx", extractionFn)) - ); - } - testQuery( - "SELECT LOOKUP(dim1, 'lookyloo'), COUNT(*) FROM foo\n" - + "WHERE LOOKUP(dim1, 'lookyloo') <> 'xxx'\n" - + "GROUP BY LOOKUP(dim1, 'lookyloo')", - ImmutableList.of(builder.build()), - NullHandling.replaceWithDefault() - ? ImmutableList.of( - new Object[]{NULL_STRING, 5L}, - new Object[]{"xabc", 1L} - ) - // sql compatible mode expression filter (correctly) leaves out null values - : ImmutableList.of( - new Object[]{"xabc", 1L} - ) - ); - } - @Test - public void testLookupReplaceMissingValueWith() - { - // Cannot vectorize due to extraction dimension specs. - cannotVectorize(); - - final RegisteredLookupExtractionFn extractionFn1 = new RegisteredLookupExtractionFn( - null, - "lookyloo", - false, - "Missing_Value", - null, - true - ); - final RegisteredLookupExtractionFn extractionFnRMVNull = new RegisteredLookupExtractionFn( - null, - "lookyloo", - false, - null, - null, - true - ); - testQuery( - "SELECT LOOKUP(dim1, 'lookyloo', 'Missing_Value'), LOOKUP(dim1, 'lookyloo', null) as rmvNull, COUNT(*) FROM foo group by 1,2", - ImmutableList.of( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - dimensions( - new ExtractionDimensionSpec( - "dim1", - "d0", - ColumnType.STRING, - extractionFn1 - ), - new ExtractionDimensionSpec( - "dim1", - "d1", - ColumnType.STRING, - extractionFnRMVNull - ) - ) - ) - .setAggregatorSpecs( - aggregators( - new CountAggregatorFactory("a0") - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of( - new Object[]{"Missing_Value", NullHandling.defaultStringValue(), 5L}, - new Object[]{"xabc", "xabc", 1L} - ) - ); - } - - @Test - public void testCountDistinctOfLookup() - { - // Cannot vectorize due to extraction dimension spec. - cannotVectorize(); - final RegisteredLookupExtractionFn extractionFn = new RegisteredLookupExtractionFn( - null, - "lookyloo", - false, - null, - null, - true - ); - - testQuery( - "SELECT COUNT(DISTINCT LOOKUP(dim1, 'lookyloo')) FROM foo", - ImmutableList.of( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .aggregators(aggregators( - new CardinalityAggregatorFactory( - "a0", - null, - ImmutableList.of(new ExtractionDimensionSpec("dim1", null, extractionFn)), - false, - true - ) - )) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of( - new Object[]{NullHandling.replaceWithDefault() ? 2L : 1L} - ) - ); - } - @Test public void testGroupByExpressionFromLookup() { @@ -12004,7 +11826,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "concat(\"dim1\",'x',\"dim2\")", ColumnType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "'3x4'", ColumnType.STRING)) .filters(and( equality("dim1", "3", ColumnType.STRING), equality("dim2", "4", ColumnType.STRING) @@ -13254,78 +13076,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } - @Test - public void testLookupWithNull() - { - List expected; - if (useDefault) { - expected = ImmutableList.builder().add( - new Object[]{NULL_STRING, NULL_STRING}, - new Object[]{NULL_STRING, NULL_STRING}, - new Object[]{NULL_STRING, NULL_STRING} - ).build(); - } else { - expected = ImmutableList.builder().add( - new Object[]{NULL_STRING, NULL_STRING}, - new Object[]{NULL_STRING, NULL_STRING} - ).build(); - } - testQuery( - "SELECT dim2 ,lookup(dim2,'lookyloo') from foo where dim2 is null", - ImmutableList.of( - new Druids.ScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns( - expressionVirtualColumn("v0", "null", ColumnType.STRING) - ) - .columns("v0") - .legacy(false) - .filters(isNull("dim2")) - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - expected - ); - } - - @Test - public void testLookupWithIsNotDistinctFromNull() - { - List expected; - if (useDefault) { - expected = ImmutableList.builder().add( - new Object[]{NULL_STRING, NULL_STRING}, - new Object[]{NULL_STRING, NULL_STRING}, - new Object[]{NULL_STRING, NULL_STRING} - ).build(); - } else { - expected = ImmutableList.builder().add( - new Object[]{NULL_STRING, NULL_STRING}, - new Object[]{NULL_STRING, NULL_STRING} - ).build(); - } - testQuery( - "SELECT dim2 ,lookup(dim2,'lookyloo') from foo where dim2 is not distinct from null", - ImmutableList.of( - new Druids.ScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns( - expressionVirtualColumn("v0", "null", ColumnType.STRING) - ) - .columns("v0") - .legacy(false) - .filters(isNull("dim2")) - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - expected - ); - } - @Test public void testRoundFunc() { @@ -14084,6 +13834,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .queryContext(ImmutableMap.of()).run()); Assert.assertEquals("Illegal use of 'NULL' (line [1], column [24])", e2.getMessage()); } + @Test public void testStringAggMaxBytes() { @@ -14583,14 +14334,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + "group by 1", ImmutableList.of( GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE3) - .setInterval(querySegmentSpec(Intervals.ETERNITY)) - .setGranularity(Granularities.ALL) - .addDimension(new DefaultDimensionSpec("dim1", "_d0", ColumnType.STRING)) - .addAggregator(new LongSumAggregatorFactory("a0", "l1")) - .setPostAggregatorSpecs(ImmutableList.of( - expressionPostAgg("p0", "case_searched((\"a0\" == 0),1,0)", ColumnType.LONG))) - .build() + .setDataSource(CalciteTests.DATASOURCE3) + .setInterval(querySegmentSpec(Intervals.ETERNITY)) + .setGranularity(Granularities.ALL) + .addDimension(new DefaultDimensionSpec("dim1", "_d0", ColumnType.STRING)) + .addAggregator(new LongSumAggregatorFactory("a0", "l1")) + .setPostAggregatorSpecs(ImmutableList.of( + expressionPostAgg("p0", "case_searched((\"a0\" == 0),1,0)", ColumnType.LONG))) + .build() ), useDefault ? ImmutableList.of( @@ -14890,6 +14641,32 @@ public class CalciteQueryTest extends BaseCalciteQueryTest ); } + @Test + public void testFilterWithNVLAndInIsNotTrue() + { + testQuery( + "select __time, dim1 from druid.foo where (nvl(dim1, '') IN ('a' , '')) IS NOT TRUE", + ImmutableList.of( + newScanQueryBuilder() + .intervals(querySegmentSpec(Filtration.eternity())) + .dataSource(new TableDataSource(CalciteTests.DATASOURCE1)) + .context(QUERY_CONTEXT_DEFAULT) + .intervals(querySegmentSpec(Intervals.of( + "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"))) + .columns(ImmutableList.of("__time", "dim1")) + .filters(not(in("dim1", Arrays.asList("", "a"), null))) + .build() + ), + ImmutableList.of( + new Object[]{946771200000L, "10.1"}, + new Object[]{946857600000L, "2"}, + new Object[]{978307200000L, "1"}, + new Object[]{978393600000L, "def"}, + new Object[]{978480000000L, "abc"} + ) + ); + } + @Test public void testFilterWithNvlishCaseAndNotIn() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/rule/FilterDecomposeConcatRuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/rule/FilterDecomposeConcatRuleTest.java index 601d02c1ca8..2bdd5ed8125 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/rule/FilterDecomposeConcatRuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/rule/FilterDecomposeConcatRuleTest.java @@ -304,6 +304,25 @@ public class FilterDecomposeConcatRuleTest extends InitializedNullHandlingTest ); } + @Test + public void test_threeInputs_delimitersIgnoredWhenOutOfPosition() + { + final RexNode call = + equals( + concat(inputRef(0), literal(" ("), inputRef(1), literal("x"), inputRef(2), literal(")")), + literal("xxx (4x5)") // unambiguous, because 'x' before ' (' can be ignored + ); + + Assert.assertEquals( + and( + equals(inputRef(0), literal("xxx")), + equals(inputRef(1), literal("4")), + equals(inputRef(2), literal("5")) + ), + shuttle.apply(call) + ); + } + @Test public void test_twoInputs_backToBackLiterals() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/LookylooModule.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/LookylooModule.java index 4487372a15c..7dec7babb9b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/LookylooModule.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/LookylooModule.java @@ -22,10 +22,13 @@ package org.apache.druid.sql.calcite.util; import com.fasterxml.jackson.databind.Module; import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.multibindings.MapBinder; import org.apache.druid.guice.ExpressionModule; import org.apache.druid.initialization.DruidModule; import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; import org.apache.druid.query.expression.LookupExprMacro; +import org.apache.druid.query.extraction.MapLookupExtractor; +import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.sql.calcite.expression.builtin.QueryLookupOperatorConversion; @@ -35,26 +38,62 @@ import org.apache.druid.timeline.DataSegment; import java.util.List; /** - * Provides a lookup called {@link LookupEnabledTestExprMacroTable#LOOKYLOO}, provides the SQL {@code LOOKUP} - * function, and provides the native expression function {@code lookup}. + * Provides a lookup called {@link LookupEnabledTestExprMacroTable#LOOKYLOO}, a one-to-one lookup on "dim1" from + * {@link TestDataBuilder#FOO_SCHEMA} called {@link #LOOKYLOO_INJECTIVE}, a lookup chainable on lookyloo called + * {@link #LOOKYLOO_CHAINED}. Also adds the SQL {@code LOOKUP} function and the native expression + * function {@code lookup}. */ public class LookylooModule implements DruidModule { + private static final String LOOKYLOO_INJECTIVE = "lookyloo121"; + private static final String LOOKYLOO_CHAINED = "lookyloo-chain"; + @Override public void configure(Binder binder) { - final LookupExtractorFactoryContainerProvider lookupProvider = - LookupEnabledTestExprMacroTable.createTestLookupProvider( - ImmutableMap.of( - "a", "xa", - "abc", "xabc", - "nosuchkey", "mysteryvalue", - "6", "x6" - ) - ); + // Allows SqlBenchmark to add additional lookup tables. + final MapBinder lookupBinder = + MapBinder.newMapBinder(binder, String.class, LookupExtractor.class); + + lookupBinder.addBinding(LookupEnabledTestExprMacroTable.LOOKYLOO).toInstance( + new MapLookupExtractor( + ImmutableMap.builder() + .put("a", "xa") + .put("abc", "xabc") + .put("nosuchkey", "mysteryvalue") + .put("6", "x6") + .build(), + false + ) + ); + + lookupBinder.addBinding(LOOKYLOO_CHAINED).toInstance( + new MapLookupExtractor( + ImmutableMap.builder() + .put("xa", "za") + .put("xabc", "zabc") + .put("x6", "z6") + .build(), + false + ) + ); + + lookupBinder.addBinding(LOOKYLOO_INJECTIVE).toInstance( + new MapLookupExtractor( + ImmutableMap.builder() + .put("", "x") + .put("10.1", "x10.1") + .put("2", "x2") + .put("1", "x1") + .put("def", "xdef") + .put("abc", "xabc") + .build(), + true + ) + ); binder.bind(DataSegment.PruneSpecsHolder.class).toInstance(DataSegment.PruneSpecsHolder.DEFAULT); - binder.bind(LookupExtractorFactoryContainerProvider.class).toInstance(lookupProvider); + binder.bind(LookupExtractorFactoryContainerProvider.class).to(TestLookupProvider.class); SqlBindings.addOperatorConversion(binder, QueryLookupOperatorConversion.class); ExpressionModule.addExprMacro(binder, LookupExprMacro.class); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestLookupProvider.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestLookupProvider.java new file mode 100644 index 00000000000..f4f8bd14560 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestLookupProvider.java @@ -0,0 +1,58 @@ +/* + * 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.util; + +import com.google.inject.Inject; +import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.query.lookup.LookupExtractorFactoryContainer; +import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +public class TestLookupProvider implements LookupExtractorFactoryContainerProvider +{ + private final Map lookupMap; + + @Inject + public TestLookupProvider(Map lookupMap) + { + this.lookupMap = lookupMap; + } + + @Override + public Set getAllLookupNames() + { + return lookupMap.keySet(); + } + + @Override + public Optional get(String lookupName) + { + final LookupExtractor theLookup = lookupMap.get(lookupName); + if (theLookup != null) { + return Optional.of(new LookupEnabledTestExprMacroTable.TestLookupContainer(theLookup)); + } else { + return Optional.empty(); + } + } +}