This is an automated email from the ASF dual-hosted git repository.
ibzib pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new 3434af0 [BEAM-12033] Create ZetaSqlException, which contains a GRPC
Status code.
new a370df2 Merge pull request #14393 from ibzib/out-of-range
3434af0 is described below
commit 3434af07a184708cf30cf79cbcbc3b982015eea5
Author: Kyle Weaver <[email protected]>
AuthorDate: Wed Mar 31 14:09:37 2021 -0700
[BEAM-12033] Create ZetaSqlException, which contains a GRPC Status code.
And change existing usages of SqlConversionException in ZetaSQL to use
ZetaSqlException instead.
Change unimplemented exception from #14332 to out of range to satisfy
compliance
tests.
---
.../extensions/sql/zetasql/BeamZetaSqlCatalog.java | 3 +-
.../extensions/sql/zetasql/ZetaSqlException.java | 37 +++++++++++++
.../zetasql/translation/ExpressionConverter.java | 60 ++++++++++++++--------
.../sql/zetasql/ZetaSqlNativeUdfTest.java | 3 +-
.../sql/zetasql/ZetaSqlNumberTypesTest.java | 7 ++-
5 files changed, 80 insertions(+), 30 deletions(-)
diff --git
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalog.java
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalog.java
index 81c6e51..a73a7ce 100644
---
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalog.java
+++
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalog.java
@@ -43,7 +43,6 @@ import java.util.stream.Collectors;
import org.apache.beam.sdk.extensions.sql.impl.JavaUdfLoader;
import org.apache.beam.sdk.extensions.sql.impl.ScalarFnReflector;
import org.apache.beam.sdk.extensions.sql.impl.ScalarFunctionImpl;
-import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
import org.apache.beam.sdk.extensions.sql.impl.utils.TVFStreamingUtils;
import org.apache.beam.sdk.extensions.sql.udf.ScalarFn;
import
org.apache.beam.sdk.extensions.sql.zetasql.translation.UserFunctionDefinitions;
@@ -382,7 +381,7 @@ public class BeamZetaSqlCatalog {
TableResolution.resolveCalciteTable(calciteSchema, tablePath);
if (calciteTable == null) {
- throw new SqlConversionException(
+ throw new ZetaSqlException(
"Wasn't able to resolve the path "
+ tablePath
+ " in schema: "
diff --git
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlException.java
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlException.java
new file mode 100644
index 0000000..7a948ef
--- /dev/null
+++
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.beam.sdk.extensions.sql.zetasql;
+
+import com.google.zetasql.io.grpc.Status;
+import com.google.zetasql.io.grpc.StatusRuntimeException;
+
+/**
+ * Exception to be thrown by the Beam ZetaSQL planner.
+ *
+ * <p>Wraps a {@link StatusRuntimeException} containing a GRPC status code.
+ */
+public class ZetaSqlException extends RuntimeException {
+
+ public ZetaSqlException(StatusRuntimeException cause) {
+ super(cause);
+ }
+
+ public ZetaSqlException(String message) {
+ this(Status.UNIMPLEMENTED.withDescription(message).asRuntimeException());
+ }
+}
diff --git
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
index d39c8e9..349b5c0 100644
---
a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
+++
b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
@@ -42,6 +42,7 @@ import
com.google.zetasql.TableValuedFunction.FixedOutputSchemaTVF;
import com.google.zetasql.Type;
import com.google.zetasql.Value;
import com.google.zetasql.ZetaSQLType.TypeKind;
+import com.google.zetasql.io.grpc.Status;
import com.google.zetasql.resolvedast.ResolvedColumn;
import com.google.zetasql.resolvedast.ResolvedNodes;
import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedAggregateScan;
@@ -66,10 +67,10 @@ import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner.QueryParameters;
-import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
import
org.apache.beam.sdk.extensions.sql.impl.ZetaSqlUserDefinedSQLNativeTableValuedFunction;
import org.apache.beam.sdk.extensions.sql.impl.utils.TVFStreamingUtils;
import
org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlCalciteTranslationUtils;
+import org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlException;
import
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.TimeUnit;
import
org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
@@ -381,7 +382,7 @@ public class ExpressionConverter {
if (functionCall.getFunction().getName().equals(FIXED_WINDOW)
|| functionCall.getFunction().getName().equals(SLIDING_WINDOW)
|| functionCall.getFunction().getName().equals(SESSION_WINDOW)) {
- throw new SqlConversionException(
+ throw new ZetaSqlException(
functionCall.getFunction().getName() + " shouldn't appear in SELECT
exprlist.");
}
@@ -719,7 +720,7 @@ public class ExpressionConverter {
private RexNode convertIntervalToRexIntervalLiteral(ResolvedLiteral
resolvedLiteral) {
if (resolvedLiteral.getType().getKind() != TYPE_STRING) {
- throw new SqlConversionException(INTERVAL_FORMAT_MSG);
+ throw new ZetaSqlException(INTERVAL_FORMAT_MSG);
}
String valStr = resolvedLiteral.getValue().getStringValue();
@@ -727,18 +728,22 @@ public class ExpressionConverter {
Arrays.stream(valStr.split(" ")).filter(s ->
!s.isEmpty()).collect(Collectors.toList());
if (stringList.size() != 3) {
- throw new SqlConversionException(INTERVAL_FORMAT_MSG);
+ throw new ZetaSqlException(INTERVAL_FORMAT_MSG);
}
if (!Ascii.toUpperCase(stringList.get(0)).equals("INTERVAL")) {
- throw new SqlConversionException(INTERVAL_FORMAT_MSG);
+ throw new ZetaSqlException(INTERVAL_FORMAT_MSG);
}
long intervalValue;
try {
intervalValue = Long.parseLong(stringList.get(1));
} catch (NumberFormatException e) {
- throw new SqlConversionException(INTERVAL_FORMAT_MSG, e);
+ throw new ZetaSqlException(
+ Status.UNIMPLEMENTED
+ .withDescription(INTERVAL_FORMAT_MSG)
+ .withCause(e)
+ .asRuntimeException());
}
String intervalDatepart = Ascii.toUpperCase(stringList.get(2));
@@ -771,7 +776,7 @@ public class ExpressionConverter {
case INTERVAL_SECOND:
return new BigDecimal(value * ONE_SECOND_IN_MILLIS);
default:
- throw new SqlConversionException(qualifier.typeName().toString());
+ throw new ZetaSqlException(qualifier.typeName().toString());
}
}
@@ -797,7 +802,7 @@ public class ExpressionConverter {
case "MILLISECOND":
return new SqlIntervalQualifier(TimeUnit.MILLISECOND, null,
SqlParserPos.ZERO);
default:
- throw new SqlConversionException(
+ throw new ZetaSqlException(
String.format(
"Received an undefined INTERVAL unit: %s. Please specify unit
from the following"
+ " list: %s.",
@@ -844,24 +849,35 @@ public class ExpressionConverter {
&& input instanceof RexLiteral) {
BigDecimal value = (BigDecimal) ((RexLiteral) input).getValue();
if
(value.compareTo(ZetaSqlCalciteTranslationUtils.ZETASQL_NUMERIC_MAX_VALUE) > 0)
{
- throw new SqlConversionException(
- String.format(
- "Casting %s as %s would cause overflow of literal %s.",
fromType, toType, value));
+ throw new ZetaSqlException(
+ Status.OUT_OF_RANGE
+ .withDescription(
+ String.format(
+ "Casting %s as %s would cause overflow of literal %s.",
+ fromType, toType, value))
+ .asRuntimeException());
}
if
(value.compareTo(ZetaSqlCalciteTranslationUtils.ZETASQL_NUMERIC_MIN_VALUE) < 0)
{
- throw new SqlConversionException(
- String.format(
- "Casting %s as %s would cause underflow of literal %s.",
fromType, toType, value));
+ throw new ZetaSqlException(
+ Status.OUT_OF_RANGE
+ .withDescription(
+ String.format(
+ "Casting %s as %s would cause underflow of literal
%s.",
+ fromType, toType, value))
+ .asRuntimeException());
}
if (value.scale() >
ZetaSqlCalciteTranslationUtils.ZETASQL_NUMERIC_SCALE) {
- throw new SqlConversionException(
- String.format(
- "Cannot cast %s as %s: scale %d exceeds %d for literal %s.",
- fromType,
- toType,
- value.scale(),
- ZetaSqlCalciteTranslationUtils.ZETASQL_NUMERIC_SCALE,
- value));
+ throw new ZetaSqlException(
+ Status.OUT_OF_RANGE
+ .withDescription(
+ String.format(
+ "Cannot cast %s as %s: scale %d exceeds %d for literal
%s.",
+ fromType,
+ toType,
+ value.scale(),
+ ZetaSqlCalciteTranslationUtils.ZETASQL_NUMERIC_SCALE,
+ value))
+ .asRuntimeException());
}
}
}
diff --git
a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNativeUdfTest.java
b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNativeUdfTest.java
index d1fecc2..de7e856 100644
---
a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNativeUdfTest.java
+++
b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNativeUdfTest.java
@@ -20,7 +20,6 @@ package org.apache.beam.sdk.extensions.sql.zetasql;
import static org.hamcrest.Matchers.isA;
import com.google.zetasql.SqlException;
-import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
import org.apache.beam.sdk.schemas.Schema;
@@ -215,7 +214,7 @@ public class ZetaSqlNativeUdfTest extends ZetaSqlTestBase {
+ " SELECT key FROM CustomerRange(10, 14)";
ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
- thrown.expect(SqlConversionException.class);
+ thrown.expect(ZetaSqlException.class);
thrown.expectMessage("Wasn't able to resolve the path [TableNotExist] in
schema: beam");
zetaSQLQueryPlanner.convertToBeamRel(sql);
}
diff --git
a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNumberTypesTest.java
b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNumberTypesTest.java
index c1a97a2..314a4fa 100644
---
a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNumberTypesTest.java
+++
b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNumberTypesTest.java
@@ -18,7 +18,6 @@
package org.apache.beam.sdk.extensions.sql.zetasql;
import com.google.zetasql.Value;
-import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
import org.apache.beam.sdk.testing.TestPipeline;
import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
@@ -45,7 +44,7 @@ public class ZetaSqlNumberTypesTest extends ZetaSqlTestBase {
double val = 1.7976931348623157e+308;
String sql = "SELECT CAST(@p0 AS NUMERIC) AS ColA";
- thrown.expect(SqlConversionException.class);
+ thrown.expect(ZetaSqlException.class);
thrown.expectMessage("Casting TYPE_DOUBLE as TYPE_NUMERIC would cause
overflow of literal");
ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
@@ -59,7 +58,7 @@ public class ZetaSqlNumberTypesTest extends ZetaSqlTestBase {
double val = -1.7976931348623157e+308;
String sql = "SELECT CAST(@p0 AS NUMERIC) AS ColA";
- thrown.expect(SqlConversionException.class);
+ thrown.expect(ZetaSqlException.class);
thrown.expectMessage("Casting TYPE_DOUBLE as TYPE_NUMERIC would cause
underflow of literal");
ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
@@ -73,7 +72,7 @@ public class ZetaSqlNumberTypesTest extends ZetaSqlTestBase {
double val = 2.2250738585072014e-308;
String sql = "SELECT CAST(@p0 AS NUMERIC) AS ColA";
- thrown.expect(SqlConversionException.class);
+ thrown.expect(ZetaSqlException.class);
thrown.expectMessage("Cannot cast TYPE_DOUBLE as TYPE_NUMERIC: scale 1022
exceeds 9");
ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);