Expressions: fixes for round-trips of floating point literals, Long.MIN_VALUE literals, Shuffle.visitAll. (#13037)

* SQL: Fix round-trips of floating point literals.

When writing RexLiterals into Druid expressions, we now write non-integer
numeric literals in such a way that ensures they are parsed as doubles
on the other end.

* Updates from code review, and some additional stuff inspired by the
investigation.

- Remove unnecessary formatting code from DruidExpression.doubleLiteral:
  it handles things just fine with its default behavior.

- Fix a problem where expression literals could not represent Long.MIN_VALUE.
  Now, integer literals start life off as BigIntegerExpr instead of LongExpr,
  and are converted to LongExpr during flattening. This is necessary because,
  in order to avoid ambiguity between unary minus and negative literals, our
  grammar does not actually have true negative literals. Negative numbers must
  be represented as unary minus next to a positive literal.

- Fix a bug  introduced in #12230 where shuttle.visitAll(args) delegated
  to shuttle.visit(arg) instead of arg.visit(shuttle). The latter does
  a recursive visitation, which is the intended behavior.

* Style fixes.

* Move regexp to the right place.
This commit is contained in:
Gian Merlino 2022-09-12 17:06:20 -07:00 committed by GitHub
parent c00ad28ecc
commit 77925cdcdd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 307 additions and 41 deletions

View File

@ -29,6 +29,7 @@ import org.apache.druid.math.expr.vector.VectorProcessors;
import org.apache.druid.segment.column.TypeStrategy;
import javax.annotation.Nullable;
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Objects;
@ -101,6 +102,59 @@ abstract class ConstantExpr<T> implements Expr
}
}
/**
* Even though expressions don't generally support BigInteger, we need this object so we can represent
* {@link Long#MIN_VALUE} as a {@link UnaryMinusExpr} applied to {@link ConstantExpr}. Antlr cannot parse negative
* longs directly, due to ambiguity between negative numbers and unary minus.
*/
class BigIntegerExpr extends ConstantExpr<BigInteger>
{
public BigIntegerExpr(BigInteger value)
{
super(ExpressionType.LONG, Preconditions.checkNotNull(value, "value"));
}
@Override
public String toString()
{
return value.toString();
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
// Eval succeeds if the BigInteger is in long range.
// Callers that need to process out-of-long-range values, like UnaryMinusExpr, must use getLiteralValue().
return ExprEval.ofLong(value.longValueExact());
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
// No vectorization needed: Parser.flatten converts BigIntegerExpr to LongExpr at parse time.
return false;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
BigIntegerExpr otherExpr = (BigIntegerExpr) o;
return value.equals(otherExpr.value);
}
@Override
public int hashCode()
{
return value.hashCode();
}
}
class LongExpr extends ConstantExpr<Long>
{
LongExpr(Long value)

View File

@ -406,7 +406,7 @@ public interface Expr extends Cacheable
final List<Expr> newExprs = new ArrayList<>();
for (final Expr arg : exprs) {
newExprs.add(visit(arg));
newExprs.add(arg.visit(this));
}
return newExprs;

View File

@ -30,6 +30,7 @@ import org.apache.druid.math.expr.antlr.ExprBaseListener;
import org.apache.druid.math.expr.antlr.ExprParser;
import javax.annotation.Nullable;
import java.math.BigInteger;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
@ -145,7 +146,7 @@ public class ExprListenerImpl extends ExprBaseListener
{
nodes.put(
ctx,
new LongExpr(Long.parseLong(ctx.getText()))
new BigIntegerExpr(new BigInteger(ctx.getText()))
);
}

View File

@ -182,6 +182,14 @@ public class Parser
}
}
return childExpr;
}).visit(childExpr -> {
// Now that unary minus has been applied to any BigIntegerExpr where it is needed, transform all remaining
// BigIntegerExpr into LongExpr.
if (childExpr instanceof BigIntegerExpr) {
return childExpr.eval(InputBindings.nilBindings()).toExpr();
} else {
return childExpr;
}
});
}

View File

@ -29,6 +29,7 @@ import org.apache.druid.math.expr.vector.VectorProcessors;
import org.apache.druid.segment.column.Types;
import javax.annotation.Nullable;
import java.math.BigInteger;
import java.util.Objects;
/**
@ -122,6 +123,10 @@ class UnaryMinusExpr extends UnaryExpr
@Override
public ExprEval eval(ObjectBinding bindings)
{
if (expr instanceof BigIntegerExpr) {
// Special case to handle unary minus for Long.MIN_VALUE: converting the literal to long directly is impossible
return ExprEval.of(((BigInteger) expr.getLiteralValue()).multiply(BigInteger.valueOf(-1)).longValueExact());
}
ExprEval ret = expr.eval(bindings);
if (NullHandling.sqlCompatible() && (ret.value() == null)) {
return ExprEval.of(null);

View File

@ -29,12 +29,15 @@ import org.apache.druid.segment.column.TypeStrategies;
import org.apache.druid.segment.column.TypeStrategiesTest;
import org.apache.druid.segment.column.TypeStrategy;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.hamcrest.CoreMatchers;
import org.hamcrest.MatcherAssert;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.List;
@ -67,14 +70,14 @@ public class ParserTest extends InitializedNullHandlingTest
Assert.assertEquals(expected, actual);
}
@Test
public void testParseConstants()
{
validateLiteral("null", null, null);
validateLiteral("'hello'", ExpressionType.STRING, "hello");
validateLiteral("'hello \\uD83E\\uDD18'", ExpressionType.STRING, "hello \uD83E\uDD18");
validateLiteral("1", ExpressionType.LONG, 1L);
validateLiteral("1", ExpressionType.LONG, BigInteger.valueOf(1L));
validateLiteral(String.valueOf(Long.MAX_VALUE), ExpressionType.LONG, BigInteger.valueOf(Long.MAX_VALUE));
validateLiteral("1.", ExpressionType.DOUBLE, 1.0, false);
validateLiteral("1.234", ExpressionType.DOUBLE, 1.234);
validateLiteral("1e10", ExpressionType.DOUBLE, 1.0E10, false);
@ -93,6 +96,48 @@ public class ParserTest extends InitializedNullHandlingTest
validateLiteral("NaN", ExpressionType.DOUBLE, Double.NaN);
}
@Test
public void testParseOutOfRangeLong()
{
// Two greater than Long.MAX_VALUE
final String s = "9223372036854775809";
// When not flattening, the "out of long range" error happens during eval.
final Expr expr = Parser.parse(s, ExprMacroTable.nil(), false);
final ArithmeticException e = Assert.assertThrows(
ArithmeticException.class,
() -> expr.eval(InputBindings.nilBindings())
);
MatcherAssert.assertThat(e.getMessage(), CoreMatchers.containsString("BigInteger out of long range"));
// When flattening, the "out of long range" error happens during parse, not eval.
final ArithmeticException e2 = Assert.assertThrows(
ArithmeticException.class,
() -> Parser.parse(s, ExprMacroTable.nil(), true)
);
MatcherAssert.assertThat(e2.getMessage(), CoreMatchers.containsString("BigInteger out of long range"));
}
@Test
public void testFlattenBinaryOpConstantConstant()
{
final Expr expr = Parser.parse("(2 + -3)", ExprMacroTable.nil(), true);
Assert.assertTrue(expr.isLiteral());
Assert.assertEquals(-1L, expr.getLiteralValue());
}
@Test
public void testFlattenBinaryOpIdentifierConstant()
{
final Expr expr = Parser.parse("(s + -3)", ExprMacroTable.nil(), true);
Assert.assertFalse(expr.isLiteral());
MatcherAssert.assertThat(expr, CoreMatchers.instanceOf(BinPlusExpr.class));
final Expr right = ((BinPlusExpr) expr).right;
Assert.assertTrue(right.isLiteral());
Assert.assertEquals(-3L, right.getLiteralValue());
}
@Test
public void testSimpleUnaryOps1()
{
@ -108,6 +153,7 @@ public class ParserTest extends InitializedNullHandlingTest
@Test
public void testSimpleUnaryOps2()
{
validateFlatten(String.valueOf(Long.MIN_VALUE), String.valueOf(Long.MIN_VALUE), String.valueOf(Long.MIN_VALUE));
validateFlatten("-1", "-1", "-1");
validateFlatten("--1", "--1", "1");
validateFlatten("-1+2", "(+ -1 2)", "1");
@ -735,7 +781,11 @@ public class ParserTest extends InitializedNullHandlingTest
Assert.assertEquals(type, parsed.getOutputType(emptyBinding));
Assert.assertEquals(type, parsedFlat.getOutputType(emptyBinding));
Assert.assertEquals(expected, parsed.getLiteralValue());
Assert.assertEquals(expected, parsedFlat.getLiteralValue());
Assert.assertEquals(
// Special case comparison: literal integers start life as BigIntegerExpr; converted to LongExpr later.
expected instanceof BigInteger ? ((BigInteger) expected).longValueExact() : expected,
parsedFlat.getLiteralValue()
);
if (roundTrip) {
Assert.assertEquals(expr, parsed.stringify());
Assert.assertEquals(expr, parsedFlat.stringify());

View File

@ -104,9 +104,14 @@ public class DruidExpression
return escaped.toString();
}
public static String numberLiteral(final Number n)
public static String longLiteral(final long n)
{
return n == null ? nullLiteral() : n.toString();
return String.valueOf(n);
}
public static String doubleLiteral(final double n)
{
return String.valueOf(n);
}
public static String stringLiteral(final String s)

View File

@ -275,7 +275,7 @@ public class Expressions
}
@Nullable
private static DruidExpression literalToDruidExpression(
static DruidExpression literalToDruidExpression(
final PlannerContext plannerContext,
final RexNode rexNode
)
@ -286,16 +286,27 @@ public class Expressions
final ColumnType columnType = Calcites.getColumnTypeForRelDataType(rexNode.getType());
if (RexLiteral.isNullLiteral(rexNode)) {
return DruidExpression.ofLiteral(columnType, DruidExpression.nullLiteral());
} else if (SqlTypeName.INT_TYPES.contains(sqlTypeName)) {
final Number number = (Number) RexLiteral.value(rexNode);
return DruidExpression.ofLiteral(
columnType,
number == null ? DruidExpression.nullLiteral() : DruidExpression.longLiteral(number.longValue())
);
} else if (SqlTypeName.NUMERIC_TYPES.contains(sqlTypeName)) {
return DruidExpression.ofLiteral(columnType, DruidExpression.numberLiteral((Number) RexLiteral.value(rexNode)));
// Numeric, non-INT, means we represent it as a double.
final Number number = (Number) RexLiteral.value(rexNode);
return DruidExpression.ofLiteral(
columnType,
number == null ? DruidExpression.nullLiteral() : DruidExpression.doubleLiteral(number.doubleValue())
);
} else if (SqlTypeFamily.INTERVAL_DAY_TIME == sqlTypeName.getFamily()) {
// Calcite represents DAY-TIME intervals in milliseconds.
final long milliseconds = ((Number) RexLiteral.value(rexNode)).longValue();
return DruidExpression.ofLiteral(columnType, DruidExpression.numberLiteral(milliseconds));
return DruidExpression.ofLiteral(columnType, DruidExpression.longLiteral(milliseconds));
} else if (SqlTypeFamily.INTERVAL_YEAR_MONTH == sqlTypeName.getFamily()) {
// Calcite represents YEAR-MONTH intervals in months.
final long months = ((Number) RexLiteral.value(rexNode)).longValue();
return DruidExpression.ofLiteral(columnType, DruidExpression.numberLiteral(months));
return DruidExpression.ofLiteral(columnType, DruidExpression.longLiteral(months));
} else if (SqlTypeName.STRING_TYPES.contains(sqlTypeName)) {
return DruidExpression.ofStringLiteral(RexLiteral.stringValue(rexNode));
} else if (SqlTypeName.TIMESTAMP == sqlTypeName || SqlTypeName.DATE == sqlTypeName) {
@ -304,13 +315,16 @@ public class Expressions
} else {
return DruidExpression.ofLiteral(
columnType,
DruidExpression.numberLiteral(
DruidExpression.longLiteral(
Calcites.calciteDateTimeLiteralToJoda(rexNode, plannerContext.getTimeZone()).getMillis()
)
);
}
} else if (SqlTypeName.BOOLEAN == sqlTypeName) {
return DruidExpression.ofLiteral(columnType, DruidExpression.numberLiteral(RexLiteral.booleanValue(rexNode) ? 1 : 0));
return DruidExpression.ofLiteral(
columnType,
DruidExpression.longLiteral(RexLiteral.booleanValue(rexNode) ? 1 : 0)
);
} else {
// Can't translate other literals.
return null;

View File

@ -84,8 +84,8 @@ public class SubstringOperatorConversion implements SqlOperatorConversion
expression -> StringUtils.format(
"substring(%s, %s, %s)",
expression,
DruidExpression.numberLiteral(index),
DruidExpression.numberLiteral(length)
DruidExpression.longLiteral(index),
DruidExpression.longLiteral(length)
)
);
}

View File

@ -102,7 +102,7 @@ public abstract class TimeArithmeticOperatorConversion implements SqlOperatorCon
StringUtils.format("'P%sM'", RexLiteral.value(rightRexNode)) :
StringUtils.format("concat('P', %s, 'M')", expression)
),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(direction > 0 ? 1 : -1)),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(direction > 0 ? 1 : -1)),
DruidExpression.ofStringLiteral(plannerContext.getTimeZone().getID())
)
);

View File

@ -100,9 +100,9 @@ public class TimeFloorOperatorConversion implements SqlOperatorConversion
DruidExpression.ofStringLiteral(granularity.getPeriod().toString()),
DruidExpression.ofLiteral(
ColumnType.LONG,
DruidExpression.numberLiteral(
granularity.getOrigin() == null ? null : granularity.getOrigin().getMillis()
)
granularity.getOrigin() == null
? DruidExpression.nullLiteral()
: DruidExpression.longLiteral(granularity.getOrigin().getMillis())
),
DruidExpression.ofStringLiteral(granularity.getTimeZone().toString())
)
@ -153,7 +153,7 @@ public class TimeFloorOperatorConversion implements SqlOperatorConversion
if (operand.isA(SqlKind.LITERAL)) {
return DruidExpression.ofLiteral(
Calcites.getColumnTypeForRelDataType(operand.getType()),
DruidExpression.numberLiteral(
DruidExpression.longLiteral(
Calcites.calciteDateTimeLiteralToJoda(operand, plannerContext.getTimeZone()).getMillis()
)
);

View File

@ -72,7 +72,7 @@ public class TruncateOperatorConversion implements SqlOperatorConversion
} else if (digitsExpr.isLiteral()) {
final int digits = ((Number) digitsExpr.getLiteralValue()).intValue();
final double factor = Math.pow(10, digits);
factorString = DruidExpression.numberLiteral(factor);
factorString = DruidExpression.doubleLiteral(factor);
} else {
factorString = StringUtils.format("pow(10,%s)", inputExpressions.get(1));
}

View File

@ -7373,7 +7373,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
))
.setPostAggregatorSpecs(
ImmutableList.of(
expressionPostAgg("p0", "((1 - (\"a1\" / \"a0\")) * 100)")
expressionPostAgg("p0", "((1.0 - (\"a1\" / \"a0\")) * 100)")
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
@ -13807,7 +13807,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.virtualColumns(
expressionVirtualColumn("v0", "'none'", ColumnType.STRING),
expressionVirtualColumn("v1", "0", ColumnType.LONG),
expressionVirtualColumn("v2", "0", ColumnType.DOUBLE)
expressionVirtualColumn("v2", "0.0", ColumnType.DOUBLE)
)
.dimension(
new DefaultDimensionSpec("v0", "d0")

View File

@ -0,0 +1,110 @@
/*
* 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.expression;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.Parser;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.hamcrest.CoreMatchers;
import org.hamcrest.MatcherAssert;
import org.junit.Assert;
import org.junit.Test;
public class DruidExpressionTest extends InitializedNullHandlingTest
{
@Test
public void test_doubleLiteral_asString()
{
Assert.assertEquals("0.0", DruidExpression.doubleLiteral(0));
Assert.assertEquals("-2.0", DruidExpression.doubleLiteral(-2));
Assert.assertEquals("2.0", DruidExpression.doubleLiteral(2));
Assert.assertEquals("2.1", DruidExpression.doubleLiteral(2.1));
Assert.assertEquals("2.12345678", DruidExpression.doubleLiteral(2.12345678));
Assert.assertEquals("2.2E122", DruidExpression.doubleLiteral(2.2e122));
Assert.assertEquals("NaN", DruidExpression.doubleLiteral(Double.NaN));
Assert.assertEquals("Infinity", DruidExpression.doubleLiteral(Double.POSITIVE_INFINITY));
Assert.assertEquals("-Infinity", DruidExpression.doubleLiteral(Double.NEGATIVE_INFINITY));
//CHECKSTYLE.OFF: Regexp
// Min/max double are banned by regexp due to often being inappropriate; but they are appropriate here.
Assert.assertEquals("4.9E-324", DruidExpression.doubleLiteral(Double.MIN_VALUE));
Assert.assertEquals("1.7976931348623157E308", DruidExpression.doubleLiteral(Double.MAX_VALUE));
//CHECKSTYLE.ON: Regexp
Assert.assertEquals("2.2250738585072014E-308", DruidExpression.doubleLiteral(Double.MIN_NORMAL));
}
@Test
public void test_doubleLiteral_roundTrip()
{
final double[] doubles = {
0,
-2,
2,
2.1,
2.12345678,
2.2e122,
Double.NaN,
Double.POSITIVE_INFINITY,
Double.NEGATIVE_INFINITY,
//CHECKSTYLE.OFF: Regexp
// Min/max double are banned by regexp due to often being inappropriate; but they are appropriate here.
Double.MIN_VALUE,
Double.MAX_VALUE,
//CHECKSTYLE.ON: Regexp
Double.MIN_NORMAL
};
for (double n : doubles) {
final Expr expr = Parser.parse(DruidExpression.doubleLiteral(n), ExprMacroTable.nil());
Assert.assertTrue(expr.isLiteral());
MatcherAssert.assertThat(expr.getLiteralValue(), CoreMatchers.instanceOf(Double.class));
Assert.assertEquals(n, (double) expr.getLiteralValue(), 0d);
}
}
@Test
public void test_longLiteral_asString()
{
Assert.assertEquals("0", DruidExpression.longLiteral(0));
Assert.assertEquals("-2", DruidExpression.longLiteral(-2));
Assert.assertEquals("2", DruidExpression.longLiteral(2));
Assert.assertEquals("9223372036854775807", DruidExpression.longLiteral(Long.MAX_VALUE));
Assert.assertEquals("-9223372036854775808", DruidExpression.longLiteral(Long.MIN_VALUE));
}
@Test
public void longLiteral_roundTrip()
{
final long[] longs = {
0,
-2,
2,
Long.MAX_VALUE,
Long.MIN_VALUE
};
for (long n : longs) {
final Expr expr = Parser.parse(DruidExpression.longLiteral(n), ExprMacroTable.nil());
Assert.assertTrue(expr.isLiteral());
MatcherAssert.assertThat(expr.getLiteralValue(), CoreMatchers.instanceOf(Number.class));
Assert.assertEquals(n, ((Number) expr.getLiteralValue()).longValue());
}
}
}

View File

@ -588,6 +588,18 @@ public class ExpressionsTest extends ExpressionTestBase
makeExpression("format('%s %,d',\"s\",1234,6789)"),
"foo 1,234"
);
testHelper.testExpressionString(
new StringFormatOperatorConversion().calciteOperator(),
ImmutableList.of(
testHelper.makeLiteral("%.2f %.2f %.2f"),
testHelper.makeLiteral(1234d),
testHelper.makeLiteral(123.4),
testHelper.makeLiteral(123.456789)
),
makeExpression("format('%.2f %.2f %.2f',1234.0,123.4,123.456789)"),
"1234.00 123.40 123.46"
);
}
@Test
@ -651,7 +663,7 @@ public class ExpressionsTest extends ExpressionTestBase
DruidExpression.functionCall("parse_long"),
ImmutableList.of(
DruidExpression.ofColumn(ColumnType.STRING, "hexstr"),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(16))
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(16))
)
),
239L
@ -679,7 +691,7 @@ public class ExpressionsTest extends ExpressionTestBase
DruidExpression.ofColumn(ColumnType.STRING, "hexstr")
)
),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(16))
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(16))
)
),
239L
@ -929,7 +941,7 @@ public class ExpressionsTest extends ExpressionTestBase
(args) -> "(cast(cast(" + args.get(0).getExpression() + " * 10.0,'long'),'double') / 10.0)",
ImmutableList.of(
DruidExpression.ofColumn(ColumnType.FLOAT, "x"),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(1))
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(1))
)
),
2.2
@ -946,7 +958,7 @@ public class ExpressionsTest extends ExpressionTestBase
(args) -> "(cast(cast(" + args.get(0).getExpression() + " * 10.0,'long'),'double') / 10.0)",
ImmutableList.of(
DruidExpression.ofColumn(ColumnType.FLOAT, "z"),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(1))
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(1))
)
),
-2.2
@ -963,7 +975,7 @@ public class ExpressionsTest extends ExpressionTestBase
(args) -> "(cast(cast(" + args.get(0).getExpression() + " * 0.1,'long'),'double') / 0.1)",
ImmutableList.of(
DruidExpression.ofColumn(ColumnType.LONG, "b"),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(-1))
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(-1))
)
),
20.0
@ -980,7 +992,7 @@ public class ExpressionsTest extends ExpressionTestBase
(args) -> "(cast(cast(" + args.get(0).getExpression() + " * 0.1,'long'),'double') / 0.1)",
ImmutableList.of(
DruidExpression.ofColumn(ColumnType.FLOAT, "z"),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(-1))
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(-1))
)
),
0.0
@ -1029,7 +1041,7 @@ public class ExpressionsTest extends ExpressionTestBase
DruidExpression.functionCall("round"),
ImmutableList.of(
DruidExpression.ofColumn(ColumnType.LONG, "b"),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(-1))
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(-1))
)
),
30L
@ -1059,7 +1071,7 @@ public class ExpressionsTest extends ExpressionTestBase
DruidExpression.functionCall("round"),
ImmutableList.of(
DruidExpression.ofColumn(ColumnType.FLOAT, "x"),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(1))
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(1))
)
),
2.3
@ -1769,7 +1781,7 @@ public class ExpressionsTest extends ExpressionTestBase
DruidExpression.ofColumn(ColumnType.LONG, "t"),
// RexNode type "interval year to month" is not reported as ColumnType.STRING
DruidExpression.ofLiteral(null, DruidExpression.stringLiteral("P13M")),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.numberLiteral(-1)),
DruidExpression.ofLiteral(ColumnType.LONG, DruidExpression.longLiteral(-1)),
DruidExpression.ofStringLiteral("UTC")
)
),
@ -2464,11 +2476,14 @@ public class ExpressionsTest extends ExpressionTestBase
makeExpression("human_readable_binary_byte_format(9223372036854775807)"),
"8.00 EiB"
);
/*
* NOTE: Test for Long.MIN_VALUE is skipped since ExprListnerImpl#exitLongExpr fails to parse Long.MIN_VALUE
* This cases has also been verified in the tests of underlying implementation
*/
testHelper.testExpressionString(
HumanReadableFormatOperatorConversion.BINARY_BYTE_FORMAT.calciteOperator(),
ImmutableList.of(
testHelper.makeLiteral(Long.MIN_VALUE)
),
makeExpression("human_readable_binary_byte_format(-9223372036854775808)"),
"-8.00 EiB"
);
/*
* test input with variable reference
@ -2558,10 +2573,14 @@ public class ExpressionsTest extends ExpressionTestBase
makeExpression("human_readable_decimal_byte_format(9223372036854775807)"),
"9.22 EB"
);
/*
* NOTE: Test for Long.MIN_VALUE is skipped since ExprListnerImpl#exitLongExpr fails to parse Long.MIN_VALUE
*/
testHelper.testExpressionString(
HumanReadableFormatOperatorConversion.DECIMAL_BYTE_FORMAT.calciteOperator(),
ImmutableList.of(
testHelper.makeLiteral(Long.MIN_VALUE)
),
makeExpression("human_readable_decimal_byte_format(-9223372036854775808)"),
"-9.22 EB"
);
/*
* test input with variable reference