Skip to content

Commit

Permalink
Add zero period support to TIMESTAMPADD (apache#10550)
Browse files Browse the repository at this point in the history
* Allow zero period for TIMESTAMPADD

* update test cases

* add empty zone test case

* add unit test cases for TimestampShiftMacro
  • Loading branch information
FrankChen021 authored and JulianJaffePinterest committed Jan 22, 2021
1 parent d40d898 commit dbec0b7
Show file tree
Hide file tree
Showing 6 changed files with 405 additions and 17 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -60,21 +60,27 @@ public Expr apply(final List<Expr> args)
}
}

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)
{
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;
Expand All @@ -84,9 +90,8 @@ private static class TimestampShiftExpr extends ExprMacroTable.BaseScalarMacroFu
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());
}

Expand Down Expand Up @@ -123,9 +128,8 @@ private static class TimestampShiftDynamicExpr extends ExprMacroTable.BaseScalar
@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));
}
Expand Down
Original file line number Diff line number Diff line change
@@ -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;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -121,6 +121,7 @@ public void testTimestampShift()
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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,9 @@

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;
Expand Down Expand Up @@ -89,7 +91,10 @@ public DruidExpression toDruidExpression(
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()))
Expand Down
Loading

0 comments on commit dbec0b7

Please sign in to comment.