mirror of
https://github.com/apache/druid.git
synced 2025-02-24 11:45:34 +00:00
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:
parent
72d15ab321
commit
2e42d04038
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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(
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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)
|
||||
|
Loading…
x
Reference in New Issue
Block a user