mirror of https://github.com/apache/druid.git
Add zero period support to TIMESTAMPADD (#10550)
* Allow zero period for TIMESTAMPADD * update test cases * add empty zone test case * add unit test cases for TimestampShiftMacro
This commit is contained in:
parent
1272fb17e5
commit
d7d2c804ad
|
@ -19,8 +19,8 @@
|
|||
|
||||
package org.apache.druid.query.expression;
|
||||
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
|
@ -60,14 +60,9 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
|
|||
}
|
||||
}
|
||||
|
||||
private static PeriodGranularity getGranularity(final List<Expr> args, final Expr.ObjectBinding bindings)
|
||||
private static Period getPeriod(final List<Expr> args, final Expr.ObjectBinding bindings)
|
||||
{
|
||||
return ExprUtils.toPeriodGranularity(
|
||||
args.get(1),
|
||||
null,
|
||||
args.size() > 3 ? args.get(3) : null,
|
||||
bindings
|
||||
);
|
||||
return new Period(args.get(1).eval(bindings).asString());
|
||||
}
|
||||
|
||||
private static int getStep(final List<Expr> args, final Expr.ObjectBinding bindings)
|
||||
|
@ -75,6 +70,17 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
|
|||
return args.get(2).eval(bindings).asInt();
|
||||
}
|
||||
|
||||
private static ISOChronology getTimeZone(final List<Expr> args, final Expr.ObjectBinding bindings)
|
||||
{
|
||||
final Expr timeZoneArg = args.size() > 3 ? args.get(3) : null;
|
||||
if (timeZoneArg == null) {
|
||||
return ISOChronology.getInstance(null);
|
||||
} else {
|
||||
final String zone = timeZoneArg.eval(bindings).asString();
|
||||
return ISOChronology.getInstance(zone != null ? DateTimes.inferTzFromString(zone) : null);
|
||||
}
|
||||
}
|
||||
|
||||
private static class TimestampShiftExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
|
||||
{
|
||||
private final Chronology chronology;
|
||||
|
@ -84,9 +90,8 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
|
|||
TimestampShiftExpr(final List<Expr> args)
|
||||
{
|
||||
super(FN_NAME, args);
|
||||
final PeriodGranularity granularity = getGranularity(args, ExprUtils.nilBindings());
|
||||
period = granularity.getPeriod();
|
||||
chronology = ISOChronology.getInstance(granularity.getTimeZone());
|
||||
period = getPeriod(args, ExprUtils.nilBindings());
|
||||
chronology = getTimeZone(args, ExprUtils.nilBindings());
|
||||
step = getStep(args, ExprUtils.nilBindings());
|
||||
}
|
||||
|
||||
|
@ -123,9 +128,8 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
|
|||
@Override
|
||||
public ExprEval eval(final ObjectBinding bindings)
|
||||
{
|
||||
final PeriodGranularity granularity = getGranularity(args, bindings);
|
||||
final Period period = granularity.getPeriod();
|
||||
final Chronology chronology = ISOChronology.getInstance(granularity.getTimeZone());
|
||||
final Period period = getPeriod(args, bindings);
|
||||
final Chronology chronology = getTimeZone(args, bindings);
|
||||
final int step = getStep(args, bindings);
|
||||
return ExprEval.of(chronology.add(period, args.get(0).eval(bindings).asLong(), step));
|
||||
}
|
||||
|
|
|
@ -0,0 +1,241 @@
|
|||
/*
|
||||
* 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.query.expression;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExprMacroTable;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Days;
|
||||
import org.joda.time.Minutes;
|
||||
import org.joda.time.Months;
|
||||
import org.joda.time.Years;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
|
||||
public class TimestampShiftMacroTest extends MacroTestBase
|
||||
{
|
||||
public TimestampShiftMacroTest()
|
||||
{
|
||||
super(new TimestampShiftExprMacro());
|
||||
}
|
||||
|
||||
private DateTime timestamp = DateTimes.of("2020-11-05T04:05:06");
|
||||
//TIME_SHIFT(<timestamp_expr>, <period>, <step>, [<timezone>])
|
||||
|
||||
@Test
|
||||
public void testZeroArguments()
|
||||
{
|
||||
expectException(IAE.class, "Function[timestamp_shift] must have 3 to 4 arguments");
|
||||
apply(Collections.emptyList());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneArguments()
|
||||
{
|
||||
expectException(IAE.class, "Function[timestamp_shift] must have 3 to 4 arguments");
|
||||
apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr()
|
||||
));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTwoArguments()
|
||||
{
|
||||
expectException(IAE.class, "Function[timestamp_shift] must have 3 to 4 arguments");
|
||||
apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||
ExprEval.of("P1M").toExpr()
|
||||
));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMoreThanFourArguments()
|
||||
{
|
||||
expectException(IAE.class, "Function[timestamp_shift] must have 3 to 4 arguments");
|
||||
apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||
ExprEval.of("P1M").toExpr(),
|
||||
ExprEval.of("1").toExpr(),
|
||||
ExprEval.of("+08:00").toExpr(),
|
||||
ExprEval.of("extra").toExpr()
|
||||
));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZeroStep()
|
||||
{
|
||||
int step = 0;
|
||||
Expr expr = apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||
ExprEval.of("P1M").toExpr(),
|
||||
ExprEval.of(step).toExpr()
|
||||
));
|
||||
|
||||
Assert.assertEquals(
|
||||
timestamp.withPeriodAdded(Months.ONE, step).getMillis(),
|
||||
expr.eval(ExprUtils.nilBindings()).asLong()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPositiveStep()
|
||||
{
|
||||
int step = 5;
|
||||
Expr expr = apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||
ExprEval.of("P1M").toExpr(),
|
||||
ExprEval.of(step).toExpr()
|
||||
));
|
||||
|
||||
Assert.assertEquals(
|
||||
timestamp.withPeriodAdded(Months.ONE, step).getMillis(),
|
||||
expr.eval(ExprUtils.nilBindings()).asLong()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNegativeStep()
|
||||
{
|
||||
int step = -3;
|
||||
Expr expr = apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||
ExprEval.of("P1M").toExpr(),
|
||||
ExprEval.of(step).toExpr()
|
||||
));
|
||||
|
||||
Assert.assertEquals(
|
||||
timestamp.withPeriodAdded(Months.ONE, step).getMillis(),
|
||||
expr.eval(ExprUtils.nilBindings()).asLong()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPeriodMinute()
|
||||
{
|
||||
Expr expr = apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||
ExprEval.of("PT1M").toExpr(),
|
||||
ExprEval.of(1).toExpr()
|
||||
));
|
||||
|
||||
Assert.assertEquals(
|
||||
timestamp.withPeriodAdded(Minutes.ONE, 1).getMillis(),
|
||||
expr.eval(ExprUtils.nilBindings()).asLong()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPeriodDay()
|
||||
{
|
||||
Expr expr = apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||
ExprEval.of("P1D").toExpr(),
|
||||
ExprEval.of(1).toExpr()
|
||||
));
|
||||
|
||||
Assert.assertEquals(
|
||||
timestamp.withPeriodAdded(Days.ONE, 1).getMillis(),
|
||||
expr.eval(ExprUtils.nilBindings()).asLong()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPeriodYearAndTimeZone()
|
||||
{
|
||||
Expr expr = apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||
ExprEval.of("P1Y").toExpr(),
|
||||
ExprEval.of(1).toExpr(),
|
||||
ExprEval.of("America/Los_Angeles").toExpr()
|
||||
));
|
||||
|
||||
Assert.assertEquals(
|
||||
timestamp.toDateTime(DateTimes.inferTzFromString("America/Los_Angeles")).withPeriodAdded(Years.ONE, 1).getMillis(),
|
||||
expr.eval(ExprUtils.nilBindings()).asLong()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDynamicExpression()
|
||||
{
|
||||
// step parameter is not a literal expression
|
||||
Expr expr = apply(
|
||||
ImmutableList.of(
|
||||
ExprEval.of(timestamp.getMillis()).toExpr(),
|
||||
ExprEval.of("P1Y").toExpr(),
|
||||
new NotLiteralExpr("step"),
|
||||
ExprEval.of("America/Los_Angeles").toExpr()
|
||||
));
|
||||
|
||||
final int step = 3;
|
||||
Assert.assertEquals(
|
||||
timestamp.toDateTime(DateTimes.inferTzFromString("America/Los_Angeles")).withPeriodAdded(Years.ONE, step).getMillis(),
|
||||
expr.eval(new Expr.ObjectBinding()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Object get(String name)
|
||||
{
|
||||
if ("step".equals(name)) {
|
||||
return step;
|
||||
} else {
|
||||
throw new IAE("Invalid bindings");
|
||||
}
|
||||
}
|
||||
}).asLong()
|
||||
);
|
||||
}
|
||||
|
||||
private static class NotLiteralExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
|
||||
{
|
||||
NotLiteralExpr(String name)
|
||||
{
|
||||
super(name, ExprEval.of(name).toExpr());
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExprEval eval(ObjectBinding bindings)
|
||||
{
|
||||
return ExprEval.bestEffortOf(bindings.get(name));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expr visit(Shuttle shuttle)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -121,6 +121,7 @@ public class ExprMacroTest
|
|||
assertExpr("timestamp_shift(t, 'P1D', 2, 'America/Los_Angeles')", DateTimes.of("2000-02-05T04:05:06").getMillis());
|
||||
assertExpr("timestamp_shift(t, 'P1D', 2, CityOfAngels)", DateTimes.of("2000-02-05T04:05:06").getMillis());
|
||||
assertExpr("timestamp_shift(t, 'P1D', 2, '-08:00')", DateTimes.of("2000-02-05T04:05:06").getMillis());
|
||||
assertExpr("timestamp_shift(t, 'P1D', 2, '')", DateTimes.of("2000-02-05T04:05:06").getMillis());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -21,7 +21,9 @@ package org.apache.druid.sql.calcite.expression.builtin;
|
|||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.calcite.rex.RexCall;
|
||||
import org.apache.calcite.rex.RexLiteral;
|
||||
import org.apache.calcite.rex.RexNode;
|
||||
import org.apache.calcite.sql.SqlKind;
|
||||
import org.apache.calcite.sql.SqlOperator;
|
||||
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
|
||||
import org.apache.calcite.sql.type.SqlTypeFamily;
|
||||
|
@ -89,7 +91,10 @@ public abstract class TimeArithmeticOperatorConversion implements SqlOperatorCon
|
|||
leftExpr,
|
||||
rightExpr.map(
|
||||
simpleExtraction -> null,
|
||||
expression -> StringUtils.format("concat('P', %s, 'M')", expression)
|
||||
expression ->
|
||||
rightRexNode.isA(SqlKind.LITERAL) ?
|
||||
StringUtils.format("'P%sM'", RexLiteral.value(rightRexNode)) :
|
||||
StringUtils.format("concat('P', %s, 'M')", expression)
|
||||
),
|
||||
DruidExpression.fromExpression(DruidExpression.numberLiteral(direction > 0 ? 1 : -1)),
|
||||
DruidExpression.fromExpression(DruidExpression.stringLiteral(plannerContext.getTimeZone().getID()))
|
||||
|
|
|
@ -127,6 +127,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@RunWith(JUnitParamsRunner.class)
|
||||
public class CalciteQueryTest extends BaseCalciteQueryTest
|
||||
|
@ -16016,6 +16017,142 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimeStampAddZeroDayPeriod() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT TIMESTAMPADD(DAY, 0, \"__time\") FROM druid.foo",
|
||||
|
||||
// verify if SQL matches given native query
|
||||
ImmutableList.of(newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(
|
||||
expressionVirtualColumn("v0", "(\"__time\" + 0)", ValueType.LONG)
|
||||
)
|
||||
.columns("v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()),
|
||||
|
||||
//Since adding a zero period does not change the timestamp, just compare the stamp with the orignal
|
||||
CalciteTests.ROWS1.stream()
|
||||
.map(row -> new Object[]{row.getTimestampFromEpoch()})
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimeStampAddZeroMonthPeriod() throws Exception
|
||||
{
|
||||
testQuery(
|
||||
"SELECT TIMESTAMPADD(MONTH, 0, \"__time\") FROM druid.foo",
|
||||
|
||||
// verify if SQL matches given native query
|
||||
ImmutableList.of(newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(
|
||||
expressionVirtualColumn("v0", "timestamp_shift(\"__time\",'P0M',1,'UTC')", ValueType.LONG)
|
||||
)
|
||||
.columns("v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()),
|
||||
|
||||
//Since adding a zero period does not change the timestamp, just compare the stamp with the orignal
|
||||
CalciteTests.ROWS1.stream()
|
||||
.map(row -> new Object[]{row.getTimestampFromEpoch()})
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimeStampAddZeroYearPeriod() throws Exception
|
||||
{
|
||||
skipVectorize();
|
||||
|
||||
testQuery(
|
||||
"SELECT TIMESTAMPADD(YEAR, 0, \"__time\") FROM druid.foo",
|
||||
|
||||
// verify if SQL matches given native query
|
||||
ImmutableList.of(newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(
|
||||
expressionVirtualColumn("v0", "timestamp_shift(\"__time\",'P0M',1,'UTC')", ValueType.LONG)
|
||||
)
|
||||
.columns("v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()),
|
||||
|
||||
//Since adding a zero period does not change the timestamp, just compare the stamp with the orignal
|
||||
CalciteTests.ROWS1.stream()
|
||||
.map(row -> new Object[]{row.getTimestampFromEpoch()})
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* TIMESTAMPADD is converted to timestamp_shift function call and its parameters will be converted to a Period string or an expression
|
||||
* see https://github.com/apache/druid/issues/10530 for more information
|
||||
*/
|
||||
@Test
|
||||
public void testTimeStampAddConversion() throws Exception
|
||||
{
|
||||
final PeriodGranularity periodGranularity = new PeriodGranularity(new Period("P1M"), null, null);
|
||||
|
||||
//
|
||||
// 2nd parameter for TIMESTAMPADD is literal, it will be translated to 'P1M' string
|
||||
//
|
||||
testQuery(
|
||||
"SELECT TIMESTAMPADD(MONTH, 1, \"__time\") FROM druid.foo",
|
||||
|
||||
// verify if SQL matches given native query
|
||||
ImmutableList.of(newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(
|
||||
expressionVirtualColumn("v0", "timestamp_shift(\"__time\",'P1M',1,'UTC')", ValueType.LONG)
|
||||
)
|
||||
.columns("v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()),
|
||||
|
||||
// verify if query results match the given
|
||||
CalciteTests.ROWS1.stream()
|
||||
.map(r -> new Object[]{periodGranularity.increment(r.getTimestamp()).getMillis()})
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
|
||||
//
|
||||
// 2nd parameter for TIMESTAMPADD is an expression, it will be explained as a function call: concat('P', (1 * \"cnt\"), 'M')
|
||||
//
|
||||
testQuery(
|
||||
"SELECT TIMESTAMPADD(MONTH, \"cnt\", \"__time\") FROM druid.foo",
|
||||
|
||||
// verify if SQL matches given native query
|
||||
ImmutableList.of(newScanQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(
|
||||
expressionVirtualColumn("v0", "timestamp_shift(\"__time\",concat('P', (1 * \"cnt\"), 'M'),1,'UTC')", ValueType.LONG)
|
||||
)
|
||||
.columns("v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.build()),
|
||||
|
||||
// verify if query results match the given
|
||||
// "cnt" for each row is 1
|
||||
CalciteTests.ROWS1.stream()
|
||||
.map(row -> new Object[]{periodGranularity.increment(row.getTimestamp()).getMillis()})
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* This is a provider of query contexts that should be used by join tests.
|
||||
* It tests various configs that can be passed to join queries. All the configs provided by this provider should
|
||||
|
|
|
@ -1475,7 +1475,7 @@ public class ExpressionsTest extends ExpressionTestBase
|
|||
),
|
||||
DruidExpression.of(
|
||||
null,
|
||||
"timestamp_shift(\"t\",concat('P', 13, 'M'),1,'UTC')"
|
||||
"timestamp_shift(\"t\",'P13M',1,'UTC')"
|
||||
),
|
||||
DateTimes.of("2000-02-03T04:05:06").plus(period).getMillis()
|
||||
);
|
||||
|
@ -1521,7 +1521,7 @@ public class ExpressionsTest extends ExpressionTestBase
|
|||
),
|
||||
DruidExpression.of(
|
||||
null,
|
||||
"timestamp_shift(\"t\",concat('P', 13, 'M'),-1,'UTC')"
|
||||
"timestamp_shift(\"t\",'P13M',-1,'UTC')"
|
||||
),
|
||||
DateTimes.of("2000-02-03T04:05:06").minus(period).getMillis()
|
||||
);
|
||||
|
|
Loading…
Reference in New Issue