SQL: Create millisecond precision timestamp literals. (#12407)

* SQL: Create millisecond precision timestamp literals.

Fixes a bug where implicit casts of strings to timestamps would use seconds
precision rather than milliseconds. The new test case
testCountStarWithBetweenTimeFilterUsingMillisecondsInStringLiterals
exercises this.

* Update sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java

Co-authored-by: Frank Chen <frankchen@apache.org>

* Correct precision handling.

- Set default precision to 3 (millis) for things involving timestamps.
- Respect precision specified in types when available.

* Silence, checkstyle.

Co-authored-by: Frank Chen <frankchen@apache.org>
This commit is contained in:
Gian Merlino 2022-04-27 14:17:07 -07:00 committed by GitHub
parent 72d15ab321
commit 2e42d04038
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 225 additions and 32 deletions

View File

@ -25,6 +25,7 @@ import com.google.common.io.BaseEncoding;
import com.google.common.primitives.Chars;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlCollation;
@ -82,7 +83,8 @@ public class Calcites
private static final DateTimeFormatter CALCITE_TIME_PRINTER = DateTimeFormat.forPattern("HH:mm:ss.S");
private static final DateTimeFormatter CALCITE_DATE_PRINTER = DateTimeFormat.forPattern("yyyy-MM-dd");
private static final DateTimeFormatter CALCITE_TIMESTAMP_PRINTER = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.S");
private static final DateTimeFormatter CALCITE_TIMESTAMP_PRINTER =
DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS");
private static final Charset DEFAULT_CHARSET = Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME);
@ -187,6 +189,7 @@ public class Calcites
{
return SqlTypeName.FRACTIONAL_TYPES.contains(sqlTypeName) || SqlTypeName.APPROX_TYPES.contains(sqlTypeName);
}
public static boolean isLongType(SqlTypeName sqlTypeName)
{
return SqlTypeName.TIMESTAMP == sqlTypeName ||
@ -299,20 +302,27 @@ public class Calcites
}
/**
* Calcite expects TIMESTAMP literals to be represented by TimestampStrings in the local time zone.
* Creates a Calcite TIMESTAMP literal from a Joda DateTime.
*
* @param dateTime joda timestamp
* @param timeZone session time zone
* @param dateTime joda timestamp
* @param sessionTimeZone session time zone
*
* @return Calcite style Calendar, appropriate for literals
*/
public static TimestampString jodaToCalciteTimestampString(final DateTime dateTime, final DateTimeZone timeZone)
public static RexLiteral jodaToCalciteTimestampLiteral(
final RexBuilder rexBuilder,
final DateTime dateTime,
final DateTimeZone sessionTimeZone,
final int precision
)
{
// The replaceAll is because Calcite doesn't like trailing zeroes in its fractional seconds part.
String timestampString = TRAILING_ZEROS
.matcher(CALCITE_TIMESTAMP_PRINTER.print(dateTime.withZone(timeZone)))
// Calcite expects TIMESTAMP literals to be represented by TimestampStrings in the session time zone.
// The TRAILING_ZEROS ... replaceAll is because Calcite doesn't like trailing zeroes in its fractional seconds part.
final String timestampString = TRAILING_ZEROS
.matcher(CALCITE_TIMESTAMP_PRINTER.print(dateTime.withZone(sessionTimeZone)))
.replaceAll("");
return new TimestampString(timestampString);
return rexBuilder.makeTimestampLiteral(new TimestampString(timestampString), precision);
}
/**

View File

@ -0,0 +1,52 @@
/*
* 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.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlOperatorBinding;
import org.apache.calcite.sql.fun.SqlAbstractTimeFunction;
import org.apache.calcite.sql.type.SqlTypeName;
/**
* Used for functions like CURRENT_TIMESTAMP and LOCALTIME.
*
* Similar to {@link SqlAbstractTimeFunction}, but default precision is
* {@link DruidTypeSystem#DEFAULT_TIMESTAMP_PRECISION} instead of 0.
*/
public class CurrentTimestampSqlFunction extends SqlAbstractTimeFunction
{
private final SqlTypeName typeName;
public CurrentTimestampSqlFunction(final String name, final SqlTypeName typeName)
{
super(name, typeName);
this.typeName = typeName;
}
@Override
public RelDataType inferReturnType(SqlOperatorBinding opBinding)
{
if (opBinding.getOperandCount() == 0) {
return opBinding.getTypeFactory().createSqlType(typeName, DruidTypeSystem.DEFAULT_TIMESTAMP_PRECISION);
} else {
return super.inferReturnType(opBinding);
}
}
}

View File

@ -23,10 +23,13 @@ import com.google.common.collect.ImmutableList;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlCall;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlLiteral;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.fun.SqlLibraryOperators;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.sql2rel.SqlRexContext;
import org.apache.calcite.sql2rel.SqlRexConvertlet;
import org.apache.calcite.sql2rel.SqlRexConvertletTable;
@ -43,12 +46,24 @@ public class DruidConvertletTable implements SqlRexConvertletTable
// Apply a convertlet that doesn't do anything other than a "dumb" call translation.
private static final SqlRexConvertlet BYPASS_CONVERTLET = StandardConvertletTable.INSTANCE::convertCall;
/**
* Use instead of {@link SqlStdOperatorTable#CURRENT_TIMESTAMP} to get the proper default precision.
*/
private static final SqlFunction CURRENT_TIMESTAMP =
new CurrentTimestampSqlFunction("CURRENT_TIMESTAMP", SqlTypeName.TIMESTAMP);
/**
* Use instead of {@link SqlStdOperatorTable#LOCALTIMESTAMP} to get the proper default precision.
*/
private static final SqlFunction LOCALTIMESTAMP =
new CurrentTimestampSqlFunction("LOCALTIMESTAMP", SqlTypeName.TIMESTAMP);
private static final List<SqlOperator> CURRENT_TIME_CONVERTLET_OPERATORS =
ImmutableList.<SqlOperator>builder()
.add(SqlStdOperatorTable.CURRENT_TIMESTAMP)
.add(CURRENT_TIMESTAMP)
.add(SqlStdOperatorTable.CURRENT_TIME)
.add(SqlStdOperatorTable.CURRENT_DATE)
.add(SqlStdOperatorTable.LOCALTIMESTAMP)
.add(LOCALTIMESTAMP)
.add(SqlStdOperatorTable.LOCALTIME)
.build();
@ -130,11 +145,20 @@ public class DruidConvertletTable implements SqlRexConvertletTable
public RexNode convertCall(final SqlRexContext cx, final SqlCall call)
{
final SqlOperator operator = call.getOperator();
if (operator.equals(SqlStdOperatorTable.CURRENT_TIMESTAMP)
|| operator.equals(SqlStdOperatorTable.LOCALTIMESTAMP)) {
return cx.getRexBuilder().makeTimestampLiteral(
Calcites.jodaToCalciteTimestampString(plannerContext.getLocalNow(), plannerContext.getTimeZone()),
RelDataType.PRECISION_NOT_SPECIFIED
if (CURRENT_TIMESTAMP.equals(operator) || LOCALTIMESTAMP.equals(operator)) {
int precision = DruidTypeSystem.DEFAULT_TIMESTAMP_PRECISION;
if (call.operandCount() > 0) {
// Call is CURRENT_TIMESTAMP(precision) or LOCALTIMESTAMP(precision)
final SqlLiteral precisionLiteral = call.operand(0);
precision = precisionLiteral.intValue(true);
}
return Calcites.jodaToCalciteTimestampLiteral(
cx.getRexBuilder(),
plannerContext.getLocalNow(),
plannerContext.getTimeZone(),
precision
);
} else if (operator.equals(SqlStdOperatorTable.CURRENT_TIME) || operator.equals(SqlStdOperatorTable.LOCALTIME)) {
return cx.getRexBuilder().makeTimeLiteral(

View File

@ -19,7 +19,6 @@
package org.apache.druid.sql.calcite.planner;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexExecutor;
import org.apache.calcite.rex.RexNode;
@ -110,12 +109,11 @@ public class DruidRexExecutor implements RexExecutor
throw new UnsupportedSQLQueryException("Illegal TIMESTAMP constant: %s", constExp);
}
literal = rexBuilder.makeTimestampLiteral(
Calcites.jodaToCalciteTimestampString(
DateTimes.utc(exprResult.asLong()),
plannerContext.getTimeZone()
),
RelDataType.PRECISION_NOT_SPECIFIED
literal = Calcites.jodaToCalciteTimestampLiteral(
rexBuilder,
DateTimes.utc(exprResult.asLong()),
plannerContext.getTimeZone(),
constExp.getType().getPrecision()
);
} else if (SqlTypeName.NUMERIC_TYPES.contains(sqlTypeName)) {
final BigDecimal bigDecimal;

View File

@ -28,6 +28,11 @@ public class DruidTypeSystem implements RelDataTypeSystem
{
public static final DruidTypeSystem INSTANCE = new DruidTypeSystem();
/**
* Druid uses millisecond precision for timestamps internally. This is also the default at the SQL layer.
*/
public static final int DEFAULT_TIMESTAMP_PRECISION = 3;
private DruidTypeSystem()
{
// Singleton.
@ -42,7 +47,13 @@ public class DruidTypeSystem implements RelDataTypeSystem
@Override
public int getDefaultPrecision(final SqlTypeName typeName)
{
return RelDataTypeSystem.DEFAULT.getDefaultPrecision(typeName);
switch (typeName) {
case TIMESTAMP:
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return DEFAULT_TIMESTAMP_PRECISION;
default:
return RelDataTypeSystem.DEFAULT.getDefaultPrecision(typeName);
}
}
@Override

View File

@ -5540,6 +5540,48 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testCountStarWithBetweenTimeFilterUsingMilliseconds() throws Exception
{
testQuery(
"SELECT COUNT(*) FROM druid.foo "
+ "WHERE __time BETWEEN TIMESTAMP '2000-01-01 00:00:00' AND TIMESTAMP '2000-12-31 23:59:59.999'",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Intervals.of("2000-01-01/2001-01-01")))
.granularity(Granularities.ALL)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{3L}
)
);
}
@Test
public void testCountStarWithBetweenTimeFilterUsingMillisecondsInStringLiterals() throws Exception
{
testQuery(
"SELECT COUNT(*) FROM druid.foo "
+ "WHERE __time BETWEEN '2000-01-01 00:00:00' AND '2000-12-31 23:59:59.999'",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(querySegmentSpec(Intervals.of("2000-01-01/2001-01-01")))
.granularity(Granularities.ALL)
.aggregators(aggregators(new CountAggregatorFactory("a0")))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{3L}
)
);
}
@Test
public void testRemoveUselessCaseWhen() throws Exception
{

View File

@ -43,14 +43,18 @@ import org.apache.druid.query.topn.TopNQueryBuilder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.sql.SqlPlanningException;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.junit.Test;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class CalciteSelectQueryTest extends BaseCalciteQueryTest
{
@ -879,11 +883,22 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
@Test
public void testSelectCurrentTimeAndDateLosAngeles() throws Exception
{
final Map<String, Object> context = new HashMap<>(QUERY_CONTEXT_DEFAULT);
context.put(PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00.123Z");
context.put(PlannerContext.CTX_SQL_TIME_ZONE, LOS_ANGELES);
DateTimeZone timeZone = DateTimes.inferTzFromString(LOS_ANGELES);
testQuery(
PLANNER_CONFIG_DEFAULT,
QUERY_CONTEXT_LOS_ANGELES,
"SELECT CURRENT_TIMESTAMP, CURRENT_DATE, CURRENT_DATE + INTERVAL '1' DAY",
context,
"SELECT "
+ "CURRENT_TIMESTAMP, "
+ "CURRENT_TIMESTAMP(0), "
+ "CURRENT_TIMESTAMP(1), "
+ "CURRENT_TIMESTAMP(2), "
+ "CURRENT_TIMESTAMP(3), "
+ "CURRENT_DATE, "
+ "CURRENT_DATE + INTERVAL '1' DAY",
CalciteTests.REGULAR_USER_AUTH_RESULT,
ImmutableList.of(
Druids.newScanQueryBuilder()
@ -895,31 +910,66 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
// 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("2000-01-01T00:00:00.123Z", timeZone)
.withZone(DateTimeZone.UTC).getMillis(),
new DateTime("2000-01-01T00:00:00.000Z", timeZone)
.withZone(DateTimeZone.UTC).getMillis(),
new DateTime("2000-01-01T00:00:00.100Z", timeZone)
.withZone(DateTimeZone.UTC).getMillis(),
new DateTime("2000-01-01T00:00:00.120Z", timeZone)
.withZone(DateTimeZone.UTC).getMillis(),
new DateTime("2000-01-01T00:00:00.123Z", 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", ColumnType.LONG)
.add("CURRENT_DATE", ColumnType.LONG)
.add("EXPR$1", ColumnType.LONG)
.add("EXPR$2", ColumnType.LONG)
.add("EXPR$3", ColumnType.LONG)
.add("EXPR$4", ColumnType.LONG)
.add("CURRENT_DATE", ColumnType.LONG)
.add("EXPR$6", ColumnType.LONG)
.build()
)
)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("CURRENT_DATE", "CURRENT_TIMESTAMP", "EXPR$2")
.columns("CURRENT_DATE", "CURRENT_TIMESTAMP", "EXPR$1", "EXPR$2", "EXPR$3", "EXPR$4", "EXPR$6")
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
.legacy(false)
.context(QUERY_CONTEXT_LOS_ANGELES)
.context(context)
.build()
),
ImmutableList.of(
new Object[]{timestamp("2000-01-01T00Z", LOS_ANGELES), day("1999-12-31"), day("2000-01-01")}
new Object[]{
timestamp("2000-01-01T00:00:00.123Z", LOS_ANGELES),
timestamp("2000-01-01T00:00:00.000Z", LOS_ANGELES),
timestamp("2000-01-01T00:00:00.100Z", LOS_ANGELES),
timestamp("2000-01-01T00:00:00.120Z", LOS_ANGELES),
timestamp("2000-01-01T00:00:00.123Z", LOS_ANGELES),
day("1999-12-31"),
day("2000-01-01")
}
)
);
}
@Test
public void testSelectCurrentTimePrecisionTooHigh() throws Exception
{
testQueryThrows(
"SELECT CURRENT_TIMESTAMP(4)",
expectedException -> {
expectedException.expect(SqlPlanningException.class);
expectedException.expectMessage(
"Argument to function 'CURRENT_TIMESTAMP' must be a valid precision between '0' and '3'"
);
}
);
}
@Test
public void testSelectCountStar() throws Exception
{

View File

@ -45,6 +45,7 @@ import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.virtual.VirtualizedColumnSelectorFactory;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.DruidTypeSystem;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
@ -138,7 +139,12 @@ class ExpressionTestHelper
RexNode makeLiteral(DateTime timestamp)
{
return rexBuilder.makeTimestampLiteral(Calcites.jodaToCalciteTimestampString(timestamp, DateTimeZone.UTC), 0);
return Calcites.jodaToCalciteTimestampLiteral(
rexBuilder,
timestamp,
DateTimeZone.UTC,
DruidTypeSystem.DEFAULT_TIMESTAMP_PRECISION
);
}
RexNode makeLiteral(Integer integer)