Skip to content

Commit

Permalink
perf: Remove parsing of integer literals (#3019)
Browse files Browse the repository at this point in the history
* perf: Remove parsing of integer literals

Old code converted INT and BIGINT literals to strings and then parsed them when processing every row. This adds unnecessary overhead to the row processing and so has been replaced with just the int/long value.

Also improved error handling of numeric parsing: DOUBLEs now throw a parse exception if they result in NaN or Infinity and parse exceptions include correct location.

* Rohan's requested changes.
  • Loading branch information
big-andy-coates committed Jun 28, 2019
1 parent aae1357 commit 6195b76
Show file tree
Hide file tree
Showing 11 changed files with 239 additions and 39 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -233,16 +233,18 @@ private String formatQualifiedName(final QualifiedName name) {
}

protected Pair<String, Schema> visitLongLiteral(
final LongLiteral node, final Void context) {
return new Pair<>("Long.parseLong(\"" + node.getValue() + "\")",
Schema.OPTIONAL_INT64_SCHEMA);
final LongLiteral node,
final Void context
) {
return new Pair<>(node.getValue() + "L", Schema.OPTIONAL_INT64_SCHEMA);
}

@Override
protected Pair<String, Schema> visitIntegerLiteral(final IntegerLiteral node,
final Void context) {
return new Pair<>("Integer.parseInt(\"" + node.getValue() + "\")",
Schema.OPTIONAL_INT32_SCHEMA);
protected Pair<String, Schema> visitIntegerLiteral(
final IntegerLiteral node,
final Void context
) {
return new Pair<>(Integer.toString(node.getValue()), Schema.OPTIONAL_INT32_SCHEMA);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ public void shouldProcessArrayExpressionCorrectly() {
.process(analysis.getSelectExpressions().get(0));

assertThat(javaExpression,
equalTo("((Double) ((java.util.List)TEST1_COL4).get((int)(Integer.parseInt(\"0\"))))"));
equalTo("((Double) ((java.util.List)TEST1_COL4).get((int)(0)))"));
}

@Test
Expand Down Expand Up @@ -150,9 +150,9 @@ public void shouldPostfixFunctionInstancesWithUniqueId() {

assertThat(javaExpression, is(
"((String) CONCAT_0.evaluate("
+ "((String) SUBSTRING_1.evaluate(TEST1_COL1, Integer.parseInt(\"1\"), Integer.parseInt(\"3\"))), "
+ "((String) SUBSTRING_1.evaluate(TEST1_COL1, 1, 3)), "
+ "((String) CONCAT_2.evaluate(\"-\","
+ " ((String) SUBSTRING_3.evaluate(TEST1_COL1, Integer.parseInt(\"4\"), Integer.parseInt(\"5\")))))))"));
+ " ((String) SUBSTRING_3.evaluate(TEST1_COL1, 4, 5))))))"));
}

@Test
Expand Down Expand Up @@ -240,7 +240,7 @@ public void shouldGenerateCorrectCodeForCaseStatement() {
.process(analysis.getSelectExpressions().get(0));

// ThenL
assertThat(javaExpression, equalTo("((java.lang.String)SearchedCaseFunction.searchedCaseFunction(ImmutableList.of( SearchedCaseFunction.whenClause( new Supplier<Boolean>() { @Override public Boolean get() { return ((((Object)(TEST1_COL7)) == null || ((Object)(Integer.parseInt(\"10\"))) == null) ? false : (TEST1_COL7 < Integer.parseInt(\"10\"))); }}, new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"small\"; }}), SearchedCaseFunction.whenClause( new Supplier<Boolean>() { @Override public Boolean get() { return ((((Object)(TEST1_COL7)) == null || ((Object)(Integer.parseInt(\"100\"))) == null) ? false : (TEST1_COL7 < Integer.parseInt(\"100\"))); }}, new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"medium\"; }})), new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"large\"; }}))"));
assertThat(javaExpression, equalTo("((java.lang.String)SearchedCaseFunction.searchedCaseFunction(ImmutableList.of( SearchedCaseFunction.whenClause( new Supplier<Boolean>() { @Override public Boolean get() { return ((((Object)(TEST1_COL7)) == null || ((Object)(10)) == null) ? false : (TEST1_COL7 < 10)); }}, new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"small\"; }}), SearchedCaseFunction.whenClause( new Supplier<Boolean>() { @Override public Boolean get() { return ((((Object)(TEST1_COL7)) == null || ((Object)(100)) == null) ? false : (TEST1_COL7 < 100)); }}, new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"medium\"; }})), new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"large\"; }}))"));
}

@Test
Expand All @@ -258,7 +258,7 @@ public void shouldGenerateCorrectCodeForCaseStatementWithNoElse() {
.process(analysis.getSelectExpressions().get(0));

// ThenL
assertThat(javaExpression, equalTo("((java.lang.String)SearchedCaseFunction.searchedCaseFunction(ImmutableList.of( SearchedCaseFunction.whenClause( new Supplier<Boolean>() { @Override public Boolean get() { return ((((Object)(TEST1_COL7)) == null || ((Object)(Integer.parseInt(\"10\"))) == null) ? false : (TEST1_COL7 < Integer.parseInt(\"10\"))); }}, new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"small\"; }}), SearchedCaseFunction.whenClause( new Supplier<Boolean>() { @Override public Boolean get() { return ((((Object)(TEST1_COL7)) == null || ((Object)(Integer.parseInt(\"100\"))) == null) ? false : (TEST1_COL7 < Integer.parseInt(\"100\"))); }}, new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"medium\"; }})), new Supplier<java.lang.String>() { @Override public java.lang.String get() { return null; }}))"));
assertThat(javaExpression, equalTo("((java.lang.String)SearchedCaseFunction.searchedCaseFunction(ImmutableList.of( SearchedCaseFunction.whenClause( new Supplier<Boolean>() { @Override public Boolean get() { return ((((Object)(TEST1_COL7)) == null || ((Object)(10)) == null) ? false : (TEST1_COL7 < 10)); }}, new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"small\"; }}), SearchedCaseFunction.whenClause( new Supplier<Boolean>() { @Override public Boolean get() { return ((((Object)(TEST1_COL7)) == null || ((Object)(100)) == null) ? false : (TEST1_COL7 < 100)); }}, new Supplier<java.lang.String>() { @Override public java.lang.String get() { return \"medium\"; }})), new Supplier<java.lang.String>() { @Override public java.lang.String get() { return null; }}))"));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -382,7 +382,7 @@ public void shouldHandleNullKey() {
new StringLiteral("str"),
new IntegerLiteral(0),
new LongLiteral(2),
new DoubleLiteral("3.0"),
new DoubleLiteral(3.0),
new BooleanLiteral("TRUE"),
new StringLiteral("str"),
new StringLiteral("1.2"))
Expand Down Expand Up @@ -541,7 +541,7 @@ public void shouldFailOnDowncast() {
final ConfiguredStatement<InsertValues> statement = givenInsertValues(
ImmutableList.of("INT"),
ImmutableList.of(
new DoubleLiteral("1.1")
new DoubleLiteral(1.1)
)
);

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
{
"tests": [
{
"name": "BOOLEAN literal",
"statements": [
"CREATE STREAM INPUT (ID bigint) WITH (kafka_topic='input', value_format='DELIMITED');",
"CREATE STREAM OUTPUT AS select id, TRUE, True, true, FALSE, False, false FROM INPUT;"
],
"inputs": [
{"topic": "input", "value": "0"}
],
"outputs": [
{"topic": "OUTPUT", "value": "0,true,true,true,false,false,false"}
]
},
{
"name": "INT literal min/max",
"statements": [
"CREATE STREAM INPUT (ID bigint) WITH (kafka_topic='input', value_format='DELIMITED');",
"CREATE STREAM OUTPUT AS select id, -00002147483647 AS MIN, 000002147483647 AS MAX FROM INPUT;"
],
"inputs": [
{"topic": "input", "value": "0"}
],
"outputs": [
{"topic": "OUTPUT", "value": "0,-2147483647,2147483647"}
]
},
{
"name": "BIGINT literal min/max",
"note": "Long.MIN_VALUE is actually -9223372036854775808, which KSQL can not handle as the LongLiteral is stored unsigned",
"statements": [
"CREATE STREAM INPUT (ID bigint) WITH (kafka_topic='input', value_format='DELIMITED');",
"CREATE STREAM OUTPUT AS select id, -00009223372036854775807 AS MIN, 000009223372036854775807 AS MAX FROM INPUT;"
],
"inputs": [
{"topic": "input", "value": "0"}
],
"outputs": [
{"topic": "OUTPUT", "value": "0,-9223372036854775807,9223372036854775807"}
]
},
{
"name": "DOUBLE literal min/max",
"statements": [
"CREATE STREAM INPUT (ID bigint) WITH (kafka_topic='input', value_format='DELIMITED');",
"CREATE STREAM OUTPUT AS select id, 04.90E-324 AS MIN_VALUE, -4.9E-324 AS NEG_MIN_VALUE, 2.2250738585072014E-308 AS MIN_NORMAL, -2.2250738585072014E-308 AS NEG_MIN_NORMAL, 1.7976931348623157E308 AS MAX_VALUE, -1.7976931348623157E308 AS NEG_MAX_VALUE FROM INPUT;"
],
"inputs": [
{"topic": "input", "value": "0"}
],
"outputs": [
{"topic": "OUTPUT", "value": "0,4.9E-324,-4.9E-324,2.2250738585072014E-308,-2.2250738585072014E-308,1.7976931348623157E308,-1.7976931348623157E308"}
]
},
{
"name": "BIGINT literal positive overflow",
"statements": [
"CREATE STREAM INPUT (ID bigint) WITH (kafka_topic='input', value_format='DELIMITED');",
"CREATE STREAM OUTPUT AS select id, 9223372036854775808 FROM INPUT;"
],
"expectedException": {
"type": "io.confluent.ksql.parser.exception.ParseFailedException",
"message": "Failed to prepare statement: line 2:37: Invalid numeric literal: 9223372036854775808"
}
},
{
"name": "BIGINT literal negative overflow",
"note": "Long.MIN_VALUE is actually -9223372036854775808, which KSQL can not handle as the LongLiteral is stored unsigned",
"statements": [
"CREATE STREAM INPUT (ID bigint) WITH (kafka_topic='input', value_format='DELIMITED');",
"CREATE STREAM OUTPUT AS select id, -9223372036854775808 FROM INPUT;"
],
"expectedException": {
"type": "io.confluent.ksql.parser.exception.ParseFailedException",
"message": "Failed to prepare statement: line 2:38: Invalid numeric literal: 9223372036854775808"
}
},
{
"name": "DOUBLE literal positive overflow",
"statements": [
"CREATE STREAM INPUT (ID bigint) WITH (kafka_topic='input', value_format='DELIMITED');",
"CREATE STREAM OUTPUT AS select id, 1.7976931348623159E308 FROM INPUT;"
],
"expectedException": {
"type": "io.confluent.ksql.parser.exception.ParseFailedException",
"message": "Failed to prepare statement: line 2:37: Number overflows DOUBLE: 1.7976931348623159E308"
}
},
{
"name": "DOUBLE literal negative overflow",
"statements": [
"CREATE STREAM INPUT (ID bigint) WITH (kafka_topic='input', value_format='DELIMITED');",
"CREATE STREAM OUTPUT AS select id, -1.7976931348623159E308 FROM INPUT;"
],
"expectedException": {
"type": "io.confluent.ksql.parser.exception.ParseFailedException",
"message": "Failed to prepare statement: line 2:38: Number overflows DOUBLE: 1.7976931348623159E308"
}
}
]
}
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@
import io.confluent.ksql.parser.tree.DecimalLiteral;
import io.confluent.ksql.parser.tree.DereferenceExpression;
import io.confluent.ksql.parser.tree.DescribeFunction;
import io.confluent.ksql.parser.tree.DoubleLiteral;
import io.confluent.ksql.parser.tree.DropStream;
import io.confluent.ksql.parser.tree.DropTable;
import io.confluent.ksql.parser.tree.Explain;
Expand Down Expand Up @@ -1044,7 +1043,7 @@ public Node visitIntegerLiteral(final SqlBaseParser.IntegerLiteralContext contex

@Override
public Node visitDecimalLiteral(final SqlBaseParser.DecimalLiteralContext context) {
return new DoubleLiteral(getLocation(context), context.getText());
return ParserUtil.parseDecimalLiteral(context);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import static java.lang.String.format;

import io.confluent.ksql.parser.tree.NodeLocation;
import java.util.Optional;
import org.antlr.v4.runtime.RecognitionException;

public class ParsingException
Expand All @@ -42,6 +43,15 @@ public ParsingException(final String message, final NodeLocation nodeLocation) {
this(message, null, nodeLocation.getLineNumber(), nodeLocation.getColumnNumber());
}

public ParsingException(final String message, final Optional<NodeLocation> nodeLocation) {
this(
message,
null,
nodeLocation.map(NodeLocation::getLineNumber).orElse(1),
nodeLocation.map(NodeLocation::getColumnNumber).orElse(0)
);
}

public int getLineNumber() {
return line;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,6 @@

package io.confluent.ksql.parser.tree;

import static java.util.Objects.requireNonNull;

import com.google.errorprone.annotations.Immutable;
import java.util.Optional;

Expand All @@ -25,13 +23,13 @@ public class DoubleLiteral extends Literal {

private final double value;

public DoubleLiteral(final String value) {
public DoubleLiteral(final double value) {
this(Optional.empty(), value);
}

public DoubleLiteral(final Optional<NodeLocation> location, final String value) {
public DoubleLiteral(final Optional<NodeLocation> location, final double value) {
super(location);
this.value = Double.parseDouble(requireNonNull(value, "value"));
this.value = value;
}

@Override
Expand All @@ -55,11 +53,7 @@ public boolean equals(final Object o) {

final DoubleLiteral that = (DoubleLiteral) o;

if (Double.compare(that.value, value) != 0) {
return false;
}

return true;
return Double.compare(that.value, value) == 0;
}

@SuppressWarnings("UnaryPlus")
Expand Down
28 changes: 25 additions & 3 deletions ksql-parser/src/main/java/io/confluent/ksql/util/ParserUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

package io.confluent.ksql.util;

import static io.confluent.ksql.parser.SqlBaseParser.DecimalLiteralContext;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.toList;

Expand All @@ -24,6 +25,7 @@
import io.confluent.ksql.parser.SqlBaseParser;
import io.confluent.ksql.parser.SqlBaseParser.IntegerLiteralContext;
import io.confluent.ksql.parser.SqlBaseParser.NumberContext;
import io.confluent.ksql.parser.tree.DoubleLiteral;
import io.confluent.ksql.parser.tree.IntegerLiteral;
import io.confluent.ksql.parser.tree.Literal;
import io.confluent.ksql.parser.tree.LongLiteral;
Expand Down Expand Up @@ -108,19 +110,39 @@ public static int processIntegerNumber(final NumberContext number, final String
}

public static Literal visitIntegerLiteral(final IntegerLiteralContext context) {
final Optional<NodeLocation> location = getLocation(context);

final long valueAsLong;
try {
valueAsLong = Long.parseLong(context.getText());
} catch (final NumberFormatException e) {
throw new ParsingException("Invalid numeric literal: " + context.getText());
throw new ParsingException("Invalid numeric literal: " + context.getText(), location);
}
if (valueAsLong < 0) {
throw new RuntimeException("Unexpected negative value in literal: " + valueAsLong);
}

if (valueAsLong <= Integer.MAX_VALUE) {
return new IntegerLiteral(getLocation(context), (int) valueAsLong);
return new IntegerLiteral(location, (int) valueAsLong);
} else {
return new LongLiteral(getLocation(context), valueAsLong);
return new LongLiteral(location, valueAsLong);
}
}

public static DoubleLiteral parseDecimalLiteral(final DecimalLiteralContext context) {
final Optional<NodeLocation> location = getLocation(context);

try {
final double value = Double.parseDouble(context.getText());
if (Double.isNaN(value)) {
throw new ParsingException("Not a number: " + context.getText(), location);
}
if (Double.isInfinite(value)) {
throw new ParsingException("Number overflows DOUBLE: " + context.getText(), location);
}
return new DoubleLiteral(location, value);
} catch (final NumberFormatException e) {
throw new ParsingException("Invalid numeric literal: " + context.getText(), location);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ public void shouldFormatStringLiteral() {
public void shouldFormatSubscriptExpression() {
assertThat(ExpressionFormatter.formatExpression(new SubscriptExpression(
new StringLiteral("abc"),
new DoubleLiteral("3.0"))),
new DoubleLiteral(3.0))),
equalTo("'abc'[3.0]"));
}

Expand All @@ -84,7 +84,7 @@ public void shouldFormatLongLiteral() {

@Test
public void shouldFormatDoubleLiteral() {
assertThat(ExpressionFormatter.formatExpression(new DoubleLiteral("2.0")), equalTo("2.0"));
assertThat(ExpressionFormatter.formatExpression(new DoubleLiteral(2.0)), equalTo("2.0"));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,13 +29,13 @@ public void shouldImplementHashCodeAndEqualsProperty() {
new EqualsTester()
.addEqualityGroup(
// Note: At the moment location does not take part in equality testing
new DoubleLiteral("0.567"),
new DoubleLiteral("0.567"),
new DoubleLiteral(Optional.of(SOME_LOCATION), "0.567"),
new DoubleLiteral(Optional.of(SOME_LOCATION), "0.567")
new DoubleLiteral(0.567),
new DoubleLiteral(0.567),
new DoubleLiteral(Optional.of(SOME_LOCATION), 0.567),
new DoubleLiteral(Optional.of(SOME_LOCATION), 0.567)
)
.addEqualityGroup(
new DoubleLiteral("45.666")
new DoubleLiteral(45.666)
)
.testEquals();
}
Expand Down
Loading

0 comments on commit 6195b76

Please sign in to comment.