mirror of https://github.com/apache/druid.git
Add a planner rule to handle empty tables (#11058)
* Add a planner rule to handle empty tables * adjust comment * type handling * add tests * unused imports and fix test * fix more tests * fix more test * javadoc
This commit is contained in:
parent
0df0bff44b
commit
b51ede5b49
|
@ -230,6 +230,12 @@
|
|||
<artifactId>equalsverifier</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-core</artifactId>
|
||||
<version>${mockito.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
|
|
@ -254,7 +254,11 @@ public class DruidPlanner implements Closeable
|
|||
final Supplier<Sequence<Object[]>> resultsSupplier = () -> {
|
||||
// sanity check
|
||||
Preconditions.checkState(
|
||||
plannerContext.getResources().isEmpty() == druidRel.getDataSourceNames().isEmpty(),
|
||||
plannerContext.getResources().isEmpty() == druidRel.getDataSourceNames().isEmpty()
|
||||
// The resources found in the plannerContext can be less than the datasources in
|
||||
// the query plan, because the query planner can eliminate empty tables by replacing
|
||||
// them with InlineDataSource of empty rows.
|
||||
|| plannerContext.getResources().size() >= druidRel.getDataSourceNames().size(),
|
||||
"Authorization sanity check failed"
|
||||
);
|
||||
if (root.isRefTrivial()) {
|
||||
|
|
|
@ -74,6 +74,7 @@ import org.apache.calcite.tools.Program;
|
|||
import org.apache.calcite.tools.Programs;
|
||||
import org.apache.calcite.tools.RelBuilder;
|
||||
import org.apache.druid.sql.calcite.rel.QueryMaker;
|
||||
import org.apache.druid.sql.calcite.rule.DruidLogicalValuesRule;
|
||||
import org.apache.druid.sql.calcite.rule.DruidRelToDruidRule;
|
||||
import org.apache.druid.sql.calcite.rule.DruidRules;
|
||||
import org.apache.druid.sql.calcite.rule.DruidTableScanRule;
|
||||
|
@ -242,6 +243,7 @@ public class Rules
|
|||
.addAll(baseRuleSet(plannerContext))
|
||||
.add(DruidRelToDruidRule.instance())
|
||||
.add(new DruidTableScanRule(queryMaker))
|
||||
.add(new DruidLogicalValuesRule(queryMaker))
|
||||
.addAll(DruidRules.rules(plannerContext));
|
||||
|
||||
return retVal.build();
|
||||
|
|
|
@ -29,11 +29,13 @@ import org.apache.calcite.plan.RelOptTable;
|
|||
import org.apache.calcite.plan.RelTraitSet;
|
||||
import org.apache.calcite.rel.RelWriter;
|
||||
import org.apache.calcite.rel.logical.LogicalTableScan;
|
||||
import org.apache.calcite.rel.logical.LogicalValues;
|
||||
import org.apache.calcite.rel.metadata.RelMetadataQuery;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.sql.calcite.table.DruidTable;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -41,21 +43,22 @@ import java.util.Set;
|
|||
*/
|
||||
public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
||||
{
|
||||
private final RelOptTable table;
|
||||
@Nullable
|
||||
private final RelOptTable table; // must not be null except for inline data
|
||||
private final DruidTable druidTable;
|
||||
private final PartialDruidQuery partialQuery;
|
||||
|
||||
private DruidQueryRel(
|
||||
final RelOptCluster cluster,
|
||||
final RelTraitSet traitSet,
|
||||
final RelOptTable table,
|
||||
@Nullable final RelOptTable table,
|
||||
final DruidTable druidTable,
|
||||
final QueryMaker queryMaker,
|
||||
final PartialDruidQuery partialQuery
|
||||
)
|
||||
{
|
||||
super(cluster, traitSet, queryMaker);
|
||||
this.table = Preconditions.checkNotNull(table, "table");
|
||||
this.table = table;
|
||||
this.druidTable = Preconditions.checkNotNull(druidTable, "druidTable");
|
||||
this.partialQuery = Preconditions.checkNotNull(partialQuery, "partialQuery");
|
||||
}
|
||||
|
@ -80,6 +83,22 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
|
|||
);
|
||||
}
|
||||
|
||||
public static DruidQueryRel fullScan(
|
||||
final LogicalValues valuesRel,
|
||||
final DruidTable druidTable,
|
||||
final QueryMaker queryMaker
|
||||
)
|
||||
{
|
||||
return new DruidQueryRel(
|
||||
valuesRel.getCluster(),
|
||||
valuesRel.getCluster().traitSetOf(Convention.NONE),
|
||||
null,
|
||||
druidTable,
|
||||
queryMaker,
|
||||
PartialDruidQuery.create(valuesRel)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidQuery toDruidQuery(final boolean finalizeAggregations)
|
||||
{
|
||||
|
|
|
@ -31,7 +31,7 @@ public class DruidRels
|
|||
public static Optional<DruidTable> druidTableIfLeafRel(final DruidRel<?> druidRel)
|
||||
{
|
||||
if (druidRel instanceof DruidQueryRel) {
|
||||
return Optional.of(druidRel.getTable().unwrap(DruidTable.class));
|
||||
return Optional.of(((DruidQueryRel) druidRel).getDruidTable());
|
||||
} else {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,126 @@
|
|||
/*
|
||||
* 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.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.calcite.plan.RelOptRule;
|
||||
import org.apache.calcite.plan.RelOptRuleCall;
|
||||
import org.apache.calcite.rel.logical.LogicalValues;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.sql.calcite.planner.Calcites;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.apache.druid.sql.calcite.rel.DruidQueryRel;
|
||||
import org.apache.druid.sql.calcite.rel.QueryMaker;
|
||||
import org.apache.druid.sql.calcite.table.DruidTable;
|
||||
import org.apache.druid.sql.calcite.table.RowSignatures;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* A {@link RelOptRule} that converts {@link LogicalValues} into {@link InlineDataSource}.
|
||||
* This rule is used when the query directly reads in-memory tuples. For example, given a query of
|
||||
* `SELECT 1 + 1`, the query planner will create {@link LogicalValues} that contains one tuple,
|
||||
* which in turn containing one column of value 2.
|
||||
*
|
||||
* The query planner can sometimes reduce a regular query to a query that reads in-memory tuples.
|
||||
* For example, `SELECT count(*) FROM foo WHERE 1 = 0` is reduced to `SELECT 0`. This rule will
|
||||
* be used for this case as well.
|
||||
*/
|
||||
public class DruidLogicalValuesRule extends RelOptRule
|
||||
{
|
||||
private final QueryMaker queryMaker;
|
||||
|
||||
public DruidLogicalValuesRule(QueryMaker queryMaker)
|
||||
{
|
||||
super(operand(LogicalValues.class, any()));
|
||||
this.queryMaker = queryMaker;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMatch(RelOptRuleCall call)
|
||||
{
|
||||
final LogicalValues values = call.rel(0);
|
||||
final List<ImmutableList<RexLiteral>> tuples = values.getTuples();
|
||||
final List<Object[]> objectTuples = tuples
|
||||
.stream()
|
||||
.map(tuple -> tuple
|
||||
.stream()
|
||||
.map(v -> getValueFromLiteral(v, queryMaker.getPlannerContext()))
|
||||
.collect(Collectors.toList())
|
||||
.toArray(new Object[0])
|
||||
)
|
||||
.collect(Collectors.toList());
|
||||
final RowSignature rowSignature = RowSignatures.fromRelDataType(
|
||||
values.getRowType().getFieldNames(),
|
||||
values.getRowType()
|
||||
);
|
||||
final DruidTable druidTable = new DruidTable(
|
||||
InlineDataSource.fromIterable(objectTuples, rowSignature),
|
||||
rowSignature,
|
||||
true,
|
||||
false
|
||||
);
|
||||
call.transformTo(
|
||||
DruidQueryRel.fullScan(values, druidTable, queryMaker)
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves value from the literal based on Druid data type mapping
|
||||
* (https://druid.apache.org/docs/latest/querying/sql.html#standard-types).
|
||||
*
|
||||
* @throws IllegalArgumentException for unsupported types
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static Object getValueFromLiteral(RexLiteral literal, PlannerContext plannerContext)
|
||||
{
|
||||
switch (literal.getType().getSqlTypeName()) {
|
||||
case CHAR:
|
||||
case VARCHAR:
|
||||
return literal.getValueAs(String.class);
|
||||
case FLOAT:
|
||||
return literal.getValueAs(Float.class);
|
||||
case DOUBLE:
|
||||
case REAL:
|
||||
case DECIMAL:
|
||||
return literal.getValueAs(Double.class);
|
||||
case TINYINT:
|
||||
case SMALLINT:
|
||||
case INTEGER:
|
||||
case BIGINT:
|
||||
return literal.getValueAs(Long.class);
|
||||
case BOOLEAN:
|
||||
return literal.isAlwaysTrue() ? 1L : 0L;
|
||||
case TIMESTAMP:
|
||||
case DATE:
|
||||
return Calcites.calciteDateTimeLiteralToJoda(literal, plannerContext.getTimeZone()).getMillis();
|
||||
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
|
||||
case TIME:
|
||||
case TIME_WITH_LOCAL_TIME_ZONE:
|
||||
default:
|
||||
throw new IAE("Unsupported type[%s]", literal.getTypeName());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.DateTimes;
|
|||
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.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
|
||||
|
@ -33,6 +34,8 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
|||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.query.scan.ScanQuery.ResultFormat;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.sql.calcite.filtration.Filtration;
|
||||
import org.apache.druid.sql.calcite.util.CalciteTests;
|
||||
|
@ -54,7 +57,20 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest
|
|||
{
|
||||
testQuery(
|
||||
"SELECT 1 + ?",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(new Object[]{2L}),
|
||||
RowSignature.builder().add("EXPR$0", ValueType.LONG).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("EXPR$0")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{2}
|
||||
),
|
||||
|
@ -588,11 +604,16 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testWrongTypeParameter() throws Exception
|
||||
{
|
||||
if (!useDefault) {
|
||||
// cannot vectorize inline datasource
|
||||
cannotVectorize();
|
||||
}
|
||||
testQuery(
|
||||
"SELECT COUNT(*)\n"
|
||||
+ "FROM druid.numfoo\n"
|
||||
+ "WHERE l1 > ? AND f1 = ?",
|
||||
useDefault ? ImmutableList.of(
|
||||
useDefault
|
||||
? ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
|
@ -606,7 +627,20 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest
|
|||
.aggregators(aggregators(new CountAggregatorFactory("a0")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
) : ImmutableList.of(),
|
||||
)
|
||||
: ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(),
|
||||
RowSignature.builder().add("f1", ValueType.FLOAT).add("l1", ValueType.LONG).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.aggregators(aggregators(new CountAggregatorFactory("a0")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(new SqlParameter(SqlType.BIGINT, 3L), new SqlParameter(SqlType.VARCHAR, "wat"))
|
||||
);
|
||||
|
|
|
@ -35,8 +35,11 @@ import org.apache.druid.java.util.common.granularity.Granularities;
|
|||
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.GlobalTableDataSource;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.JoinDataSource;
|
||||
import org.apache.druid.query.LookupDataSource;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
|
@ -88,6 +91,7 @@ import org.apache.druid.query.filter.NotDimFilter;
|
|||
import org.apache.druid.query.filter.RegexDimFilter;
|
||||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.GroupByQuery.Builder;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.ResultRow;
|
||||
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||
|
@ -96,11 +100,13 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction;
|
|||
import org.apache.druid.query.lookup.RegisteredLookupExtractionFn;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.query.scan.ScanQuery.ResultFormat;
|
||||
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||
import org.apache.druid.query.topn.DimensionTopNMetricSpec;
|
||||
import org.apache.druid.query.topn.InvertedTopNMetricSpec;
|
||||
import org.apache.druid.query.topn.NumericTopNMetricSpec;
|
||||
import org.apache.druid.query.topn.TopNQueryBuilder;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.join.JoinType;
|
||||
import org.apache.druid.server.QueryLifecycle;
|
||||
|
@ -175,7 +181,28 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
+ " CAST(-1 / 0.0 AS BIGINT),"
|
||||
+ " CAST(-1 / -0.0 AS BIGINT),"
|
||||
+ " CAST(0/ 0.0 AS BIGINT)",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(
|
||||
new Object[]{Long.MAX_VALUE, Long.MAX_VALUE, Long.MIN_VALUE, Long.MIN_VALUE, 0L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("EXPR$0", ValueType.LONG)
|
||||
.add("EXPR$1", ValueType.LONG)
|
||||
.add("EXPR$2", ValueType.LONG)
|
||||
.add("EXPR$3", ValueType.LONG)
|
||||
.add("EXPR$4", ValueType.LONG)
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("EXPR$0", "EXPR$1", "EXPR$2", "EXPR$3", "EXPR$4")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[] {
|
||||
Long.MAX_VALUE,
|
||||
|
@ -836,11 +863,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testExplainSelectConstantExpression() throws Exception
|
||||
{
|
||||
// Skip vectorization since otherwise the "context" will change for each subtest.
|
||||
skipVectorize();
|
||||
|
||||
testQuery(
|
||||
"EXPLAIN PLAN FOR SELECT 1 + 1",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
new Object[]{"BindableValues(tuples=[[{ 2 }]])\n", "[]"}
|
||||
new Object[]{"DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"EXPR$0\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{EXPR$0:LONG}])\n", "[]"}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -4603,7 +4633,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"SELECT COUNT(*)\n"
|
||||
+ "FROM druid.numfoo\n"
|
||||
+ "WHERE l1 IS NULL",
|
||||
useDefault ? ImmutableList.of() : ImmutableList.of(
|
||||
useDefault
|
||||
? ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(new Object[]{0L}),
|
||||
RowSignature.builder().add("EXPR$0", ValueType.LONG).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("EXPR$0")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
)
|
||||
: ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
|
@ -4626,7 +4671,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"SELECT COUNT(*)\n"
|
||||
+ "FROM druid.numfoo\n"
|
||||
+ "WHERE d1 IS NULL",
|
||||
useDefault ? ImmutableList.of() : ImmutableList.of(
|
||||
useDefault
|
||||
? ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(new Object[]{0L}),
|
||||
RowSignature.builder().add("EXPR$0", ValueType.LONG).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("EXPR$0")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
)
|
||||
: ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
|
@ -4650,7 +4710,22 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"SELECT COUNT(*)\n"
|
||||
+ "FROM druid.numfoo\n"
|
||||
+ "WHERE f1 IS NULL",
|
||||
useDefault ? ImmutableList.of() : ImmutableList.of(
|
||||
useDefault
|
||||
? ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(new Object[]{0L}),
|
||||
RowSignature.builder().add("EXPR$0", ValueType.LONG).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("EXPR$0")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
)
|
||||
: ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE3)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
|
@ -4904,7 +4979,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
"SELECT COUNT(*)\n"
|
||||
+ "FROM druid.foo\n"
|
||||
+ "WHERE NULLIF(dim2, 'a') = null",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(new Object[]{0L}),
|
||||
RowSignature.builder().add("EXPR$0", ValueType.LONG).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("EXPR$0")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(new Object[]{0L})
|
||||
);
|
||||
}
|
||||
|
@ -5044,10 +5132,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
|
||||
// JOIN condition with a function of both sides.
|
||||
"SELECT foo.dim1, foo.dim2, l.k, l.v\n"
|
||||
+ "FROM foo INNER JOIN lookup.lookyloo l ON CHARACTER_LENGTH(foo.dim2 || l.k) > 3\n",
|
||||
|
||||
// Interpreted as a JOIN against VALUES.
|
||||
"SELECT COUNT(*) FROM foo WHERE dim1 IN (NULL)"
|
||||
+ "FROM foo INNER JOIN lookup.lookyloo l ON CHARACTER_LENGTH(foo.dim2 || l.k) > 3\n"
|
||||
);
|
||||
|
||||
for (final String query : queries) {
|
||||
|
@ -5055,6 +5140,40 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupingWithNullInFilter() throws Exception
|
||||
{
|
||||
// HashJoinSegmentStorageAdapter is not vectorizable
|
||||
cannotVectorize();
|
||||
|
||||
testQuery(
|
||||
"SELECT COUNT(*) FROM foo WHERE dim1 IN (NULL)",
|
||||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(
|
||||
JoinDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE1),
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(new Object[]{null}),
|
||||
RowSignature.builder().add("ROW_VALUE", ValueType.STRING).build()
|
||||
),
|
||||
"j0.",
|
||||
"(\"dim1\" == \"j0.ROW_VALUE\")",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(aggregators(new CountAggregatorFactory("a0")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTwoExactCountDistincts() throws Exception
|
||||
{
|
||||
|
@ -5189,7 +5308,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
{
|
||||
testQuery(
|
||||
"SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE 1 = 0",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(new Object[]{0L, null}),
|
||||
RowSignature.builder().add("EXPR$0", ValueType.LONG).add("EXPR$1", ValueType.LONG).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("EXPR$0", "EXPR$1")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{0L, null}
|
||||
)
|
||||
|
@ -5246,7 +5378,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
{
|
||||
testQuery(
|
||||
"SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE 1 = 0 GROUP BY dim1",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(),
|
||||
RowSignature.builder().add("EXPR$0", ValueType.LONG).add("EXPR$1", ValueType.LONG).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("EXPR$0", "EXPR$1")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
@ -6517,11 +6662,25 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testCountStarWithNotOfDegenerateFilter() throws Exception
|
||||
{
|
||||
// This query is evaluated in the planner (no native queries are issued) due to the degenerate filter.
|
||||
// HashJoinSegmentStorageAdapter is not vectorizable
|
||||
cannotVectorize();
|
||||
|
||||
testQuery(
|
||||
"SELECT COUNT(*) FROM druid.foo WHERE dim2 = 'a' and not (dim1 > 'a' OR dim1 < 'b')",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(),
|
||||
RowSignature.builder().add("dim1", ValueType.STRING).add("dim2", ValueType.STRING).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(aggregators(new CountAggregatorFactory("a0")))
|
||||
.context(TIMESERIES_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
@ -7376,7 +7535,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
|
||||
testQuery(
|
||||
"SELECT * FROM (SELECT dim2 FROM druid.foo ORDER BY dim2 LIMIT 2 OFFSET 5) OFFSET 2",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(),
|
||||
RowSignature.builder().add("dim2", ValueType.STRING).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("dim2")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
@ -7388,7 +7560,20 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
|
||||
testQuery(
|
||||
"SELECT * FROM (SELECT DISTINCT dim2 FROM druid.foo ORDER BY dim2 LIMIT 2 OFFSET 5) OFFSET 2",
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(),
|
||||
RowSignature.builder().add("dim2", ValueType.STRING).build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("dim2")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of()
|
||||
);
|
||||
}
|
||||
|
@ -9262,12 +9447,41 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
@Test
|
||||
public void testSelectCurrentTimeAndDateLosAngeles() throws Exception
|
||||
{
|
||||
DateTimeZone timeZone = DateTimes.inferTzFromString(LOS_ANGELES);
|
||||
testQuery(
|
||||
PLANNER_CONFIG_DEFAULT,
|
||||
QUERY_CONTEXT_LOS_ANGELES,
|
||||
"SELECT CURRENT_TIMESTAMP, CURRENT_DATE, CURRENT_DATE + INTERVAL '1' DAY",
|
||||
CalciteTests.REGULAR_USER_AUTH_RESULT,
|
||||
ImmutableList.of(),
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(
|
||||
new Object[]{
|
||||
// milliseconds of timestamps as if they were in UTC. This looks strange
|
||||
// but intentional because they are what Calcite gives us.
|
||||
// See DruidLogicalValuesRule.getValueFromLiteral()
|
||||
// and Calcites.calciteDateTimeLiteralToJoda.
|
||||
new DateTime("2000-01-01T00Z", timeZone).withZone(DateTimeZone.UTC).getMillis(),
|
||||
new DateTime("1999-12-31", timeZone).withZone(DateTimeZone.UTC).getMillis(),
|
||||
new DateTime("2000-01-01", timeZone).withZone(DateTimeZone.UTC).getMillis()
|
||||
}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("CURRENT_TIMESTAMP", ValueType.LONG)
|
||||
.add("CURRENT_DATE", ValueType.LONG)
|
||||
.add("EXPR$2", ValueType.LONG)
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.columns("CURRENT_DATE", "CURRENT_TIMESTAMP", "EXPR$2")
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.context(QUERY_CONTEXT_LOS_ANGELES)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{timestamp("2000-01-01T00Z", LOS_ANGELES), day("1999-12-31"), day("2000-01-01")}
|
||||
)
|
||||
|
@ -17159,7 +17373,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testLookupWithNull() throws Exception
|
||||
{
|
||||
|
@ -17195,4 +17408,82 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
expected
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLeftJoinRightTableCanBeEmpty() throws Exception
|
||||
{
|
||||
// HashJoinSegmentStorageAdapter is not vectorizable
|
||||
cannotVectorize();
|
||||
|
||||
final DataSource rightTable;
|
||||
if (useDefault) {
|
||||
rightTable = InlineDataSource.fromIterable(
|
||||
ImmutableList.of(),
|
||||
RowSignature.builder().add("dim2", ValueType.STRING).build()
|
||||
);
|
||||
} else {
|
||||
rightTable = new QueryDataSource(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.filters(new SelectorDimFilter("m2", null, null))
|
||||
.columns("dim2")
|
||||
.legacy(false)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
testQuery(
|
||||
"SELECT v1.dim2, count(1) "
|
||||
+ "FROM (SELECT * FROM foo where m1 > 2) v1 "
|
||||
+ "LEFT OUTER JOIN ("
|
||||
+ " select dim2 from (select * from foo where m2 is null)"
|
||||
+ ") sm ON v1.dim2 = sm.dim2 "
|
||||
+ "group by 1",
|
||||
ImmutableList.of(
|
||||
new Builder()
|
||||
.setDataSource(
|
||||
JoinDataSource.create(
|
||||
new QueryDataSource(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.filters(new BoundDimFilter("m1", "2", null, true, false, null, null, StringComparators.NUMERIC))
|
||||
.columns("dim2")
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
rightTable,
|
||||
"j0.",
|
||||
"(\"dim2\" == \"j0.dim2\")",
|
||||
JoinType.LEFT,
|
||||
null,
|
||||
ExprMacroTable.nil()
|
||||
)
|
||||
)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(
|
||||
new DefaultDimensionSpec("dim2", "d0", ValueType.STRING)
|
||||
)
|
||||
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
useDefault
|
||||
? ImmutableList.of(
|
||||
new Object[]{"", 2L},
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"abc", 1L}
|
||||
)
|
||||
: ImmutableList.of(
|
||||
new Object[]{null, 1L},
|
||||
new Object[]{"", 1L},
|
||||
new Object[]{"a", 1L},
|
||||
new Object[]{"abc", 1L}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -333,6 +333,9 @@ public class DruidRelsTest
|
|||
final T mockRel = EasyMock.mock(clazz);
|
||||
EasyMock.expect(mockRel.getPartialDruidQuery()).andReturn(mockPartialQuery).anyTimes();
|
||||
EasyMock.expect(mockRel.getTable()).andReturn(mockRelOptTable).anyTimes();
|
||||
if (clazz == DruidQueryRel.class) {
|
||||
EasyMock.expect(((DruidQueryRel) mockRel).getDruidTable()).andReturn(druidTable).anyTimes();
|
||||
}
|
||||
additionalExpectationsFunction.accept(mockRel);
|
||||
|
||||
EasyMock.replay(mockRel, mockPartialQuery, mockRelOptTable);
|
||||
|
|
|
@ -0,0 +1,190 @@
|
|||
/*
|
||||
* 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.ImmutableList;
|
||||
import org.apache.calcite.rel.type.RelDataType;
|
||||
import org.apache.calcite.rex.RexBuilder;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
|
||||
import org.apache.calcite.sql.type.SqlTypeName;
|
||||
import org.apache.calcite.util.DateString;
|
||||
import org.apache.calcite.util.TimeString;
|
||||
import org.apache.calcite.util.TimestampString;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.sql.calcite.planner.DruidTypeSystem;
|
||||
import org.apache.druid.sql.calcite.planner.PlannerContext;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.runners.Enclosed;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
import org.mockito.ArgumentMatchers;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
@RunWith(Enclosed.class)
|
||||
public class DruidLogicalValuesRuleTest
|
||||
{
|
||||
private static final PlannerContext DEFAULT_CONTEXT = Mockito.mock(PlannerContext.class);
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public static class GetValueFromLiteralSimpleTypesTest
|
||||
{
|
||||
@Parameters(name = "{1}, {2}")
|
||||
public static Iterable<Object[]> constructorFeeder()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new Object[]{"test", SqlTypeName.CHAR, String.class},
|
||||
new Object[]{"test", SqlTypeName.VARCHAR, String.class},
|
||||
new Object[]{0.1, SqlTypeName.DOUBLE, Double.class},
|
||||
new Object[]{0.1, SqlTypeName.REAL, Double.class},
|
||||
new Object[]{0.1, SqlTypeName.DECIMAL, Double.class},
|
||||
new Object[]{1L, SqlTypeName.TINYINT, Long.class},
|
||||
new Object[]{1L, SqlTypeName.SMALLINT, Long.class},
|
||||
new Object[]{1L, SqlTypeName.INTEGER, Long.class},
|
||||
new Object[]{1L, SqlTypeName.BIGINT, Long.class}
|
||||
);
|
||||
}
|
||||
|
||||
private final Object val;
|
||||
private final SqlTypeName sqlTypeName;
|
||||
private final Class<?> javaType;
|
||||
|
||||
public GetValueFromLiteralSimpleTypesTest(Object val, SqlTypeName sqlTypeName, Class<?> javaType)
|
||||
{
|
||||
this.val = val;
|
||||
this.sqlTypeName = sqlTypeName;
|
||||
this.javaType = javaType;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetValueFromLiteral()
|
||||
{
|
||||
final RexLiteral literal = makeLiteral(val, sqlTypeName, javaType);
|
||||
final Object fromLiteral = DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT);
|
||||
Assert.assertSame(javaType, fromLiteral.getClass());
|
||||
Assert.assertEquals(val, fromLiteral);
|
||||
Mockito.verify(literal, Mockito.times(1)).getType();
|
||||
Mockito.verify(literal, Mockito.times(1)).getValueAs(ArgumentMatchers.any());
|
||||
}
|
||||
|
||||
private static RexLiteral makeLiteral(Object val, SqlTypeName typeName, Class<?> javaType)
|
||||
{
|
||||
RelDataType dataType = Mockito.mock(RelDataType.class);
|
||||
Mockito.when(dataType.getSqlTypeName()).thenReturn(typeName);
|
||||
RexLiteral literal = Mockito.mock(RexLiteral.class);
|
||||
Mockito.when(literal.getType()).thenReturn(dataType);
|
||||
Mockito.when(literal.getValueAs(ArgumentMatchers.any())).thenReturn(javaType.cast(val));
|
||||
return literal;
|
||||
}
|
||||
}
|
||||
|
||||
public static class GetValueFromLiteralOtherTypesTest
|
||||
{
|
||||
private static final PlannerContext DEFAULT_CONTEXT = Mockito.mock(PlannerContext.class);
|
||||
private static final DateTimeZone TIME_ZONE = DateTimes.inferTzFromString("Asia/Seoul");
|
||||
private static final RexBuilder REX_BUILDER = new RexBuilder(new SqlTypeFactoryImpl(DruidTypeSystem.INSTANCE));
|
||||
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
@BeforeClass
|
||||
public static void setup()
|
||||
{
|
||||
Mockito.when(DEFAULT_CONTEXT.getTimeZone()).thenReturn(TIME_ZONE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetValueFromTrueLiteral()
|
||||
{
|
||||
RexLiteral literal = REX_BUILDER.makeLiteral(true);
|
||||
|
||||
final Object fromLiteral = DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT);
|
||||
Assert.assertSame(Long.class, fromLiteral.getClass());
|
||||
Assert.assertEquals(1L, fromLiteral);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetValueFromFalseLiteral()
|
||||
{
|
||||
RexLiteral literal = REX_BUILDER.makeLiteral(false);
|
||||
|
||||
final Object fromLiteral = DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT);
|
||||
Assert.assertSame(Long.class, fromLiteral.getClass());
|
||||
Assert.assertEquals(0L, fromLiteral);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetValueFromTimestampLiteral()
|
||||
{
|
||||
RexLiteral literal = REX_BUILDER.makeTimestampLiteral(new TimestampString("2021-04-01 16:54:31"), 0);
|
||||
|
||||
final Object fromLiteral = DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT);
|
||||
Assert.assertSame(Long.class, fromLiteral.getClass());
|
||||
Assert.assertEquals(new DateTime("2021-04-01T16:54:31", TIME_ZONE).getMillis(), fromLiteral);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetValueFromDateLiteral()
|
||||
{
|
||||
RexLiteral literal = REX_BUILDER.makeDateLiteral(new DateString("2021-04-01"));
|
||||
|
||||
final Object fromLiteral = DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT);
|
||||
Assert.assertSame(Long.class, fromLiteral.getClass());
|
||||
Assert.assertEquals(new DateTime("2021-04-01", TIME_ZONE).getMillis(), fromLiteral);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetValueFromTimestampWithLocalTimeZoneLiteral()
|
||||
{
|
||||
RexLiteral literal = REX_BUILDER.makeTimestampWithLocalTimeZoneLiteral(
|
||||
new TimestampString("2021-04-01 16:54:31"),
|
||||
0
|
||||
);
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Unsupported type[TIMESTAMP_WITH_LOCAL_TIME_ZONE]");
|
||||
DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetValueFromTimeLiteral()
|
||||
{
|
||||
RexLiteral literal = REX_BUILDER.makeTimeLiteral(new TimeString("16:54:31"), 0);
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Unsupported type[TIME]");
|
||||
DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetValueFromTimeWithLocalTimeZoneLiteral()
|
||||
{
|
||||
RexLiteral literal = REX_BUILDER.makeTimeWithLocalTimeZoneLiteral(new TimeString("16:54:31"), 0);
|
||||
expectedException.expect(IllegalArgumentException.class);
|
||||
expectedException.expectMessage("Unsupported type[TIME_WITH_LOCAL_TIME_ZONE]");
|
||||
DruidLogicalValuesRule.getValueFromLiteral(literal, DEFAULT_CONTEXT);
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue