diff --git a/.github/trigger_files/beam_PostCommit_XVR_Direct.json b/.github/trigger_files/beam_PostCommit_XVR_Direct.json index bcb86e6ab5e7..8e1c80dd873d 100644 --- a/.github/trigger_files/beam_PostCommit_XVR_Direct.json +++ b/.github/trigger_files/beam_PostCommit_XVR_Direct.json @@ -1,4 +1,4 @@ { "https://github.com/apache/beam/pull/32648": "testing Flink 1.19 support", - "modification": 2 + "modification": 3 } diff --git a/.github/trigger_files/beam_PostCommit_XVR_Flink.json b/.github/trigger_files/beam_PostCommit_XVR_Flink.json index bb1b9f4c25e9..702328d16d4b 100644 --- a/.github/trigger_files/beam_PostCommit_XVR_Flink.json +++ b/.github/trigger_files/beam_PostCommit_XVR_Flink.json @@ -1,4 +1,3 @@ { - "https://github.com/apache/beam/pull/32440": "testing datastream optimizations", - "https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support" + "modification": 1 } diff --git a/CHANGES.md b/CHANGES.md index 758c11c83926..a76e55dcd73c 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -77,6 +77,7 @@ ## Breaking Changes * X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). +* Previously deprecated Beam ZetaSQL component has been removed from new releases ([#34423](https://github.com/apache/beam/issues/34423)). ## Deprecations diff --git a/build.gradle.kts b/build.gradle.kts index d1f0740e7c02..316ac4072fa6 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -385,7 +385,6 @@ tasks.register("sqlPreCommit") { dependsOn(":sdks:java:extensions:sql:perf-tests:build") dependsOn(":sdks:java:extensions:sql:udf-test-provider:build") dependsOn(":sdks:java:extensions:sql:udf:build") - dependsOn(":sdks:java:extensions:sql:zetasql:build") } tasks.register("javaPreCommitPortabilityApi") { diff --git a/sdks/go/examples/xlang/sql/sql.go b/sdks/go/examples/xlang/sql/sql.go index dcf293c7d915..eb65b4003e95 100644 --- a/sdks/go/examples/xlang/sql/sql.go +++ b/sdks/go/examples/xlang/sql/sql.go @@ -98,7 +98,7 @@ func main() { // Options for the sql transform have to be defined before the Transform call. var opts []sql.Option - // Dialect options are "calcite" and "zetasql" + // Dialect options is always "calcite" opts = append(opts, sql.Dialect("calcite")) // The expansion address can be specified per-call here or overwritten for all // calls using xlangx.RegisterOverrideForUrn(sqlx.Urn, *expansionAddr). diff --git a/sdks/go/pkg/beam/transforms/sql/sql.go b/sdks/go/pkg/beam/transforms/sql/sql.go index 878b1c12f9cd..ad4188821102 100644 --- a/sdks/go/pkg/beam/transforms/sql/sql.go +++ b/sdks/go/pkg/beam/transforms/sql/sql.go @@ -65,7 +65,7 @@ func OutputType(t reflect.Type, components ...typex.FullType) Option { } } -// Dialect specifies the SQL dialect, e.g. use 'zetasql' for ZetaSQL. +// Dialect specifies the SQL dialect. It is always Calcite func Dialect(dialect string) Option { return func(o sqlx.Options) { o.(*options).dialect = dialect diff --git a/sdks/go/pkg/beam/transforms/sql/sql_test.go b/sdks/go/pkg/beam/transforms/sql/sql_test.go index 851495015fe8..cbe0f8c49f41 100644 --- a/sdks/go/pkg/beam/transforms/sql/sql_test.go +++ b/sdks/go/pkg/beam/transforms/sql/sql_test.go @@ -190,7 +190,6 @@ func TestMultipleOptions(t *testing.T) { { name: "all_options", inputName: "test", - dialect: "zetasql", expansionAddr: "localhost:8080", typ: reflect.TypeOf(int64(0)), customOpt: sqlx.Option{Urn: "test"}, diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml index af384ff19c09..e8d4e8888da1 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle/suppressions.xml @@ -43,9 +43,6 @@ - - - @@ -69,13 +66,6 @@ - - - - - - - diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SqlTypes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SqlTypes.java index 8685e625542e..c8af8d03333e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SqlTypes.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/SqlTypes.java @@ -29,15 +29,15 @@ public class SqlTypes { private SqlTypes() {} - /** Beam LogicalType corresponding to ZetaSQL/CalciteSQL DATE type. */ + /** Beam LogicalType corresponding to CalciteSQL DATE type. */ public static final LogicalType DATE = new Date(); - /** Beam LogicalType corresponding to ZetaSQL/CalciteSQL TIME type. */ + /** Beam LogicalType corresponding to CalciteSQL TIME type. */ public static final LogicalType TIME = new Time(); - /** Beam LogicalType corresponding to ZetaSQL DATETIME type. */ + /** Beam LogicalType corresponding to DATETIME type. */ public static final LogicalType DATETIME = new DateTime(); - /** Beam LogicalType corresponding to ZetaSQL TIMESTAMP type. */ + /** Beam LogicalType corresponding to TIMESTAMP type. */ public static final LogicalType TIMESTAMP = new MicrosInstant(); } diff --git a/sdks/java/extensions/sql/datacatalog/build.gradle b/sdks/java/extensions/sql/datacatalog/build.gradle index cb557cc80776..db4a92001501 100644 --- a/sdks/java/extensions/sql/datacatalog/build.gradle +++ b/sdks/java/extensions/sql/datacatalog/build.gradle @@ -46,7 +46,6 @@ dependencies { implementation library.java.slf4j_api testImplementation project(":sdks:java:extensions:sql") - testImplementation project(":sdks:java:extensions:sql:zetasql") testImplementation project(":runners:direct-java") testImplementation project(":sdks:java:io:google-cloud-platform") testImplementation library.java.google_api_services_bigquery diff --git a/sdks/java/extensions/sql/datacatalog/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogBigQueryIT.java b/sdks/java/extensions/sql/datacatalog/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogBigQueryIT.java index b86414db3209..cdfc8ce16316 100644 --- a/sdks/java/extensions/sql/datacatalog/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogBigQueryIT.java +++ b/sdks/java/extensions/sql/datacatalog/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogBigQueryIT.java @@ -26,7 +26,6 @@ import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions; import org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner; import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner; -import org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; @@ -59,11 +58,7 @@ public static class DialectSensitiveTests { /** Parameterized by which SQL dialect, since the syntax here is the same. */ @Parameterized.Parameters(name = "{0}") public static Iterable dialects() { - return Arrays.asList( - new Object[][] { - {"ZetaSQL", ZetaSQLQueryPlanner.class}, - {"CalciteSQL", CalciteQueryPlanner.class} - }); + return Arrays.asList(new Object[][] {{"CalciteSQL", CalciteQueryPlanner.class}}); } @SuppressWarnings("initialization.fields.uninitialized") diff --git a/sdks/java/extensions/sql/expansion-service/build.gradle b/sdks/java/extensions/sql/expansion-service/build.gradle index efa8b8650dcd..8b5bd8c69240 100644 --- a/sdks/java/extensions/sql/expansion-service/build.gradle +++ b/sdks/java/extensions/sql/expansion-service/build.gradle @@ -42,7 +42,6 @@ dependencies { implementation project(path: ":sdks:java:expansion-service") permitUnusedDeclared project(path: ":sdks:java:expansion-service") // BEAM-11761 implementation project(path: ":sdks:java:extensions:sql") - implementation project(path: ":sdks:java:extensions:sql:zetasql") implementation library.java.vendored_guava_32_1_2_jre } diff --git a/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/ExternalSqlTransformRegistrar.java b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/ExternalSqlTransformRegistrar.java index d7a9bb288969..19c34668b897 100644 --- a/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/ExternalSqlTransformRegistrar.java +++ b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/ExternalSqlTransformRegistrar.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.extensions.sql.SqlTransform; import org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner; import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner; -import org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner; import org.apache.beam.sdk.transforms.ExternalTransformBuilder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PCollection; @@ -37,7 +36,6 @@ public class ExternalSqlTransformRegistrar implements ExternalTransformRegistrar private static final String URN = "beam:external:java:sql:v1"; private static final ImmutableMap> DIALECTS = ImmutableMap.>builder() - .put("zetasql", ZetaSQLQueryPlanner.class) .put("calcite", CalciteQueryPlanner.class) .build(); diff --git a/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/SqlTransformSchemaTransformProvider.java b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/SqlTransformSchemaTransformProvider.java index f032da0799d8..3ac890a7370d 100644 --- a/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/SqlTransformSchemaTransformProvider.java +++ b/sdks/java/extensions/sql/expansion-service/src/main/java/org/apache/beam/sdk/extensions/sql/expansion/SqlTransformSchemaTransformProvider.java @@ -50,9 +50,7 @@ public class SqlTransformSchemaTransformProvider implements SchemaTransformProvider { private static final Map> QUERY_PLANNERS = - ImmutableMap.of( - "zetasql", org.apache.beam.sdk.extensions.sql.zetasql.ZetaSQLQueryPlanner.class, - "calcite", org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner.class); + ImmutableMap.of("calcite", org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner.class); private static final EnumerationType QUERY_ENUMERATION = EnumerationType.create(QUERY_PLANNERS.keySet().stream().collect(Collectors.toList())); diff --git a/sdks/java/extensions/sql/zetasql/build.gradle b/sdks/java/extensions/sql/zetasql/build.gradle deleted file mode 100644 index 29a3f95402b0..000000000000 --- a/sdks/java/extensions/sql/zetasql/build.gradle +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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. - */ - -plugins { - id 'org.apache.beam.module' -} - -applyJavaNature( - automaticModuleName: 'org.apache.beam.sdk.extensions.sql.zetasql', -) - -description = "Apache Beam :: SDKs :: Java :: Extensions :: SQL :: ZetaSQL" -ext.summary = "ZetaSQL to Calcite translator" - -def zetasql_version = "2024.11.1" - -dependencies { - // TODO(https://github.com/apache/beam/issues/21156): Determine how to build without this dependency - provided "org.immutables:value:2.8.8" - permitUnusedDeclared "org.immutables:value:2.8.8" - implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) - permitUnusedDeclared enforcedPlatform(library.java.google_cloud_platform_libraries_bom) - implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(":sdks:java:extensions:sql") - implementation project(":sdks:java:extensions:sql:udf") - implementation library.java.vendored_calcite_1_28_0 - implementation library.java.guava - implementation library.java.grpc_api - implementation library.java.joda_time - implementation library.java.protobuf_java - implementation library.java.protobuf_java_util - permitUnusedDeclared library.java.protobuf_java_util // BEAM-11761 - implementation library.java.slf4j_api - implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.proto_google_common_protos // Interfaces with ZetaSQL use this - permitUnusedDeclared library.java.proto_google_common_protos // BEAM-11761 - implementation library.java.grpc_google_common_protos // Interfaces with ZetaSQL use this - permitUnusedDeclared library.java.grpc_google_common_protos // BEAM-11761 - implementation "com.google.zetasql:zetasql-client:$zetasql_version" - implementation "com.google.zetasql:zetasql-types:$zetasql_version" - implementation "com.google.zetasql:zetasql-jni-channel:$zetasql_version" - permitUnusedDeclared "com.google.zetasql:zetasql-jni-channel:$zetasql_version" // BEAM-11761 - testImplementation library.java.vendored_calcite_1_28_0 - testImplementation library.java.vendored_guava_32_1_2_jre - testImplementation library.java.junit - testImplementation library.java.hamcrest - testImplementation library.java.mockito_core - testImplementation library.java.quickcheck_core - testImplementation library.java.jackson_databind - testImplementation "org.codehaus.janino:janino:3.0.11" - testCompileOnly project(":sdks:java:extensions:sql:udf-test-provider") - testRuntimeOnly library.java.slf4j_jdk14 -} - -test { - dependsOn ":sdks:java:extensions:sql:emptyJar" - // Pass jars used by Java UDF tests via system properties. - systemProperty "beam.sql.udf.test.jar_path", project(":sdks:java:extensions:sql:udf-test-provider").jarPath - systemProperty "beam.sql.udf.test.empty_jar_path", project(":sdks:java:extensions:sql").emptyJar.archivePath -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamCalcRelType.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamCalcRelType.java deleted file mode 100644 index 70e583d677a2..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamCalcRelType.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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 org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel; -import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; -import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter; -import org.apache.beam.sdk.extensions.sql.zetasql.translation.ZetaSqlScalarFunctionImpl; -import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.RexImpTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexDynamicParam; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexFieldAccess; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLocalRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexProgram; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** {@link CalcRelSplitter.RelType} for {@link BeamCalcRel}. */ -class BeamCalcRelType extends CalcRelSplitter.RelType { - private static final Logger LOG = LoggerFactory.getLogger(BeamCalcRelType.class); - - BeamCalcRelType(String name) { - super(name); - } - - @Override - protected boolean canImplement(RexFieldAccess field) { - return supportsType(field.getType()); - } - - @Override - protected boolean canImplement(RexLiteral literal) { - return supportsType(literal.getType()); - } - - @Override - protected boolean canImplement(RexDynamicParam param) { - return supportsType(param.getType()); - } - - @Override - protected boolean canImplement(RexCall call) { - final SqlOperator operator = call.getOperator(); - - RexImpTable.RexCallImplementor implementor = RexImpTable.INSTANCE.get(operator); - if (implementor == null) { - // Reject methods with no implementation - return false; - } - - if (operator instanceof SqlUserDefinedFunction) { - SqlUserDefinedFunction udf = (SqlUserDefinedFunction) call.op; - if (udf.function instanceof ZetaSqlScalarFunctionImpl) { - ZetaSqlScalarFunctionImpl scalarFunction = (ZetaSqlScalarFunctionImpl) udf.function; - if (!scalarFunction.functionGroup.equals( - BeamZetaSqlCatalog.USER_DEFINED_JAVA_SCALAR_FUNCTIONS)) { - // Reject ZetaSQL Builtin Scalar Functions - return false; - } - for (RexNode operand : call.getOperands()) { - if (operand instanceof RexLocalRef) { - if (!supportsType(operand.getType())) { - LOG.error( - "User-defined function {} received unsupported operand type {}.", - call.op.getName(), - ((RexLocalRef) operand).getType()); - return false; - } - } else { - LOG.error( - "User-defined function {} received unrecognized operand kind {}.", - call.op.getName(), - operand.getKind()); - return false; - } - } - } else { - // Reject other UDFs - return false; - } - } else { - // Reject Calcite implementations - return false; - } - return true; - } - - @Override - protected RelNode makeRel( - RelOptCluster cluster, - RelTraitSet traitSet, - RelBuilder relBuilder, - RelNode input, - RexProgram program) { - RexProgram normalizedProgram = program.normalize(cluster.getRexBuilder(), false); - return new BeamCalcRel( - cluster, - traitSet.replace(BeamLogicalConvention.INSTANCE), - RelOptRule.convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)), - normalizedProgram); - } - - /** - * Returns true only if the data type can be correctly implemented by {@link - * org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel} in ZetaSQL. - */ - private boolean supportsType(RelDataType type) { - switch (type.getSqlTypeName()) { - case BIGINT: - case BINARY: - case BOOLEAN: - case CHAR: - case DATE: - case DECIMAL: - case DOUBLE: - case NULL: - case TIMESTAMP: - case VARBINARY: - case VARCHAR: - return true; - case ARRAY: - return supportsType( - Preconditions.checkArgumentNotNull( - type.getComponentType(), "Encountered ARRAY type with no component type.")); - case ROW: - return type.getFieldList().stream().allMatch((field) -> supportsType(field.getType())); - case TIME: // BEAM-12086 - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: // BEAM-12087 - default: - return false; - } - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRule.java deleted file mode 100644 index 84ee347c3d00..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRule.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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 org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel; -import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter; -import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcRule; -import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcSplittingRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; - -/** - * A {@link BeamCalcSplittingRule} to replace {@link Calc} with {@link BeamCalcRel}. - * - *

Equivalent to {@link BeamCalcRule} but with limits to supported types and operators. - * - *

This class is intended only for testing purposes. See {@link BeamZetaSqlCalcSplittingRule}. - */ -public class BeamJavaUdfCalcRule extends BeamCalcSplittingRule { - public static final BeamJavaUdfCalcRule INSTANCE = new BeamJavaUdfCalcRule(); - - private BeamJavaUdfCalcRule() { - super("BeamJavaUdfCalcRule"); - } - - @Override - protected CalcRelSplitter.RelType[] getRelTypes() { - // "Split" the Calc between two identical RelTypes. The second one is just a placeholder; if the - // first isn't usable, the second one won't be usable either, and the planner will fail. - return new CalcRelSplitter.RelType[] { - new BeamCalcRelType("BeamCalcRelType"), new BeamCalcRelType("BeamCalcRelType2") - }; - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcMergeRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcMergeRule.java deleted file mode 100644 index 018cd541d9b8..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcMergeRule.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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 org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleOperand; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CalcMergeRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.CoreRules; - -/** - * Planner rule to merge a {@link BeamZetaSqlCalcRel} with a {@link BeamZetaSqlCalcRel}. Subset of - * {@link CalcMergeRule}. - */ -public class BeamZetaSqlCalcMergeRule extends RelOptRule { - public static final BeamZetaSqlCalcMergeRule INSTANCE = new BeamZetaSqlCalcMergeRule(); - - public BeamZetaSqlCalcMergeRule() { - super( - operand( - BeamZetaSqlCalcRel.class, - operand(BeamZetaSqlCalcRel.class, any()), - new RelOptRuleOperand[0])); - } - - @Override - public void onMatch(RelOptRuleCall call) { - CoreRules.CALC_MERGE.onMatch(call); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java deleted file mode 100644 index d60ebe46b370..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java +++ /dev/null @@ -1,404 +0,0 @@ -/* - * 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 static org.apache.beam.sdk.schemas.Schema.Field; -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; - -import com.google.auto.value.AutoValue; -import com.google.zetasql.AnalyzerOptions; -import com.google.zetasql.PreparedExpression; -import com.google.zetasql.Value; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.util.ArrayDeque; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.function.IntFunction; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions; -import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner.QueryParameters; -import org.apache.beam.sdk.extensions.sql.impl.rel.AbstractBeamCalcRel; -import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; -import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; -import org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BeamBigQuerySqlDialect; -import org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BeamSqlUnparseContext; -import org.apache.beam.sdk.schemas.FieldAccessDescriptor; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.utils.SelectHelpers; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.POutput; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexProgram; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlDialect; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Duration; -import org.joda.time.Instant; - -/** - * BeamRelNode to replace {@code Project} and {@code Filter} node based on the {@code ZetaSQL} - * expression evaluator. - */ -@SuppressWarnings( - "unused") // TODO(https://github.com/apache/beam/issues/21230): Remove when new version of -// errorprone is released (2.11.0) -@Internal -public class BeamZetaSqlCalcRel extends AbstractBeamCalcRel { - - private static final SqlDialect DIALECT = BeamBigQuerySqlDialect.DEFAULT; - private static final int MAX_PENDING_WINDOW = 32; - private final BeamSqlUnparseContext context; - - private static final TupleTag rows = new TupleTag("output") {}; - private static final TupleTag errors = new TupleTag("errors") {}; - - private static String columnName(int i) { - return "_" + i; - } - - public BeamZetaSqlCalcRel( - RelOptCluster cluster, RelTraitSet traits, RelNode input, RexProgram program) { - super(cluster, traits, input, program); - final IntFunction fn = i -> new SqlIdentifier(columnName(i), SqlParserPos.ZERO); - context = new BeamSqlUnparseContext(fn); - } - - @Override - public Calc copy(RelTraitSet traitSet, RelNode input, RexProgram program) { - return new BeamZetaSqlCalcRel(getCluster(), traitSet, input, program); - } - - @Override - public PTransform, PCollection> buildPTransform() { - return buildPTransform(null); - } - - @Override - public PTransform, PCollection> buildPTransform( - @Nullable PTransform, ? extends POutput> errorsTransformer) { - return new Transform(errorsTransformer); - } - - @AutoValue - abstract static class TimestampedFuture { - private static TimestampedFuture create(Instant t, Future f, Row r) { - return new AutoValue_BeamZetaSqlCalcRel_TimestampedFuture(t, f, r); - } - - abstract Instant timestamp(); - - abstract Future future(); - - abstract Row row(); - } - - private class Transform extends PTransform, PCollection> { - - private final @Nullable PTransform, ? extends POutput> errorsTransformer; - - Transform(@Nullable PTransform, ? extends POutput> errorsTransformer) { - this.errorsTransformer = errorsTransformer; - } - - @Override - public PCollection expand(PCollectionList pinput) { - Preconditions.checkArgument( - pinput.size() == 1, - "%s expected a single input PCollection, but received %d.", - BeamZetaSqlCalcRel.class.getSimpleName(), - pinput.size()); - PCollection upstream = pinput.get(0); - - final RexBuilder rexBuilder = getCluster().getRexBuilder(); - RexNode rex = rexBuilder.makeCall(SqlStdOperatorTable.ROW, getProgram().getProjectList()); - - final RexNode condition = getProgram().getCondition(); - if (condition != null) { - rex = - rexBuilder.makeCall( - SqlStdOperatorTable.CASE, condition, rex, rexBuilder.makeNullLiteral(getRowType())); - } - - final Schema outputSchema = CalciteUtils.toSchema(getRowType()); - - BeamSqlPipelineOptions options = - pinput.getPipeline().getOptions().as(BeamSqlPipelineOptions.class); - CalcFn calcFn = - new CalcFn( - context.toSql(getProgram(), rex).toSqlString(DIALECT).getSql(), - createNullParams(context.getNullParams()), - upstream.getSchema(), - outputSchema, - options.getZetaSqlDefaultTimezone(), - options.getVerifyRowValues(), - errorsTransformer != null); - - PCollectionTuple tuple = - upstream.apply(ParDo.of(calcFn).withOutputTags(rows, TupleTagList.of(errors))); - tuple.get(errors).setRowSchema(calcFn.errorsSchema); - - if (errorsTransformer != null) { - tuple.get(errors).apply(errorsTransformer); - } - - return tuple.get(rows).setRowSchema(outputSchema); - } - } - - private static Map createNullParams(Map input) { - Map result = new HashMap<>(); - for (Map.Entry entry : input.entrySet()) { - result.put( - entry.getKey(), - Value.createNullValue(ZetaSqlCalciteTranslationUtils.toZetaSqlType(entry.getValue()))); - } - return result; - } - - /** - * {@code CalcFn} is the executor for a {@link BeamZetaSqlCalcRel} step. The implementation is - * based on the {@code ZetaSQL} expression evaluator. - */ - @SuppressFBWarnings("SE_TRANSIENT_FIELD_NOT_RESTORED") - private static class CalcFn extends DoFn { - private final String sql; - private final Map nullParams; - private final Schema inputSchema; - private final Schema outputSchema; - private final String defaultTimezone; - private final boolean verifyRowValues; - private final boolean dlqTransformDownstream; - - final Schema errorsSchema; - private final List referencedColumns; - - @FieldAccess("row") - private final FieldAccessDescriptor fieldAccess; - - private transient Map> pending = new HashMap<>(); - private transient PreparedExpression exp; - private transient PreparedExpression.@Nullable Stream stream; - - CalcFn( - String sql, - Map nullParams, - Schema inputSchema, - Schema outputSchema, - String defaultTimezone, - boolean verifyRowValues, - boolean dlqTransformDownstream) { - this.sql = sql; - this.exp = new PreparedExpression(sql); - this.nullParams = nullParams; - this.inputSchema = inputSchema; - this.outputSchema = outputSchema; - this.defaultTimezone = defaultTimezone; - this.verifyRowValues = verifyRowValues; - this.dlqTransformDownstream = dlqTransformDownstream; - - try (PreparedExpression exp = - prepareExpression(sql, nullParams, inputSchema, defaultTimezone)) { - ImmutableList.Builder columns = new ImmutableList.Builder<>(); - for (String c : exp.getReferencedColumns()) { - columns.add(Integer.parseInt(c.substring(1))); - } - this.referencedColumns = columns.build(); - this.fieldAccess = FieldAccessDescriptor.withFieldIds(this.referencedColumns); - Schema inputRowSchema = SelectHelpers.getOutputSchema(inputSchema, fieldAccess); - this.errorsSchema = BeamSqlRelUtils.getErrorRowSchema(inputRowSchema); - } - } - - /** exp cannot be reused and is transient so needs to be reinitialized. */ - private static PreparedExpression prepareExpression( - String sql, Map nullParams, Schema inputSchema, String defaultTimezone) { - AnalyzerOptions options = - SqlAnalyzer.getAnalyzerOptions(QueryParameters.ofNamed(nullParams), defaultTimezone); - for (int i = 0; i < inputSchema.getFieldCount(); i++) { - options.addExpressionColumn( - columnName(i), - ZetaSqlBeamTranslationUtils.toZetaSqlType(inputSchema.getField(i).getType())); - } - - PreparedExpression exp = new PreparedExpression(sql); - exp.prepare(options); - return exp; - } - - @Setup - public void setup() { - this.exp = prepareExpression(sql, nullParams, inputSchema, defaultTimezone); - this.stream = exp.stream(); - } - - @StartBundle - public void startBundle() { - pending = new HashMap<>(); - } - - @Override - public Duration getAllowedTimestampSkew() { - return Duration.millis(Long.MAX_VALUE); - } - - @ProcessElement - public void processElement( - @FieldAccess("row") Row row, - @Timestamp Instant t, - BoundedWindow w, - OutputReceiver r, - MultiOutputReceiver multiOutputReceiver) - throws InterruptedException { - - @Nullable Queue pendingWindow = pending.get(w); - if (pendingWindow == null) { - pendingWindow = new ArrayDeque<>(); - pending.put(w, pendingWindow); - } - try { - Map columns = new HashMap<>(); - for (int i : referencedColumns) { - final Field field = inputSchema.getField(i); - columns.put( - columnName(i), - ZetaSqlBeamTranslationUtils.toZetaSqlValue( - row.getBaseValue(field.getName(), Object.class), field.getType())); - } - Future valueFuture = checkArgumentNotNull(stream).execute(columns, nullParams); - pendingWindow.add(TimestampedFuture.create(t, valueFuture, row)); - - } catch (UnsupportedOperationException | ArithmeticException | IllegalArgumentException e) { - if (!dlqTransformDownstream) { - throw e; - } - multiOutputReceiver - .get(errors) - .output(Row.withSchema(errorsSchema).addValues(row, e.toString()).build()); - } - - while ((!pendingWindow.isEmpty() && pendingWindow.element().future().isDone()) - || pendingWindow.size() > MAX_PENDING_WINDOW) { - outputRow(pendingWindow.remove(), r, multiOutputReceiver.get(errors)); - } - } - - @FinishBundle - public void finishBundle(FinishBundleContext c) throws InterruptedException { - checkArgumentNotNull(stream).flush(); - for (Map.Entry> pendingWindow : pending.entrySet()) { - OutputReceiver rowOutputReciever = - new OutputReceiverForFinishBundle(c, pendingWindow.getKey(), rows); - OutputReceiver errorOutputReciever = - new OutputReceiverForFinishBundle(c, pendingWindow.getKey(), errors); - - for (TimestampedFuture timestampedFuture : pendingWindow.getValue()) { - outputRow(timestampedFuture, rowOutputReciever, errorOutputReciever); - } - } - } - - // TODO(https://github.com/apache/beam/issues/18203): Remove this when FinishBundle has added - // support for an {@link OutputReceiver} - private static class OutputReceiverForFinishBundle implements OutputReceiver { - - private final FinishBundleContext c; - private final BoundedWindow w; - - private final TupleTag tag; - - private OutputReceiverForFinishBundle( - FinishBundleContext c, BoundedWindow w, TupleTag tag) { - this.c = c; - this.w = w; - this.tag = tag; - } - - @Override - public void output(Row output) { - throw new RuntimeException("Unsupported"); - } - - @Override - public void outputWithTimestamp(Row output, Instant timestamp) { - c.output(tag, output, timestamp, w); - } - } - - private static RuntimeException extractException(Throwable e) { - try { - throw checkArgumentNotNull(e.getCause()); - } catch (RuntimeException r) { - return r; - } catch (Throwable t) { - return new RuntimeException(t); - } - } - - private void outputRow( - TimestampedFuture c, OutputReceiver r, OutputReceiver errorOutputReceiver) - throws InterruptedException { - final Value v; - try { - v = c.future().get(); - } catch (ExecutionException e) { - if (!dlqTransformDownstream) { - throw extractException(e); - } - errorOutputReceiver.outputWithTimestamp( - Row.withSchema(errorsSchema).addValues(c.row(), e.toString()).build(), c.timestamp()); - return; - } catch (Throwable thr) { - throw extractException(thr); - } - if (!v.isNull()) { - Row row = ZetaSqlBeamTranslationUtils.toBeamRow(v, outputSchema, verifyRowValues); - r.outputWithTimestamp(row, c.timestamp()); - } - } - - @Teardown - public void teardown() { - checkArgumentNotNull(stream).close(); - exp.close(); - } - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java deleted file mode 100644 index a5957a9107b1..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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 org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter; -import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcSplittingRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Calc; - -/** A {@link BeamCalcSplittingRule} to replace {@link Calc} with {@link BeamZetaSqlCalcRel}. */ -public class BeamZetaSqlCalcRule extends BeamCalcSplittingRule { - public static final BeamZetaSqlCalcRule INSTANCE = new BeamZetaSqlCalcRule(); - - private BeamZetaSqlCalcRule() { - super("BeamZetaSqlCalcRule"); - } - - @Override - protected CalcRelSplitter.RelType[] getRelTypes() { - // "Split" the Calc between two identical RelTypes. The second one is just a placeholder; if the - // first isn't usable, the second one won't be usable either, and the planner will fail. - return new CalcRelSplitter.RelType[] { - new BeamZetaSqlRelType("BeamZetaSqlRelType"), new BeamZetaSqlRelType("BeamZetaSqlRelType2") - }; - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcSplittingRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcSplittingRule.java deleted file mode 100644 index 3b9bb385fbe7..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcSplittingRule.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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 org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel; -import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter; -import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcSplittingRule; - -/** - * A {@link BeamCalcSplittingRule} that converts a {@link LogicalCalc} to a chain of {@link - * BeamZetaSqlCalcRel} and/or {@link BeamCalcRel} via {@link CalcRelSplitter}. - * - *

Only Java UDFs are implemented using {@link BeamCalcRel}. All other expressions are - * implemented using {@link BeamZetaSqlCalcRel}. - */ -public class BeamZetaSqlCalcSplittingRule extends BeamCalcSplittingRule { - public static final BeamZetaSqlCalcSplittingRule INSTANCE = new BeamZetaSqlCalcSplittingRule(); - - private BeamZetaSqlCalcSplittingRule() { - super("BeamZetaSqlCalcRule"); - } - - @Override - protected CalcRelSplitter.RelType[] getRelTypes() { - return new CalcRelSplitter.RelType[] { - new BeamZetaSqlRelType("BeamZetaSqlRelType"), new BeamCalcRelType("BeamCalcRelType") - }; - } -} 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 deleted file mode 100644 index 719abe4041bc..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalog.java +++ /dev/null @@ -1,593 +0,0 @@ -/* - * 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.common.collect.ImmutableList; -import com.google.zetasql.Analyzer; -import com.google.zetasql.AnalyzerOptions; -import com.google.zetasql.Function; -import com.google.zetasql.FunctionArgumentType; -import com.google.zetasql.FunctionSignature; -import com.google.zetasql.SimpleCatalog; -import com.google.zetasql.TVFRelation; -import com.google.zetasql.TableValuedFunction; -import com.google.zetasql.Type; -import com.google.zetasql.TypeFactory; -import com.google.zetasql.ZetaSQLBuiltinFunctionOptions; -import com.google.zetasql.ZetaSQLFunctions; -import com.google.zetasql.ZetaSQLType; -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes; -import java.lang.reflect.Method; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; -import org.apache.beam.sdk.extensions.sql.impl.JavaUdfLoader; -import org.apache.beam.sdk.extensions.sql.impl.LazyAggregateCombineFn; -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.UdafImpl; -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; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.FunctionParameter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -/** - * Catalog for registering tables and functions. Populates a {@link SimpleCatalog} based on a {@link - * SchemaPlus}. - */ -public class BeamZetaSqlCatalog { - // ZetaSQL function group identifiers. Different function groups may have divergent translation - // paths. - public static final String PRE_DEFINED_WINDOW_FUNCTIONS = "pre_defined_window_functions"; - public static final String USER_DEFINED_SQL_FUNCTIONS = "user_defined_functions"; - public static final String USER_DEFINED_JAVA_SCALAR_FUNCTIONS = - "user_defined_java_scalar_functions"; - public static final String USER_DEFINED_JAVA_AGGREGATE_FUNCTIONS = - "user_defined_java_aggregate_functions"; - /** - * Same as {@link Function}.ZETASQL_FUNCTION_GROUP_NAME. Identifies built-in ZetaSQL functions. - */ - public static final String ZETASQL_FUNCTION_GROUP_NAME = "ZetaSQL"; - - private static final ImmutableList PRE_DEFINED_WINDOW_FUNCTION_DECLARATIONS = - ImmutableList.of( - // TODO: support optional function argument (for window_offset). - "CREATE FUNCTION TUMBLE(ts TIMESTAMP, window_size STRING) AS (1);", - "CREATE FUNCTION TUMBLE_START(window_size STRING) RETURNS TIMESTAMP AS (null);", - "CREATE FUNCTION TUMBLE_END(window_size STRING) RETURNS TIMESTAMP AS (null);", - "CREATE FUNCTION HOP(ts TIMESTAMP, emit_frequency STRING, window_size STRING) AS (1);", - "CREATE FUNCTION HOP_START(emit_frequency STRING, window_size STRING) " - + "RETURNS TIMESTAMP AS (null);", - "CREATE FUNCTION HOP_END(emit_frequency STRING, window_size STRING) " - + "RETURNS TIMESTAMP AS (null);", - "CREATE FUNCTION SESSION(ts TIMESTAMP, session_gap STRING) AS (1);", - "CREATE FUNCTION SESSION_START(session_gap STRING) RETURNS TIMESTAMP AS (null);", - "CREATE FUNCTION SESSION_END(session_gap STRING) RETURNS TIMESTAMP AS (null);"); - - /** The top-level Calcite schema, which may contain sub-schemas. */ - private final SchemaPlus calciteSchema; - /** - * The top-level ZetaSQL catalog, which may contain nested catalogs for qualified table and - * function references. - */ - private final SimpleCatalog zetaSqlCatalog; - - private final JavaTypeFactory typeFactory; - - private final JavaUdfLoader javaUdfLoader = new JavaUdfLoader(); - private final Map, ResolvedNodes.ResolvedCreateFunctionStmt> sqlScalarUdfs = - new HashMap<>(); - /** User-defined table valued functions. */ - private final Map, ResolvedNode> sqlUdtvfs = new HashMap<>(); - - private final Map, UserFunctionDefinitions.JavaScalarFunction> javaScalarUdfs = - new HashMap<>(); - private final Map, Combine.CombineFn> javaUdafs = new HashMap<>(); - - private BeamZetaSqlCatalog( - SchemaPlus calciteSchema, SimpleCatalog zetaSqlCatalog, JavaTypeFactory typeFactory) { - this.calciteSchema = calciteSchema; - this.zetaSqlCatalog = zetaSqlCatalog; - this.typeFactory = typeFactory; - } - - /** Return catalog pre-populated with builtin functions. */ - static BeamZetaSqlCatalog create( - SchemaPlus calciteSchema, JavaTypeFactory typeFactory, AnalyzerOptions options) { - BeamZetaSqlCatalog catalog = - new BeamZetaSqlCatalog( - calciteSchema, new SimpleCatalog(calciteSchema.getName()), typeFactory); - catalog.addFunctionsToCatalog(options); - return catalog; - } - - SimpleCatalog getZetaSqlCatalog() { - return zetaSqlCatalog; - } - - void addTables(List> tables, QueryTrait queryTrait) { - tables.forEach(table -> addTableToLeafCatalog(table, queryTrait)); - } - - void addFunction(ResolvedNodes.ResolvedCreateFunctionStmt createFunctionStmt) { - String functionGroup = getFunctionGroup(createFunctionStmt); - switch (functionGroup) { - case USER_DEFINED_SQL_FUNCTIONS: - sqlScalarUdfs.put(createFunctionStmt.getNamePath(), createFunctionStmt); - break; - case USER_DEFINED_JAVA_SCALAR_FUNCTIONS: - String functionName = String.join(".", createFunctionStmt.getNamePath()); - for (FunctionArgumentType argumentType : - createFunctionStmt.getSignature().getFunctionArgumentList()) { - Type type = argumentType.getType(); - if (type == null) { - throw new UnsupportedOperationException( - "UDF templated argument types are not supported."); - } - validateJavaUdfZetaSqlType(type, functionName); - } - if (createFunctionStmt.getReturnType() == null) { - throw new IllegalArgumentException("UDF return type must not be null."); - } - validateJavaUdfZetaSqlType(createFunctionStmt.getReturnType(), functionName); - String jarPath = getJarPath(createFunctionStmt); - ScalarFn scalarFn = - javaUdfLoader.loadScalarFunction(createFunctionStmt.getNamePath(), jarPath); - Method method = ScalarFnReflector.getApplyMethod(scalarFn); - javaScalarUdfs.put( - createFunctionStmt.getNamePath(), - UserFunctionDefinitions.JavaScalarFunction.create(method, jarPath)); - break; - case USER_DEFINED_JAVA_AGGREGATE_FUNCTIONS: - jarPath = getJarPath(createFunctionStmt); - // Try loading the aggregate function just to make sure it exists. LazyAggregateCombineFn - // will need to fetch it again at runtime. - javaUdfLoader.loadAggregateFunction(createFunctionStmt.getNamePath(), jarPath); - Combine.CombineFn combineFn = - new LazyAggregateCombineFn<>(createFunctionStmt.getNamePath(), jarPath); - javaUdafs.put(createFunctionStmt.getNamePath(), combineFn); - break; - default: - throw new IllegalArgumentException( - String.format("Encountered unrecognized function group %s.", functionGroup)); - } - zetaSqlCatalog.addFunction( - new Function( - createFunctionStmt.getNamePath(), - functionGroup, - createFunctionStmt.getIsAggregate() - ? ZetaSQLFunctions.FunctionEnums.Mode.AGGREGATE - : ZetaSQLFunctions.FunctionEnums.Mode.SCALAR, - ImmutableList.of(createFunctionStmt.getSignature()))); - } - - /** - * Throws {@link UnsupportedOperationException} if ZetaSQL type is not supported in Java UDF. - * Supported types are a subset of the types supported by {@link BeamJavaUdfCalcRule}. - * - *

Supported types should be kept in sync with {@link #validateJavaUdfCalciteType(RelDataType, - * String)}. - */ - void validateJavaUdfZetaSqlType(Type type, String functionName) { - switch (type.getKind()) { - case TYPE_BOOL: - case TYPE_BYTES: - case TYPE_DATE: - case TYPE_DOUBLE: - case TYPE_INT64: - case TYPE_NUMERIC: - case TYPE_STRING: - case TYPE_TIMESTAMP: - // These types are supported. - break; - case TYPE_ARRAY: - validateJavaUdfZetaSqlType(type.asArray().getElementType(), functionName); - break; - case TYPE_TIME: - case TYPE_DATETIME: - case TYPE_STRUCT: - default: - throw new UnsupportedOperationException( - String.format( - "ZetaSQL type %s not allowed in function %s", type.getKind().name(), functionName)); - } - } - - void addTableValuedFunction( - ResolvedNodes.ResolvedCreateTableFunctionStmt createTableFunctionStmt) { - zetaSqlCatalog.addTableValuedFunction( - new TableValuedFunction.FixedOutputSchemaTVF( - createTableFunctionStmt.getNamePath(), - createTableFunctionStmt.getSignature(), - TVFRelation.createColumnBased( - createTableFunctionStmt.getQuery().getColumnList().stream() - .map(c -> TVFRelation.Column.create(c.getName(), c.getType())) - .collect(Collectors.toList())))); - sqlUdtvfs.put(createTableFunctionStmt.getNamePath(), createTableFunctionStmt.getQuery()); - } - - UserFunctionDefinitions getUserFunctionDefinitions() { - return UserFunctionDefinitions.newBuilder() - .setSqlScalarFunctions(ImmutableMap.copyOf(sqlScalarUdfs)) - .setSqlTableValuedFunctions(ImmutableMap.copyOf(sqlUdtvfs)) - .setJavaScalarFunctions(ImmutableMap.copyOf(javaScalarUdfs)) - .setJavaAggregateFunctions(ImmutableMap.copyOf(javaUdafs)) - .build(); - } - - private void addFunctionsToCatalog(AnalyzerOptions options) { - // Enable ZetaSQL builtin functions. - ZetaSQLBuiltinFunctionOptions zetasqlBuiltinFunctionOptions = - new ZetaSQLBuiltinFunctionOptions(options.getLanguageOptions()); - SupportedZetaSqlBuiltinFunctions.ALLOWLIST.forEach( - zetasqlBuiltinFunctionOptions::includeFunctionSignatureId); - zetaSqlCatalog.addZetaSQLFunctions(zetasqlBuiltinFunctionOptions); - - // Enable Beam SQL's builtin windowing functions. - addWindowScalarFunctions(options); - addWindowTvfs(); - - // Add user-defined functions already defined in the schema, if any. - addUdfsFromSchema(); - } - - private void addWindowScalarFunctions(AnalyzerOptions options) { - PRE_DEFINED_WINDOW_FUNCTION_DECLARATIONS.stream() - .map( - func -> - (ResolvedNodes.ResolvedCreateFunctionStmt) - Analyzer.analyzeStatement(func, options, zetaSqlCatalog)) - .map( - resolvedFunc -> - new Function( - String.join(".", resolvedFunc.getNamePath()), - PRE_DEFINED_WINDOW_FUNCTIONS, - ZetaSQLFunctions.FunctionEnums.Mode.SCALAR, - ImmutableList.of(resolvedFunc.getSignature()))) - .forEach(zetaSqlCatalog::addFunction); - } - - @SuppressWarnings({ - "nullness" // customContext and volatility are in fact nullable, but they are missing the - // annotation upstream. TODO Unsuppress when this is fixed in ZetaSQL. - }) - private void addWindowTvfs() { - FunctionArgumentType retType = - new FunctionArgumentType(ZetaSQLFunctions.SignatureArgumentKind.ARG_TYPE_RELATION); - - FunctionArgumentType inputTableType = - new FunctionArgumentType(ZetaSQLFunctions.SignatureArgumentKind.ARG_TYPE_RELATION); - - FunctionArgumentType descriptorType = - new FunctionArgumentType( - ZetaSQLFunctions.SignatureArgumentKind.ARG_TYPE_DESCRIPTOR, - FunctionArgumentType.FunctionArgumentTypeOptions.builder() - .setDescriptorResolutionTableOffset(0) - .build(), - 1); - - FunctionArgumentType stringType = - new FunctionArgumentType(TypeFactory.createSimpleType(ZetaSQLType.TypeKind.TYPE_STRING)); - - // TUMBLE - zetaSqlCatalog.addTableValuedFunction( - new TableValuedFunction.ForwardInputSchemaToOutputSchemaWithAppendedColumnTVF( - ImmutableList.of(TVFStreamingUtils.FIXED_WINDOW_TVF), - new FunctionSignature( - retType, ImmutableList.of(inputTableType, descriptorType, stringType), -1), - ImmutableList.of( - TVFRelation.Column.create( - TVFStreamingUtils.WINDOW_START, - TypeFactory.createSimpleType(ZetaSQLType.TypeKind.TYPE_TIMESTAMP)), - TVFRelation.Column.create( - TVFStreamingUtils.WINDOW_END, - TypeFactory.createSimpleType(ZetaSQLType.TypeKind.TYPE_TIMESTAMP))), - null, - null)); - - // HOP - zetaSqlCatalog.addTableValuedFunction( - new TableValuedFunction.ForwardInputSchemaToOutputSchemaWithAppendedColumnTVF( - ImmutableList.of(TVFStreamingUtils.SLIDING_WINDOW_TVF), - new FunctionSignature( - retType, - ImmutableList.of(inputTableType, descriptorType, stringType, stringType), - -1), - ImmutableList.of( - TVFRelation.Column.create( - TVFStreamingUtils.WINDOW_START, - TypeFactory.createSimpleType(ZetaSQLType.TypeKind.TYPE_TIMESTAMP)), - TVFRelation.Column.create( - TVFStreamingUtils.WINDOW_END, - TypeFactory.createSimpleType(ZetaSQLType.TypeKind.TYPE_TIMESTAMP))), - null, - null)); - - // SESSION - zetaSqlCatalog.addTableValuedFunction( - new TableValuedFunction.ForwardInputSchemaToOutputSchemaWithAppendedColumnTVF( - ImmutableList.of(TVFStreamingUtils.SESSION_WINDOW_TVF), - new FunctionSignature( - retType, - ImmutableList.of(inputTableType, descriptorType, descriptorType, stringType), - -1), - ImmutableList.of( - TVFRelation.Column.create( - TVFStreamingUtils.WINDOW_START, - TypeFactory.createSimpleType(ZetaSQLType.TypeKind.TYPE_TIMESTAMP)), - TVFRelation.Column.create( - TVFStreamingUtils.WINDOW_END, - TypeFactory.createSimpleType(ZetaSQLType.TypeKind.TYPE_TIMESTAMP))), - null, - null)); - } - - private void addUdfsFromSchema() { - for (String functionName : calciteSchema.getFunctionNames()) { - Collection - functions = calciteSchema.getFunctions(functionName); - if (functions.size() != 1) { - throw new IllegalArgumentException( - String.format( - "Expected exactly 1 definition for function '%s', but found %d." - + " Beam ZetaSQL supports only a single function definition per function name (https://github.com/apache/beam/issues/20828).", - functionName, functions.size())); - } - for (org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function function : - functions) { - List path = Arrays.asList(functionName.split("\\.")); - if (function instanceof ScalarFunctionImpl) { - ScalarFunctionImpl scalarFunction = (ScalarFunctionImpl) function; - // Validate types before converting from Calcite to ZetaSQL, since the conversion may fail - // for unsupported types. - for (FunctionParameter parameter : scalarFunction.getParameters()) { - validateJavaUdfCalciteType(parameter.getType(typeFactory), functionName); - } - validateJavaUdfCalciteType(scalarFunction.getReturnType(typeFactory), functionName); - Method method = scalarFunction.method; - javaScalarUdfs.put(path, UserFunctionDefinitions.JavaScalarFunction.create(method, "")); - FunctionArgumentType resultType = - new FunctionArgumentType( - ZetaSqlCalciteTranslationUtils.toZetaSqlType( - scalarFunction.getReturnType(typeFactory))); - FunctionSignature functionSignature = - new FunctionSignature(resultType, getArgumentTypes(scalarFunction), 0L); - zetaSqlCatalog.addFunction( - new Function( - path, - USER_DEFINED_JAVA_SCALAR_FUNCTIONS, - ZetaSQLFunctions.FunctionEnums.Mode.SCALAR, - ImmutableList.of(functionSignature))); - } else if (function instanceof UdafImpl) { - UdafImpl udaf = (UdafImpl) function; - javaUdafs.put(path, udaf.getCombineFn()); - FunctionArgumentType resultType = - new FunctionArgumentType( - ZetaSqlCalciteTranslationUtils.toZetaSqlType(udaf.getReturnType(typeFactory))); - FunctionSignature functionSignature = - new FunctionSignature(resultType, getArgumentTypes(udaf), 0L); - zetaSqlCatalog.addFunction( - new Function( - path, - USER_DEFINED_JAVA_AGGREGATE_FUNCTIONS, - ZetaSQLFunctions.FunctionEnums.Mode.AGGREGATE, - ImmutableList.of(functionSignature))); - } else { - throw new IllegalArgumentException( - String.format( - "Function %s has unrecognized implementation type %s.", - functionName, function.getClass().getName())); - } - } - } - } - - private List getArgumentTypes( - org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function function) { - return function.getParameters().stream() - .map( - (arg) -> - new FunctionArgumentType( - ZetaSqlCalciteTranslationUtils.toZetaSqlType(arg.getType(typeFactory)))) - .collect(Collectors.toList()); - } - - /** - * Throws {@link UnsupportedOperationException} if Calcite type is not supported in Java UDF. - * Supported types are a subset of the corresponding Calcite types supported by {@link - * BeamJavaUdfCalcRule}. - * - *

Supported types should be kept in sync with {@link #validateJavaUdfZetaSqlType(Type, - * String)}. - */ - private void validateJavaUdfCalciteType(RelDataType type, String functionName) { - switch (type.getSqlTypeName()) { - case BIGINT: - case BOOLEAN: - case DATE: - case DECIMAL: - case DOUBLE: - case TIMESTAMP: - case VARCHAR: - case VARBINARY: - // These types are supported. - break; - case ARRAY: - validateJavaUdfCalciteType( - Preconditions.checkArgumentNotNull( - type.getComponentType(), "Encountered ARRAY type with no component type."), - functionName); - break; - case TIME: - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - case ROW: - default: - throw new UnsupportedOperationException( - String.format( - "Calcite type %s not allowed in function %s", - type.getSqlTypeName().getName(), functionName)); - } - } - - private String getFunctionGroup(ResolvedNodes.ResolvedCreateFunctionStmt createFunctionStmt) { - switch (createFunctionStmt.getLanguage().toUpperCase()) { - case "JAVA": - return createFunctionStmt.getIsAggregate() - ? USER_DEFINED_JAVA_AGGREGATE_FUNCTIONS - : USER_DEFINED_JAVA_SCALAR_FUNCTIONS; - case "SQL": - if (createFunctionStmt.getIsAggregate()) { - throw new UnsupportedOperationException( - "Native SQL aggregate functions are not supported (https://github.com/apache/beam/issues/20193)."); - } - return USER_DEFINED_SQL_FUNCTIONS; - case "PY": - case "PYTHON": - case "JS": - case "JAVASCRIPT": - throw new UnsupportedOperationException( - String.format( - "Function %s uses unsupported language %s.", - String.join(".", createFunctionStmt.getNamePath()), - createFunctionStmt.getLanguage())); - default: - throw new IllegalArgumentException( - String.format( - "Function %s uses unrecognized language %s.", - String.join(".", createFunctionStmt.getNamePath()), - createFunctionStmt.getLanguage())); - } - } - - /** - * Assume last element in tablePath is a table name, and everything before is catalogs. So the - * logic is to create nested catalogs until the last level, then add a table at the last level. - * - *

Table schema is extracted from Calcite schema based on the table name resolution strategy, - * e.g. either by drilling down the schema.getSubschema() path or joining the table name with dots - * to construct a single compound identifier (e.g. Data Catalog use case). - */ - private void addTableToLeafCatalog(List tablePath, QueryTrait queryTrait) { - - SimpleCatalog leafCatalog = createNestedCatalogs(zetaSqlCatalog, tablePath); - - org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Table calciteTable = - TableResolution.resolveCalciteTable(calciteSchema, tablePath); - - if (calciteTable == null) { - throw new ZetaSqlException( - "Wasn't able to resolve the path " - + tablePath - + " in schema: " - + calciteSchema.getName()); - } - - RelDataType rowType = calciteTable.getRowType(typeFactory); - - TableResolution.SimpleTableWithPath tableWithPath = - TableResolution.SimpleTableWithPath.of(tablePath); - queryTrait.addResolvedTable(tableWithPath); - - addFieldsToTable(tableWithPath, rowType); - leafCatalog.addSimpleTable(tableWithPath.getTable()); - } - - private static void addFieldsToTable( - TableResolution.SimpleTableWithPath tableWithPath, RelDataType rowType) { - for (RelDataTypeField field : rowType.getFieldList()) { - tableWithPath - .getTable() - .addSimpleColumn( - field.getName(), ZetaSqlCalciteTranslationUtils.toZetaSqlType(field.getType())); - } - } - - /** For table path like a.b.c we assume c is the table and a.b are the nested catalogs/schemas. */ - private static SimpleCatalog createNestedCatalogs(SimpleCatalog catalog, List tablePath) { - SimpleCatalog currentCatalog = catalog; - for (int i = 0; i < tablePath.size() - 1; i++) { - String nextCatalogName = tablePath.get(i); - - Optional existing = tryGetExisting(currentCatalog, nextCatalogName); - - currentCatalog = - existing.isPresent() ? existing.get() : addNewCatalog(currentCatalog, nextCatalogName); - } - return currentCatalog; - } - - private static Optional tryGetExisting( - SimpleCatalog currentCatalog, String nextCatalogName) { - return currentCatalog.getCatalogList().stream() - .filter(c -> nextCatalogName.equals(c.getFullName())) - .findFirst(); - } - - private static SimpleCatalog addNewCatalog(SimpleCatalog currentCatalog, String nextCatalogName) { - SimpleCatalog nextCatalog = new SimpleCatalog(nextCatalogName); - currentCatalog.addSimpleCatalog(nextCatalog); - return nextCatalog; - } - - private static String getJarPath(ResolvedNodes.ResolvedCreateFunctionStmt createFunctionStmt) { - String jarPath = getOptionStringValue(createFunctionStmt, "path"); - if (jarPath.isEmpty()) { - throw new IllegalArgumentException( - String.format( - "No jar was provided to define function %s. Add 'OPTIONS (path=)' to the CREATE FUNCTION statement.", - String.join(".", createFunctionStmt.getNamePath()))); - } - return jarPath; - } - - private static String getOptionStringValue( - ResolvedNodes.ResolvedCreateFunctionStmt createFunctionStmt, String optionName) { - for (ResolvedNodes.ResolvedOption option : createFunctionStmt.getOptionList()) { - if (optionName.equals(option.getName())) { - if (option.getValue() == null) { - throw new IllegalArgumentException( - String.format( - "Option '%s' has null value (expected %s).", - optionName, ZetaSQLType.TypeKind.TYPE_STRING)); - } - if (option.getValue().getType().getKind() != ZetaSQLType.TypeKind.TYPE_STRING) { - throw new IllegalArgumentException( - String.format( - "Option '%s' has type %s (expected %s).", - optionName, - option.getValue().getType().getKind(), - ZetaSQLType.TypeKind.TYPE_STRING)); - } - return ((ResolvedNodes.ResolvedLiteral) option.getValue()).getValue().getStringValue(); - } - } - return ""; - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlRelType.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlRelType.java deleted file mode 100644 index c952273ce106..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlRelType.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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 org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; -import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter; -import org.apache.beam.sdk.extensions.sql.zetasql.translation.ZetaSqlScalarFunctionImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexDynamicParam; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexFieldAccess; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexProgram; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RelBuilder; - -/** {@link CalcRelSplitter.RelType} for {@link BeamZetaSqlCalcRel}. */ -class BeamZetaSqlRelType extends CalcRelSplitter.RelType { - BeamZetaSqlRelType(String name) { - super(name); - } - - @Override - protected boolean canImplement(RexFieldAccess field) { - return true; - } - - @Override - protected boolean canImplement(RexDynamicParam param) { - return true; - } - - @Override - protected boolean canImplement(RexLiteral literal) { - return true; - } - - @Override - protected boolean canImplement(RexCall call) { - if (call.getOperator() instanceof SqlUserDefinedFunction) { - SqlUserDefinedFunction udf = (SqlUserDefinedFunction) call.op; - if (udf.function instanceof ZetaSqlScalarFunctionImpl) { - ZetaSqlScalarFunctionImpl scalarFunction = (ZetaSqlScalarFunctionImpl) udf.function; - if (scalarFunction.functionGroup.equals( - BeamZetaSqlCatalog.USER_DEFINED_JAVA_SCALAR_FUNCTIONS)) { - return false; - } - } - } - return true; - } - - @Override - protected RelNode makeRel( - RelOptCluster cluster, - RelTraitSet traitSet, - RelBuilder relBuilder, - RelNode input, - RexProgram program) { - RexProgram normalizedProgram = program.normalize(cluster.getRexBuilder(), false); - return new BeamZetaSqlCalcRel( - cluster, - traitSet.replace(BeamLogicalConvention.INSTANCE), - RelOptRule.convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)), - normalizedProgram); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/DateTimeUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/DateTimeUtils.java deleted file mode 100644 index a60cd395b581..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/DateTimeUtils.java +++ /dev/null @@ -1,244 +0,0 @@ -/* - * 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 static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; - -import com.google.zetasql.Value; -import io.grpc.Status; -import java.time.LocalTime; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.TimeUnit; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.math.LongMath; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; - -/** DateTimeUtils. */ -public class DateTimeUtils { - public static final Long MILLIS_PER_DAY = 86400000L; - private static final Long MICROS_PER_MILLI = 1000L; - - private enum TimestampPatterns { - TIMESTAMP_PATTERN, - TIMESTAMP_PATTERN_SUBSECOND, - TIMESTAMP_PATTERN_T, - TIMESTAMP_PATTERN_SUBSECOND_T, - } - - private static final ImmutableMap - TIMESTAMP_PATTERN_WITHOUT_TZ = - ImmutableMap.of( - TimestampPatterns.TIMESTAMP_PATTERN, DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss"), - TimestampPatterns.TIMESTAMP_PATTERN_SUBSECOND, - DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS"), - TimestampPatterns.TIMESTAMP_PATTERN_T, - DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss"), - TimestampPatterns.TIMESTAMP_PATTERN_SUBSECOND_T, - DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSS")); - - private static final ImmutableMap - TIMESTAMP_PATTERN_WITH_TZ = - ImmutableMap.of( - TimestampPatterns.TIMESTAMP_PATTERN, - DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ssZZ"), - TimestampPatterns.TIMESTAMP_PATTERN_SUBSECOND, - DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSZZ"), - TimestampPatterns.TIMESTAMP_PATTERN_T, - DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ssZZ"), - TimestampPatterns.TIMESTAMP_PATTERN_SUBSECOND_T, - DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSSZZ")); - - public static DateTimeFormatter findDateTimePattern(String str) { - if (str.indexOf('+') == -1) { - return findDateTimePattern(str, TIMESTAMP_PATTERN_WITHOUT_TZ); - } else { - return findDateTimePattern(str, TIMESTAMP_PATTERN_WITH_TZ); - } - } - - public static DateTimeFormatter findDateTimePattern( - String str, ImmutableMap patternMap) { - if (str.indexOf('.') == -1) { - if (str.indexOf('T') == -1) { - return checkNotNull(patternMap.get(TimestampPatterns.TIMESTAMP_PATTERN)); - } else { - return checkNotNull(patternMap.get(TimestampPatterns.TIMESTAMP_PATTERN_T)); - } - } else { - if (str.indexOf('T') == -1) { - return checkNotNull(patternMap.get(TimestampPatterns.TIMESTAMP_PATTERN_SUBSECOND)); - } else { - return checkNotNull(patternMap.get(TimestampPatterns.TIMESTAMP_PATTERN_SUBSECOND_T)); - } - } - } - - // https://cloud.google.com/bigquery/docs/reference/standard-sql/migrating-from-legacy-sql#timestamp_differences - // 0001-01-01 00:00:00 to 9999-12-31 23:59:59.999999 UTC. - // -62135596800000000 to 253402300799999999 - @SuppressWarnings("GoodTime") - public static final Long MIN_UNIX_MILLIS = -62135596800000L; - - @SuppressWarnings("GoodTime") - public static final Long MAX_UNIX_MILLIS = 253402300799999L; - - public static DateTime parseTimestampWithUTCTimeZone(String str) { - return findDateTimePattern(str).withZoneUTC().parseDateTime(str); - } - - @SuppressWarnings("unused") - public static DateTime parseTimestampWithLocalTimeZone(String str) { - return findDateTimePattern(str).withZone(DateTimeZone.getDefault()).parseDateTime(str); - } - - public static DateTime parseTimestampWithTimeZone(String str) { - // for example, accept "1990-10-20 13:24:01+0730" - if (str.indexOf('.') == -1) { - return DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ssZ").parseDateTime(str); - } else { - return DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSSZ").parseDateTime(str); - } - } - - public static String formatTimestampWithTimeZone(DateTime dt) { - String resultWithoutZone; - if (dt.getMillisOfSecond() == 0) { - resultWithoutZone = dt.toString(DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss")); - } else { - resultWithoutZone = dt.toString(DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.SSS")); - } - - // ZetaSQL expects a 2-digit timezone offset (-05) if the minute part is zero, and it expects - // a 4-digit timezone with a colon (-07:52) if the minute part is non-zero. None of the - // variations on z,Z,ZZ,.. do this for us so we have to do it manually here. - String zone = dt.toString(DateTimeFormat.forPattern("ZZ")); - List zoneParts = Lists.newArrayList(Splitter.on(':').limit(2).split(zone)); - if (zoneParts.size() == 2 && zoneParts.get(1).equals("00")) { - zone = zoneParts.get(0); - } - - return resultWithoutZone + zone; - } - - @SuppressWarnings("unused") - public static DateTime parseTimestampWithoutTimeZone(String str) { - return DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").parseDateTime(str); - } - - public static DateTime parseDate(String str) { - return DateTimeFormat.forPattern("yyyy-MM-dd").withZoneUTC().parseDateTime(str); - } - - public static DateTime parseTime(String str) { - // DateTimeFormat does not parse "08:10:10" for pattern "HH:mm:ss.SSS". In this case, '.' must - // appear. - if (str.indexOf('.') == -1) { - return DateTimeFormat.forPattern("HH:mm:ss").withZoneUTC().parseDateTime(str); - } else { - return DateTimeFormat.forPattern("HH:mm:ss.SSS").withZoneUTC().parseDateTime(str); - } - } - - public static Value parseDateToValue(String dateString) { - DateTime dateTime = parseDate(dateString); - return Value.createDateValue((int) (dateTime.getMillis() / MILLIS_PER_DAY)); - } - - public static Value parseTimeToValue(String timeString) { - LocalTime localTime = LocalTime.parse(timeString); - return Value.createTimeValue(localTime); - } - - public static Value parseTimestampWithTZToValue(String timestampString) { - DateTime dateTime = parseTimestampWithTimeZone(timestampString); - // convert from micros. - // TODO: how to handle overflow. - return Value.createTimestampValueFromUnixMicros( - LongMath.checkedMultiply(dateTime.getMillis(), MICROS_PER_MILLI)); - } - - /** - * This function validates that Long representation of timestamp is compatible with ZetaSQL - * timestamp values range. - * - *

Invoked via reflection. @see SqlOperators - * - * @param ts Timestamp to validate. - * @return Unchanged timestamp sent for validation. - */ - @SuppressWarnings("GoodTime") - public static @Nullable Long validateTimestamp(@Nullable Long ts) { - if (ts == null) { - return null; - } - - if ((ts < MIN_UNIX_MILLIS) || (ts > MAX_UNIX_MILLIS)) { - throw Status.OUT_OF_RANGE - .withDescription("Timestamp is out of valid range.") - .asRuntimeException(); - } - - return ts; - } - - /** - * This function validates that interval is compatible with ZetaSQL timestamp values range. - * - *

ZetaSQL validates that if we represent interval in milliseconds, it will fit into Long. - * - *

In case of SECOND or smaller time unit, it converts timestamp to microseconds, so we need to - * convert those to microsecond and verify that we do not cause overflow. - * - *

Invoked via reflection. @see SqlOperators - * - * @param arg Argument for the interval. - * @param unit Time unit used in this interval. - * @return Argument for the interval. - */ - @SuppressWarnings("GoodTime") - public static @Nullable Long validateTimeInterval(@Nullable Long arg, TimeUnit unit) { - if (arg == null) { - return null; - } - - // multiplier to convert to milli or microseconds. - long multiplier = unit.multiplier.longValue(); - switch (unit) { - case SECOND: - case MILLISECOND: - multiplier *= 1000L; // Change multiplier from milliseconds to microseconds. - break; - default: - break; - } - - if ((arg > Long.MAX_VALUE / multiplier) || (arg < Long.MIN_VALUE / multiplier)) { - throw Status.OUT_OF_RANGE - .withDescription("Interval is out of valid range") - .asRuntimeException(); - } - - return arg; - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/QueryTrait.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/QueryTrait.java deleted file mode 100644 index ac1cd0820387..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/QueryTrait.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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 static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - -import com.google.zetasql.Table; -import com.google.zetasql.resolvedast.ResolvedColumn; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedOutputColumn; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedWithEntry; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.beam.sdk.extensions.sql.zetasql.TableResolution.SimpleTableWithPath; - -/** QueryTrait. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class QueryTrait { - public Map withEntries = new HashMap<>(); - - public Map outputColumnMap = new HashMap<>(); - - public Map resolvedTables = new HashMap<>(); - - // TODO: move query parameter map to QueryTrait. - - public void addOutputColumnList(List outputColumnList) { - outputColumnList.forEach( - column -> { - outputColumnMap.put(column.getColumn(), column.getName()); - }); - } - - /** Store a table together with its full path for repeated resolutions. */ - public void addResolvedTable(SimpleTableWithPath tableWithPath) { - // table ids are autoincremted in SimpleTable - resolvedTables.put(tableWithPath.getTable().getId(), tableWithPath); - } - - /** True if the table was resolved using the Calcite schema. */ - public boolean isTableResolved(Table table) { - return resolvedTables.containsKey(table.getId()); - } - - /** Returns a full table path (exlucding top-level schema) for a given ZetaSQL Table. */ - public List getTablePath(Table table) { - checkArgument( - isTableResolved(table), - "Attempting to get a path of an unresolved table. Resolve and add the table first: %s", - table.getFullName()); - return resolvedTables.get(table.getId()).getPath(); - } - - public List retrieveFieldNames(List resolvedColumnList) { - return resolvedColumnList.stream().map(this::resolveAlias).collect(Collectors.toList()); - } - - public String resolveAlias(ResolvedColumn resolvedColumn) { - return this.outputColumnMap.getOrDefault(resolvedColumn, resolvedColumn.getName()); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java deleted file mode 100644 index 0b5d09515b0e..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * 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 static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_CREATE_FUNCTION_STMT; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_CREATE_TABLE_FUNCTION_STMT; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_QUERY_STMT; -import static java.nio.charset.StandardCharsets.UTF_8; - -import com.google.zetasql.Analyzer; -import com.google.zetasql.AnalyzerOptions; -import com.google.zetasql.ParseResumeLocation; -import com.google.zetasql.Value; -import com.google.zetasql.ZetaSQLOptions.ErrorMessageMode; -import com.google.zetasql.ZetaSQLOptions.LanguageFeature; -import com.google.zetasql.ZetaSQLOptions.ParameterMode; -import com.google.zetasql.ZetaSQLOptions.ProductMode; -import com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind; -import com.google.zetasql.resolvedast.ResolvedNodes; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedCreateFunctionStmt; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedCreateTableFunctionStmt; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedStatement; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner.QueryParameters; -import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner.QueryParameters.Kind; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; - -/** Adapter for {@link Analyzer} to simplify the API for parsing the query and resolving the AST. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class SqlAnalyzer { - private static final ImmutableSet SUPPORTED_STATEMENT_KINDS = - ImmutableSet.of( - RESOLVED_QUERY_STMT, RESOLVED_CREATE_FUNCTION_STMT, RESOLVED_CREATE_TABLE_FUNCTION_STMT); - - SqlAnalyzer() {} - - /** Returns table names from all statements in the SQL string. */ - List> extractTableNames(String sql, AnalyzerOptions options) { - ParseResumeLocation parseResumeLocation = new ParseResumeLocation(sql); - ImmutableList.Builder> tables = ImmutableList.builder(); - while (!isEndOfInput(parseResumeLocation)) { - List> statementTables = - Analyzer.extractTableNamesFromNextStatement(parseResumeLocation, options); - tables.addAll(statementTables); - } - return tables.build(); - } - - /** - * Analyzes the entire SQL code block (which may consist of multiple statements) and returns the - * resolved query. - * - *

Assumes there is exactly one SELECT statement in the input, and it must be the last - * statement in the input. - */ - ResolvedNodes.ResolvedQueryStmt analyzeQuery( - String sql, AnalyzerOptions options, BeamZetaSqlCatalog catalog) { - ParseResumeLocation parseResumeLocation = new ParseResumeLocation(sql); - ResolvedStatement statement; - do { - statement = analyzeNextStatement(parseResumeLocation, options, catalog); - if (statement.nodeKind() == RESOLVED_QUERY_STMT) { - if (!SqlAnalyzer.isEndOfInput(parseResumeLocation)) { - throw new UnsupportedOperationException( - "No additional statements are allowed after a SELECT statement."); - } - } - } while (!SqlAnalyzer.isEndOfInput(parseResumeLocation)); - - if (!(statement instanceof ResolvedNodes.ResolvedQueryStmt)) { - throw new UnsupportedOperationException( - "Statement list must end in a SELECT statement, not " + statement.nodeKindString()); - } - return (ResolvedNodes.ResolvedQueryStmt) statement; - } - - private static boolean isEndOfInput(ParseResumeLocation parseResumeLocation) { - return parseResumeLocation.getBytePosition() - >= parseResumeLocation.getInput().getBytes(UTF_8).length; - } - - /** - * Accepts the ParseResumeLocation for the current position in the SQL string. Advances the - * ParseResumeLocation to the start of the next statement. Adds user-defined functions to the - * catalog for use in following statements. Returns the resolved AST. - */ - private ResolvedStatement analyzeNextStatement( - ParseResumeLocation parseResumeLocation, - AnalyzerOptions options, - BeamZetaSqlCatalog catalog) { - ResolvedStatement resolvedStatement = - Analyzer.analyzeNextStatement(parseResumeLocation, options, catalog.getZetaSqlCatalog()); - if (resolvedStatement.nodeKind() == RESOLVED_CREATE_FUNCTION_STMT) { - ResolvedCreateFunctionStmt createFunctionStmt = - (ResolvedCreateFunctionStmt) resolvedStatement; - try { - catalog.addFunction(createFunctionStmt); - } catch (IllegalArgumentException e) { - throw new RuntimeException( - String.format( - "Failed to define function '%s'", - String.join(".", createFunctionStmt.getNamePath())), - e); - } - } else if (resolvedStatement.nodeKind() == RESOLVED_CREATE_TABLE_FUNCTION_STMT) { - ResolvedCreateTableFunctionStmt createTableFunctionStmt = - (ResolvedCreateTableFunctionStmt) resolvedStatement; - catalog.addTableValuedFunction(createTableFunctionStmt); - } else if (!SUPPORTED_STATEMENT_KINDS.contains(resolvedStatement.nodeKind())) { - throw new UnsupportedOperationException( - "Unrecognized statement type " + resolvedStatement.nodeKindString()); - } - return resolvedStatement; - } - - static AnalyzerOptions baseAnalyzerOptions() { - AnalyzerOptions options = new AnalyzerOptions(); - options.setErrorMessageMode(ErrorMessageMode.ERROR_MESSAGE_MULTI_LINE_WITH_CARET); - - options.getLanguageOptions().setProductMode(ProductMode.PRODUCT_EXTERNAL); - options - .getLanguageOptions() - .setEnabledLanguageFeatures( - new HashSet<>( - Arrays.asList( - LanguageFeature.FEATURE_CREATE_AGGREGATE_FUNCTION, - LanguageFeature.FEATURE_CREATE_TABLE_FUNCTION, - LanguageFeature.FEATURE_DISALLOW_GROUP_BY_FLOAT, - LanguageFeature.FEATURE_NUMERIC_TYPE, - LanguageFeature.FEATURE_TABLE_VALUED_FUNCTIONS, - LanguageFeature.FEATURE_TEMPLATE_FUNCTIONS, - LanguageFeature.FEATURE_V_1_1_SELECT_STAR_EXCEPT_REPLACE, - LanguageFeature.FEATURE_V_1_2_CIVIL_TIME, - LanguageFeature.FEATURE_V_1_3_ADDITIONAL_STRING_FUNCTIONS))); - options.getLanguageOptions().setSupportedStatementKinds(SUPPORTED_STATEMENT_KINDS); - - return options; - } - - static AnalyzerOptions getAnalyzerOptions(QueryParameters queryParams, String defaultTimezone) { - AnalyzerOptions options = baseAnalyzerOptions(); - - options.setDefaultTimezone(defaultTimezone); - - if (queryParams.getKind() == Kind.NAMED) { - options.setParameterMode(ParameterMode.PARAMETER_NAMED); - for (Map.Entry entry : ((Map) queryParams.named()).entrySet()) { - options.addQueryParameter(entry.getKey(), entry.getValue().getType()); - } - } else if (queryParams.getKind() == Kind.POSITIONAL) { - options.setParameterMode(ParameterMode.PARAMETER_POSITIONAL); - for (Value param : (List) queryParams.positional()) { - options.addPositionalQueryParameter(param.getType()); - } - } - - return options; - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java deleted file mode 100644 index b3f71cea9e34..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java +++ /dev/null @@ -1,663 +0,0 @@ -/* - * 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.ZetaSQLFunction.FunctionSignatureId; -import java.util.List; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; - -/** - * List of ZetaSQL builtin functions supported by Beam ZetaSQL. Keep this list in sync with - * https://github.com/google/zetasql/blob/master/zetasql/public/builtin_function.proto. Uncomment - * the corresponding entries to enable parser support to the operators/functions. - * - *

Last synced ZetaSQL release: 2020.06.01 - */ -class SupportedZetaSqlBuiltinFunctions { - static final List ALLOWLIST = - ImmutableList.of( - FunctionSignatureId.FN_ADD_DOUBLE, // $add - FunctionSignatureId.FN_ADD_INT64, // $add - FunctionSignatureId.FN_ADD_NUMERIC, // $add - // FunctionSignatureId.FN_ADD_BIGNUMERIC, // $add - // FunctionSignatureId.FN_ADD_DATE_INT64, // $add - // FunctionSignatureId.FN_ADD_INT64_DATE, // $add - FunctionSignatureId.FN_AND, // $and - FunctionSignatureId.FN_CASE_NO_VALUE, // $case_no_value - FunctionSignatureId.FN_CASE_WITH_VALUE, // $case_with_value - FunctionSignatureId.FN_DIVIDE_DOUBLE, // $divide - FunctionSignatureId.FN_DIVIDE_NUMERIC, // $divide - // FunctionSignatureId.FN_DIVIDE_BIGNUMERIC, // $divide - FunctionSignatureId.FN_GREATER, // $greater - FunctionSignatureId.FN_GREATER_OR_EQUAL, // $greater_or_equal - FunctionSignatureId.FN_LESS, // $less - FunctionSignatureId.FN_LESS_OR_EQUAL, // $less_or_equal - FunctionSignatureId.FN_EQUAL, // $equal - FunctionSignatureId.FN_STRING_LIKE, // $like - FunctionSignatureId.FN_BYTE_LIKE, // $like - FunctionSignatureId.FN_IN, // $in - FunctionSignatureId.FN_IN_ARRAY, // $in_array - // FunctionSignatureId.FN_BETWEEN, // $between - FunctionSignatureId.FN_IS_NULL, // $is_null - FunctionSignatureId.FN_IS_TRUE, // $is_true - FunctionSignatureId.FN_IS_FALSE, // $is_false - FunctionSignatureId.FN_MULTIPLY_DOUBLE, // $multiply - FunctionSignatureId.FN_MULTIPLY_INT64, // $multiply - FunctionSignatureId.FN_MULTIPLY_NUMERIC, // $multiply - // FunctionSignatureId.FN_MULTIPLY_BIGNUMERIC, // $multiply - FunctionSignatureId.FN_NOT, // $not - FunctionSignatureId.FN_NOT_EQUAL, // $not_equal - FunctionSignatureId.FN_OR, // $or - FunctionSignatureId.FN_SUBTRACT_DOUBLE, // $subtract - FunctionSignatureId.FN_SUBTRACT_INT64, // $subtract - FunctionSignatureId.FN_SUBTRACT_NUMERIC, // $subtract - // FunctionSignatureId.FN_SUBTRACT_BIGNUMERIC, // $subtract - // FunctionSignatureId.FN_SUBTRACT_DATE_INT64, // $subtract - FunctionSignatureId.FN_UNARY_MINUS_INT64, // $unary_minus - FunctionSignatureId.FN_UNARY_MINUS_DOUBLE, // $unary_minus - FunctionSignatureId.FN_UNARY_MINUS_NUMERIC, // $unary_minus - // FunctionSignatureId.FN_UNARY_MINUS_BIGNUMERIC, // $unary_minus - - // Bitwise unary operators. - // FunctionSignatureId.FN_BITWISE_NOT_INT64, // $bitwise_not - // FunctionSignatureId.FN_BITWISE_NOT_BYTES, // $bitwise_not - // Bitwise binary operators. - // FunctionSignatureId.FN_BITWISE_OR_INT64, // $bitwise_or - // FunctionSignatureId.FN_BITWISE_OR_BYTES, // $bitwise_or - // FunctionSignatureId.FN_BITWISE_XOR_INT64, // $bitwise_xor - // FunctionSignatureId.FN_BITWISE_XOR_BYTES, // $bitwise_xor - // FunctionSignatureId.FN_BITWISE_AND_INT64, // $bitwise_and - // FunctionSignatureId.FN_BITWISE_AND_BYTES, // $bitwise_and - // FunctionSignatureId.FN_BITWISE_LEFT_SHIFT_INT64, // $bitwise_left_shift - // FunctionSignatureId.FN_BITWISE_LEFT_SHIFT_BYTES, // $bitwise_left_shift - // FunctionSignatureId.FN_BITWISE_RIGHT_SHIFT_INT64, // $bitwise_right_shift - // FunctionSignatureId.FN_BITWISE_RIGHT_SHIFT_BYTES, // $bitwise_right_shift - - // BIT_COUNT functions. - // FunctionSignatureId.FN_BIT_COUNT_INT64, // bit_count(int64) -> int64 - // FunctionSignatureId.FN_BIT_COUNT_BYTES, // bit_count(bytes) -> int64 - - // FunctionSignatureId.FN_ERROR,// error(string) -> {unused result, coercible to any type} - - FunctionSignatureId.FN_COUNT_STAR, // $count_star - - // - // The following functions use standard function call syntax. - // - - // String functions - FunctionSignatureId.FN_CONCAT_STRING, // concat(repeated string) -> string - // FunctionSignatureId.FN_CONCAT_BYTES, // concat(repeated bytes) -> bytes - // FunctionSignatureId.FN_CONCAT_OP_STRING, // concat(string, string) -> string - // FunctionSignatureId.FN_CONCAT_OP_BYTES, // concat(bytes, bytes) -> bytes - FunctionSignatureId.FN_STRPOS_STRING, // strpos(string, string) -> int64 - // FunctionSignatureId.FN_STRPOS_BYTES, // strpos(bytes, bytes) -> int64 - - // FunctionSignatureId.FN_INSTR_STRING,// instr(string, string[, int64[, int64]]) -> int64 - // FunctionSignatureId.FN_INSTR_BYTES, // instr(bytes, bytes[, int64[, int64]]) -> int64 - FunctionSignatureId.FN_LOWER_STRING, // lower(string) -> string - // FunctionSignatureId.FN_LOWER_BYTES, // lower(bytes) -> bytes - FunctionSignatureId.FN_UPPER_STRING, // upper(string) -> string - // FunctionSignatureId.FN_UPPER_BYTES, // upper(bytes) -> bytes - FunctionSignatureId.FN_LENGTH_STRING, // length(string) -> int64 - // FunctionSignatureId.FN_LENGTH_BYTES, // length(bytes) -> int64 - FunctionSignatureId.FN_STARTS_WITH_STRING, // starts_with(string, string) -> string - // FunctionSignatureId.FN_STARTS_WITH_BYTES, // starts_with(bytes, bytes) -> bytes - FunctionSignatureId.FN_ENDS_WITH_STRING, // ends_with(string, string) -> string - // FunctionSignatureId.FN_ENDS_WITH_BYTES, // ends_with(bytes, bytes) -> bytes - FunctionSignatureId.FN_SUBSTR_STRING, // substr(string, int64[, int64]) -> string - // FunctionSignatureId.FN_SUBSTR_BYTES, // substr(bytes, int64[, int64]) -> bytes - FunctionSignatureId.FN_TRIM_STRING, // trim(string[, string]) -> string - // FunctionSignatureId.FN_TRIM_BYTES, // trim(bytes, bytes) -> bytes - FunctionSignatureId.FN_LTRIM_STRING, // ltrim(string[, string]) -> string - // FunctionSignatureId.FN_LTRIM_BYTES, // ltrim(bytes, bytes) -> bytes - FunctionSignatureId.FN_RTRIM_STRING, // rtrim(string[, string]) -> string - // FunctionSignatureId.FN_RTRIM_BYTES, // rtrim(bytes, bytes) -> bytes - FunctionSignatureId.FN_REPLACE_STRING, // replace(string, string, string) -> string - // FunctionSignatureId.FN_REPLACE_BYTES, // replace(bytes, bytes, bytes) -> bytes - // FunctionSignatureId.FN_REGEXP_MATCH_STRING, // regexp_match(string, string) -> bool - // FunctionSignatureId.FN_REGEXP_MATCH_BYTES, // regexp_match(bytes, bytes) -> bool - // FunctionSignatureId.FN_REGEXP_EXTRACT_STRING,//regexp_extract(string, string) -> string - // FunctionSignatureId.FN_REGEXP_EXTRACT_BYTES, // regexp_extract(bytes, bytes) -> bytes - // FunctionSignatureId.FN_REGEXP_REPLACE_STRING, - // regexp_replace(string, string, string) -> string - // FunctionSignatureId.FN_REGEXP_REPLACE_BYTES, - // regexp_replace(bytes, bytes, bytes) -> bytes - // FunctionSignatureId.FN_REGEXP_EXTRACT_ALL_STRING, - // regexp_extract_all(string, string) -> array of string - // FunctionSignatureId.FN_REGEXP_EXTRACT_ALL_BYTES, - // regexp_extract_all(bytes, bytes) -> array of bytes - // FunctionSignatureId.FN_BYTE_LENGTH_STRING, // byte_length(string) -> int64 - // FunctionSignatureId.FN_BYTE_LENGTH_BYTES, // byte_length(bytes) -> int64 - // semantically identical to FN_LENGTH_BYTES - FunctionSignatureId.FN_CHAR_LENGTH_STRING, // char_length(string) -> int64 - // semantically identical to FN_LENGTH_STRING - // FunctionSignatureId.FN_FORMAT_STRING, // format(string, ...) -> string - // FunctionSignatureId.FN_SPLIT_STRING, // split(string, string) -> array of string - // FunctionSignatureId.FN_SPLIT_BYTES, // split(bytes, bytes) -> array of bytes - // FunctionSignatureId.FN_REGEXP_CONTAINS_STRING,//regexp_contains(string, string) -> bool - // FunctionSignatureId.FN_REGEXP_CONTAINS_BYTES, // regexp_contains(bytes, bytes) -> bool - // Converts bytes to string by replacing invalid UTF-8 characters with - // replacement char U+FFFD. - // FunctionSignatureId.FN_SAFE_CONVERT_BYTES_TO_STRING, - // Unicode normalization and casefolding functions. - // FunctionSignatureId.FN_NORMALIZE_STRING, // normalize(string [, mode]) -> string - // normalize_and_casefold(string [, mode]) -> string - // FunctionSignatureId.FN_NORMALIZE_AND_CASEFOLD_STRING, - // FunctionSignatureId.FN_TO_BASE64, // to_base64(bytes) -> string - // FunctionSignatureId.FN_FROM_BASE64, // from_base64(string) -> bytes - // FunctionSignatureId.FN_TO_HEX, // to_hex(bytes) -> string - // FunctionSignatureId.FN_FROM_HEX, // from_hex(string) -> bytes - // FunctionSignatureId.FN_TO_BASE32, // to_base32(bytes) -> string - // FunctionSignatureId.FN_FROM_BASE32, // from_base32(string) -> bytes - // to_code_points(string) -> array - // FunctionSignatureId.FN_TO_CODE_POINTS_STRING, - // to_code_points(bytes) -> array - // FunctionSignatureId.FN_TO_CODE_POINTS_BYTES, - // code_points_to_string(array) -> string - // FunctionSignatureId.FN_CODE_POINTS_TO_STRING, - // code_points_to_bytes(array) -> bytes - // FunctionSignatureId.FN_CODE_POINTS_TO_BYTES, - // FunctionSignatureId.FN_LPAD_BYTES, // lpad(bytes, int64[, bytes]) -> bytes - // FunctionSignatureId.FN_LPAD_STRING, // lpad(string, int64[, string]) -> string - // FunctionSignatureId.FN_RPAD_BYTES, // rpad(bytes, int64[, bytes]) -> bytes - // FunctionSignatureId.FN_RPAD_STRING, // rpad(string, int64[, string]) -> string - FunctionSignatureId.FN_LEFT_STRING, // left(string, int64) -> string - // FunctionSignatureId.FN_LEFT_BYTES, // left(bytes, int64) -> bytes - FunctionSignatureId.FN_RIGHT_STRING, // right(string, int64) -> string - // FunctionSignatureId.FN_RIGHT_BYTES, // right(bytes, int64) -> bytes - // FunctionSignatureId.FN_REPEAT_BYTES, // repeat(bytes, int64) -> bytes - // FunctionSignatureId.FN_REPEAT_STRING, // repeat(string, int64) -> string - FunctionSignatureId.FN_REVERSE_STRING, // reverse(string) -> string - // FunctionSignatureId.FN_REVERSE_BYTES, // reverse(bytes) -> bytes - // FunctionSignatureId.FN_SOUNDEX_STRING, // soundex(string) -> string - // FunctionSignatureId.FN_ASCII_STRING, // ASCII(string) -> int64 - // FunctionSignatureId.FN_ASCII_BYTES, // ASCII(bytes) -> int64 - // FunctionSignatureId.FN_TRANSLATE_STRING, // translate(string, string, string) -> string - // FunctionSignatureId.FN_TRANSLATE_BYTES, // soundex(bytes, bytes, bytes) -> bytes - // FunctionSignatureId.FN_INITCAP_STRING, // initcap(string[, string]) -> string - // FunctionSignatureId.FN_UNICODE_STRING, // unicode(string) -> int64 - // FunctionSignatureId.FN_CHR_STRING, // chr(int64) -> string - - // Control flow functions - FunctionSignatureId.FN_IF, // if - // Coalesce is used to express the output join column in FULL JOIN. - FunctionSignatureId.FN_COALESCE, // coalesce - FunctionSignatureId.FN_IFNULL, // ifnull - FunctionSignatureId.FN_NULLIF, // nullif - - // Time functions - FunctionSignatureId.FN_CURRENT_DATE, // current_date - FunctionSignatureId.FN_CURRENT_DATETIME, // current_datetime - FunctionSignatureId.FN_CURRENT_TIME, // current_time - FunctionSignatureId.FN_CURRENT_TIMESTAMP, // current_timestamp - FunctionSignatureId.FN_DATE_ADD_DATE, // date_add - FunctionSignatureId.FN_DATETIME_ADD, // datetime_add - FunctionSignatureId.FN_TIME_ADD, // time_add - FunctionSignatureId.FN_TIMESTAMP_ADD, // timestamp_add - FunctionSignatureId.FN_DATE_DIFF_DATE, // date_diff - FunctionSignatureId.FN_DATETIME_DIFF, // datetime_diff - FunctionSignatureId.FN_TIME_DIFF, // time_diff - FunctionSignatureId.FN_TIMESTAMP_DIFF, // timestamp_diff - FunctionSignatureId.FN_DATE_SUB_DATE, // date_sub - FunctionSignatureId.FN_DATETIME_SUB, // datetime_sub - FunctionSignatureId.FN_TIME_SUB, // time_sub - FunctionSignatureId.FN_TIMESTAMP_SUB, // timestamp_sub - FunctionSignatureId.FN_DATE_TRUNC_DATE, // date_trunc - FunctionSignatureId.FN_DATETIME_TRUNC, // datetime_trunc - FunctionSignatureId.FN_TIME_TRUNC, // time_trunc - FunctionSignatureId.FN_TIMESTAMP_TRUNC, // timestamp_trunc - FunctionSignatureId.FN_DATE_FROM_UNIX_DATE, // date_from_unix_date - FunctionSignatureId.FN_TIMESTAMP_FROM_INT64_SECONDS, // timestamp_seconds - FunctionSignatureId.FN_TIMESTAMP_FROM_INT64_MILLIS, // timestamp_millis - // FunctionSignatureId.FN_TIMESTAMP_FROM_INT64_MICROS, // timestamp_micros - FunctionSignatureId.FN_TIMESTAMP_FROM_UNIX_SECONDS_INT64, // timestamp_from_unix_seconds - // timestamp_from_unix_seconds - // FunctionSignatureId.FN_TIMESTAMP_FROM_UNIX_SECONDS_TIMESTAMP, - FunctionSignatureId.FN_TIMESTAMP_FROM_UNIX_MILLIS_INT64, // timestamp_from_unix_millis - // timestamp_from_unix_millis - // FunctionSignatureId.FN_TIMESTAMP_FROM_UNIX_MILLIS_TIMESTAMP, - // FunctionSignatureId.FN_TIMESTAMP_FROM_UNIX_MICROS_INT64, // timestamp_from_unix_micros - // timestamp_from_unix_micros - // FunctionSignatureId.FN_TIMESTAMP_FROM_UNIX_MICROS_TIMESTAMP, - FunctionSignatureId.FN_UNIX_DATE, // unix_date - FunctionSignatureId.FN_UNIX_SECONDS_FROM_TIMESTAMP, - FunctionSignatureId.FN_UNIX_MILLIS_FROM_TIMESTAMP, - // FunctionSignatureId.FN_UNIX_MICROS_FROM_TIMESTAMP, - FunctionSignatureId.FN_DATE_FROM_TIMESTAMP, // date - FunctionSignatureId.FN_DATE_FROM_DATETIME, // date - // FunctionSignatureId.FN_DATE_FROM_DATE, // date - // FunctionSignatureId.FN_DATE_FROM_STRING, // date - FunctionSignatureId.FN_DATE_FROM_YEAR_MONTH_DAY, // date - FunctionSignatureId.FN_TIMESTAMP_FROM_STRING, // timestamp - FunctionSignatureId.FN_TIMESTAMP_FROM_DATE, // timestamp - FunctionSignatureId.FN_TIMESTAMP_FROM_DATETIME, // timestamp - // FunctionSignatureId.FN_TIMESTAMP_FROM_TIMESTAMP, // timestamp - FunctionSignatureId.FN_TIME_FROM_HOUR_MINUTE_SECOND, // time - FunctionSignatureId.FN_TIME_FROM_TIMESTAMP, // time - FunctionSignatureId.FN_TIME_FROM_DATETIME, // time - // FunctionSignatureId.FN_TIME_FROM_TIME, // time - // FunctionSignatureId.FN_TIME_FROM_STRING, // time - FunctionSignatureId.FN_DATETIME_FROM_DATE_AND_TIME, // datetime - FunctionSignatureId.FN_DATETIME_FROM_YEAR_MONTH_DAY_HOUR_MINUTE_SECOND, // datetime - FunctionSignatureId.FN_DATETIME_FROM_TIMESTAMP, // datetime - FunctionSignatureId.FN_DATETIME_FROM_DATE, // datetime - // FunctionSignatureId.FN_DATETIME_FROM_DATETIME, // datetime - // FunctionSignatureId.FN_DATETIME_FROM_STRING, // datetime - // FunctionSignatureId.FN_STRING_FROM_DATE, // string - FunctionSignatureId.FN_STRING_FROM_TIMESTAMP, // string - // FunctionSignatureId.FN_STRING_FROM_DATETIME, // string - // FunctionSignatureId.FN_STRING_FROM_TIME, // string - - // Signatures for extracting date parts, taking a date/timestamp - // and the target date part as arguments. - FunctionSignatureId.FN_EXTRACT_FROM_DATE, // $extract - FunctionSignatureId.FN_EXTRACT_FROM_DATETIME, // $extract - FunctionSignatureId.FN_EXTRACT_FROM_TIME, // $extract - FunctionSignatureId.FN_EXTRACT_FROM_TIMESTAMP, // $extract - - // Signatures specific to extracting the DATE date part from a DATETIME or a - // TIMESTAMP. - FunctionSignatureId.FN_EXTRACT_DATE_FROM_DATETIME, // $extract_date - FunctionSignatureId.FN_EXTRACT_DATE_FROM_TIMESTAMP, // $extract_date - - // Signatures specific to extracting the TIME date part from a DATETIME or a - // TIMESTAMP. - FunctionSignatureId.FN_EXTRACT_TIME_FROM_DATETIME, // $extract_time - FunctionSignatureId.FN_EXTRACT_TIME_FROM_TIMESTAMP, // $extract_time - - // Signature specific to extracting the DATETIME date part from a TIMESTAMP. - FunctionSignatureId.FN_EXTRACT_DATETIME_FROM_TIMESTAMP, // $extract_datetime - - // Signature for formatting and parsing - FunctionSignatureId.FN_FORMAT_DATE, // format_date - FunctionSignatureId.FN_FORMAT_DATETIME, // format_datetime - FunctionSignatureId.FN_FORMAT_TIME, // format_time - FunctionSignatureId.FN_FORMAT_TIMESTAMP, // format_timestamp - FunctionSignatureId.FN_PARSE_DATE, // parse_date - FunctionSignatureId.FN_PARSE_DATETIME, // parse_datetime - FunctionSignatureId.FN_PARSE_TIME, // parse_time - FunctionSignatureId.FN_PARSE_TIMESTAMP, // parse_timestamp - // FunctionSignatureId.FN_LAST_DAY_DATE, // last_day date - // FunctionSignatureId.FN_LAST_DAY_DATETIME, // last_day datetime - - // Math functions - FunctionSignatureId.FN_ABS_INT64, // abs - FunctionSignatureId.FN_ABS_DOUBLE, // abs - FunctionSignatureId.FN_ABS_NUMERIC, // abs - // FunctionSignatureId.FN_ABS_BIGNUMERIC, // abs - FunctionSignatureId.FN_SIGN_INT64, // sign - FunctionSignatureId.FN_SIGN_DOUBLE, // sign - FunctionSignatureId.FN_SIGN_NUMERIC, // sign - // FunctionSignatureId.FN_SIGN_BIGNUMERIC, // sign - - FunctionSignatureId.FN_ROUND_DOUBLE, // round(double) -> double - FunctionSignatureId.FN_ROUND_NUMERIC, // round(numeric) -> numeric - // FunctionSignatureId.FN_ROUND_BIGNUMERIC, // round(bignumeric) -> bignumeric - FunctionSignatureId.FN_ROUND_WITH_DIGITS_DOUBLE, // round(double, int64) -> double - FunctionSignatureId.FN_ROUND_WITH_DIGITS_NUMERIC, // round(numeric, int64) -> numeric - // round(bignumeric, int64) -> bignumeric - // FunctionSignatureId.FN_ROUND_WITH_DIGITS_BIGNUMERIC, - FunctionSignatureId.FN_TRUNC_DOUBLE, // trunc(double) -> double - FunctionSignatureId.FN_TRUNC_NUMERIC, // trunc(numeric) -> numeric - // FunctionSignatureId.FN_TRUNC_BIGNUMERIC, // trunc(bignumeric) -> bignumeric - FunctionSignatureId.FN_TRUNC_WITH_DIGITS_DOUBLE, // trunc(double, int64) -> double - FunctionSignatureId.FN_TRUNC_WITH_DIGITS_NUMERIC, // trunc(numeric, int64) -> numeric - // trunc(bignumeric, int64) -> bignumeric - // FunctionSignatureId.FN_TRUNC_WITH_DIGITS_BIGNUMERIC, - FunctionSignatureId.FN_CEIL_DOUBLE, // ceil(double) -> double - FunctionSignatureId.FN_CEIL_NUMERIC, // ceil(numeric) -> numeric - // FunctionSignatureId.FN_CEIL_BIGNUMERIC, // ceil(bignumeric) -> bignumeric - FunctionSignatureId.FN_FLOOR_DOUBLE, // floor(double) -> double - FunctionSignatureId.FN_FLOOR_NUMERIC, // floor(numeric) -> numeric - // FunctionSignatureId.FN_FLOOR_BIGNUMERIC, // floor(bignumeric) -> bignumeric - - FunctionSignatureId.FN_MOD_INT64, // mod(int64, int64) -> int64 - FunctionSignatureId.FN_MOD_NUMERIC, // mod(numeric, numeric) -> numeric - // FunctionSignatureId.FN_MOD_BIGNUMERIC, // mod(bignumeric, bignumeric) -> bignumeric - FunctionSignatureId.FN_DIV_INT64, // div(int64, int64) -> int64 - FunctionSignatureId.FN_DIV_NUMERIC, // div(numeric, numeric) -> numeric - // FunctionSignatureId.FN_DIV_BIGNUMERIC, // div(bignumeric, bignumeric) -> bignumeric - - FunctionSignatureId.FN_IS_INF, // is_inf - FunctionSignatureId.FN_IS_NAN, // is_nan - FunctionSignatureId.FN_IEEE_DIVIDE_DOUBLE, // ieee_divide - FunctionSignatureId.FN_SAFE_DIVIDE_DOUBLE, // safe_divide - FunctionSignatureId.FN_SAFE_DIVIDE_NUMERIC, // safe_divide - // FunctionSignatureId.FN_SAFE_DIVIDE_BIGNUMERIC, // safe_divide - FunctionSignatureId.FN_SAFE_ADD_INT64, // safe_add - FunctionSignatureId.FN_SAFE_ADD_DOUBLE, // safe_add - FunctionSignatureId.FN_SAFE_ADD_NUMERIC, // safe_add - // FunctionSignatureId.FN_SAFE_ADD_BIGNUMERIC, // safe_add - FunctionSignatureId.FN_SAFE_SUBTRACT_INT64, // safe_subtract - FunctionSignatureId.FN_SAFE_SUBTRACT_DOUBLE, // safe_subtract - FunctionSignatureId.FN_SAFE_SUBTRACT_NUMERIC, // safe_subtract - // FunctionSignatureId.FN_SAFE_SUBTRACT_BIGNUMERIC, // safe_subtract - FunctionSignatureId.FN_SAFE_MULTIPLY_INT64, // safe_multiply - FunctionSignatureId.FN_SAFE_MULTIPLY_DOUBLE, // safe_multiply - FunctionSignatureId.FN_SAFE_MULTIPLY_NUMERIC, // safe_multiply - // FunctionSignatureId.FN_SAFE_MULTIPLY_BIGNUMERIC, // safe_multiply - FunctionSignatureId.FN_SAFE_UNARY_MINUS_INT64, // safe_negate - FunctionSignatureId.FN_SAFE_UNARY_MINUS_DOUBLE, // safe_negate - FunctionSignatureId.FN_SAFE_UNARY_MINUS_NUMERIC, // safe_negate - // FunctionSignatureId.FN_SAFE_UNARY_MINUS_BIGNUMERIC, // safe_negate - - // FunctionSignatureId.FN_GREATEST, // greatest - // FunctionSignatureId.FN_LEAST, // least - - FunctionSignatureId.FN_SQRT_DOUBLE, // sqrt - FunctionSignatureId.FN_SQRT_NUMERIC, // sqrt(numeric) -> numeric - // FunctionSignatureId.FN_SQRT_BIGNUMERIC, // sqrt(bignumeric) -> bignumeric - FunctionSignatureId.FN_POW_DOUBLE, // pow - FunctionSignatureId.FN_POW_NUMERIC, // pow(numeric, numeric) -> numeric - // FunctionSignatureId.FN_POW_BIGNUMERIC, // pow(bignumeric, bignumeric) -> bignumeric - FunctionSignatureId.FN_EXP_DOUBLE, // exp - FunctionSignatureId.FN_EXP_NUMERIC, // exp(numeric) -> numeric - // FunctionSignatureId.FN_EXP_BIGNUMERIC, // exp(bignumeric) -> bignumeric - FunctionSignatureId.FN_NATURAL_LOGARITHM_DOUBLE, // ln - FunctionSignatureId.FN_NATURAL_LOGARITHM_NUMERIC, // ln(numeric) -> numeric - // FunctionSignatureId.FN_NATURAL_LOGARITHM_BIGNUMERIC, // ln(bignumeric) -> bignumeric - FunctionSignatureId.FN_DECIMAL_LOGARITHM_DOUBLE, // log10 - FunctionSignatureId.FN_DECIMAL_LOGARITHM_NUMERIC, // log10(numeric) -> numeric - // FunctionSignatureId.FN_DECIMAL_LOGARITHM_BIGNUMERIC, // log10(bignumeric) -> bignumeric - FunctionSignatureId.FN_LOGARITHM_DOUBLE, // log - FunctionSignatureId.FN_LOGARITHM_NUMERIC, // log(numeric, numeric) -> numeric - // FunctionSignatureId.FN_LOGARITHM_BIGNUMERIC,//log(bignumeric, bignumeric) -> bignumeric - - FunctionSignatureId.FN_COS_DOUBLE, // cos - FunctionSignatureId.FN_COSH_DOUBLE, // cosh - FunctionSignatureId.FN_ACOS_DOUBLE, // acos - FunctionSignatureId.FN_ACOSH_DOUBLE, // acosh - FunctionSignatureId.FN_SIN_DOUBLE, // sin - FunctionSignatureId.FN_SINH_DOUBLE, // sinh - FunctionSignatureId.FN_ASIN_DOUBLE, // asin - FunctionSignatureId.FN_ASINH_DOUBLE, // asinh - FunctionSignatureId.FN_TAN_DOUBLE, // tan - FunctionSignatureId.FN_TANH_DOUBLE, // tanh - FunctionSignatureId.FN_ATAN_DOUBLE, // atan - FunctionSignatureId.FN_ATANH_DOUBLE, // atanh - FunctionSignatureId.FN_ATAN2_DOUBLE, // atan2 - - // Aggregate functions. - FunctionSignatureId.FN_ANY_VALUE, // any_value - FunctionSignatureId.FN_ARRAY_AGG, // array_agg - // FunctionSignatureId.FN_ARRAY_CONCAT_AGG, // array_concat_agg - FunctionSignatureId.FN_AVG_INT64, // avg - FunctionSignatureId.FN_AVG_DOUBLE, // avg - FunctionSignatureId.FN_AVG_NUMERIC, // avg - // FunctionSignatureId.FN_AVG_BIGNUMERIC, // avg - FunctionSignatureId.FN_COUNT, // count - FunctionSignatureId.FN_MAX, // max - FunctionSignatureId.FN_MIN, // min - FunctionSignatureId.FN_STRING_AGG_STRING, // string_agg(s) - FunctionSignatureId.FN_STRING_AGG_DELIM_STRING, // string_agg(s, delim_s) - FunctionSignatureId.FN_STRING_AGG_BYTES, // string_agg(b) - FunctionSignatureId.FN_STRING_AGG_DELIM_BYTES, // string_agg(b, delim_b) - FunctionSignatureId.FN_SUM_INT64, // sum - FunctionSignatureId.FN_SUM_DOUBLE, // sum - FunctionSignatureId.FN_SUM_NUMERIC, // sum - // FunctionSignatureId.FN_SUM_BIGNUMERIC, // sum - FunctionSignatureId.FN_BIT_AND_INT64, // bit_and - FunctionSignatureId.FN_BIT_OR_INT64, // bit_or - FunctionSignatureId.FN_BIT_XOR_INT64, // bit_xor - // FunctionSignatureId.FN_LOGICAL_AND, // logical_and - // FunctionSignatureId.FN_LOGICAL_OR, // logical_or - // Approximate aggregate functions. - // FunctionSignatureId.FN_APPROX_COUNT_DISTINCT, // approx_count_distinct - // FunctionSignatureId.FN_APPROX_QUANTILES, // approx_quantiles - // FunctionSignatureId.FN_APPROX_TOP_COUNT, // approx_top_count - // FunctionSignatureId.FN_APPROX_TOP_SUM_INT64, // approx_top_sum - // FunctionSignatureId.FN_APPROX_TOP_SUM_DOUBLE, // approx_top_sum - // FunctionSignatureId.FN_APPROX_TOP_SUM_NUMERIC, // approx_top_sum - // FunctionSignatureId.FN_APPROX_TOP_SUM_BIGNUMERIC, // approx_top_sum - - // Approximate count functions that expose the intermediate sketch. - // These are all found in the "hll_count.*" namespace. - // FunctionSignatureId.FN_HLL_COUNT_MERGE, // hll_count.merge(bytes) - // FunctionSignatureId.FN_HLL_COUNT_EXTRACT, // hll_count.extract(bytes), scalar - // FunctionSignatureId.FN_HLL_COUNT_INIT_INT64, // hll_count.init(int64) - // FunctionSignatureId.FN_HLL_COUNT_INIT_NUMERIC, // hll_count.init(numeric) - // FunctionSignatureId.FN_HLL_COUNT_INIT_BIGNUMERIC, // hll_count.init(bignumeric) - // FunctionSignatureId.FN_HLL_COUNT_INIT_STRING, // hll_count.init(string) - // FunctionSignatureId.FN_HLL_COUNT_INIT_BYTES, // hll_count.init(bytes) - // FunctionSignatureId.FN_HLL_COUNT_MERGE_PARTIAL, // hll_count.merge_partial(bytes) - - // Statistical aggregate functions. - // FunctionSignatureId.FN_CORR, // corr - // FunctionSignatureId.FN_CORR_NUMERIC, // corr - // FunctionSignatureId.FN_CORR_BIGNUMERIC, // corr - // FunctionSignatureId.FN_COVAR_POP, // covar_pop - // FunctionSignatureId.FN_COVAR_POP_NUMERIC, // covar_pop - // FunctionSignatureId.FN_COVAR_POP_BIGNUMERIC, // covar_pop - // FunctionSignatureId.FN_COVAR_SAMP, // covar_samp - // FunctionSignatureId.FN_COVAR_SAMP_NUMERIC, // covar_samp - // FunctionSignatureId.FN_COVAR_SAMP_BIGNUMERIC, // covar_samp - // FunctionSignatureId.FN_STDDEV_POP, // stddev_pop - // FunctionSignatureId.FN_STDDEV_POP_NUMERIC, // stddev_pop - // FunctionSignatureId.FN_STDDEV_POP_BIGNUMERIC, // stddev_pop - // FunctionSignatureId.FN_STDDEV_SAMP, // stddev_samp - // FunctionSignatureId.FN_STDDEV_SAMP_NUMERIC, // stddev_samp - // FunctionSignatureId.FN_STDDEV_SAMP_BIGNUMERIC, // stddev_samp - // FunctionSignatureId.FN_VAR_POP, // var_pop - // FunctionSignatureId.FN_VAR_POP_NUMERIC, // var_pop - // FunctionSignatureId.FN_VAR_POP_BIGNUMERIC, // var_pop - // FunctionSignatureId.FN_VAR_SAMP, // var_samp - // FunctionSignatureId.FN_VAR_SAMP_NUMERIC, // var_samp - // FunctionSignatureId.FN_VAR_SAMP_BIGNUMERIC, // var_samp - - FunctionSignatureId.FN_COUNTIF, // countif - - // Approximate quantiles functions that produce or consume intermediate - // sketches. All found in the "kll_quantiles.*" namespace. - // FunctionSignatureId.FN_KLL_QUANTILES_INIT_INT64, - // FunctionSignatureId.FN_KLL_QUANTILES_INIT_DOUBLE, - // FunctionSignatureId.FN_KLL_QUANTILES_MERGE_PARTIAL, - // FunctionSignatureId.FN_KLL_QUANTILES_MERGE_INT64, - // FunctionSignatureId.FN_KLL_QUANTILES_MERGE_DOUBLE, - // FunctionSignatureId.FN_KLL_QUANTILES_EXTRACT_INT64, // scalar - // FunctionSignatureId.FN_KLL_QUANTILES_EXTRACT_DOUBLE, // scalar - // FunctionSignatureId.FN_KLL_QUANTILES_MERGE_POINT_INT64, - // FunctionSignatureId.FN_KLL_QUANTILES_MERGE_POINT_DOUBLE, - // FunctionSignatureId.FN_KLL_QUANTILES_EXTRACT_POINT_INT64, // scalar - // FunctionSignatureId.FN_KLL_QUANTILES_EXTRACT_POINT_DOUBLE, // scalar - - // Analytic functions. - // FunctionSignatureId.FN_DENSE_RANK, // dense_rank - // FunctionSignatureId.FN_RANK, // rank - // FunctionSignatureId.FN_ROW_NUMBER, // row_number - // FunctionSignatureId.FN_PERCENT_RANK, // percent_rank - // FunctionSignatureId.FN_CUME_DIST, // cume_dist - // FunctionSignatureId.FN_NTILE, // ntile - // FunctionSignatureId.FN_LEAD, // lead - // FunctionSignatureId.FN_LAG, // lag - // FunctionSignatureId.FN_FIRST_VALUE, // first_value - // FunctionSignatureId.FN_LAST_VALUE, // last_value - // FunctionSignatureId.FN_NTH_VALUE, // nth_value - // FunctionSignatureId.FN_PERCENTILE_CONT, // percentile_cont - // FunctionSignatureId.FN_PERCENTILE_CONT_NUMERIC, // percentile_cont - // FunctionSignatureId.FN_PERCENTILE_CONT_BIGNUMERIC, // percentile_cont - // FunctionSignatureId.FN_PERCENTILE_DISC, // percentile_disc - // FunctionSignatureId.FN_PERCENTILE_DISC_NUMERIC, // percentile_disc - // FunctionSignatureId.FN_PERCENTILE_DISC_BIGNUMERIC, // percentile_disc - - // Misc functions. - // FunctionSignatureId.FN_BIT_CAST_INT64_TO_INT64, // bit_cast_to_int64(int64) - - // FunctionSignatureId.FN_SESSION_USER, // session_user - - // FunctionSignatureId.FN_GENERATE_ARRAY_INT64, // generate_array(int64) - // FunctionSignatureId.FN_GENERATE_ARRAY_NUMERIC, // generate_array(numeric) - // FunctionSignatureId.FN_GENERATE_ARRAY_BIGNUMERIC, // generate_array(bignumeric) - // FunctionSignatureId.FN_GENERATE_ARRAY_DOUBLE, // generate_array(double) - // FunctionSignatureId.FN_GENERATE_DATE_ARRAY, // generate_date_array(date) - // FunctionSignatureId.FN_GENERATE_TIMESTAMP_ARRAY, // generate_timestamp_array(timestamp) - - // FunctionSignatureId.FN_ARRAY_REVERSE, // array_reverse(array) -> array - - // FunctionSignatureId.FN_RANGE_BUCKET, // range_bucket(T, array) -> int64 - - // FunctionSignatureId.FN_RAND // rand() -> double - // FunctionSignatureId.FN_GENERATE_UUID, // generate_uuid() -> string - - FunctionSignatureId.FN_JSON_EXTRACT, // json_extract(string, string) - // FunctionSignatureId.FN_JSON_EXTRACT_JSON, // json_extract(json, string) -> json - FunctionSignatureId.FN_JSON_EXTRACT_SCALAR, // json_extract_scalar(string, string) - // json_extract_scalar(json, string) -> string - // FunctionSignatureId.FN_JSON_EXTRACT_SCALAR_JSON, - // json_extract_array(string[, string]) -> array - FunctionSignatureId.FN_JSON_EXTRACT_ARRAY, - FunctionSignatureId.FN_TO_JSON_STRING, // to_json_string(any[, bool]) -> string - FunctionSignatureId.FN_JSON_QUERY, // json_query(string, string) -> string - // FunctionSignatureId.FN_JSON_QUERY_JSON, // json_query(json, string) -> json - FunctionSignatureId.FN_JSON_VALUE, // json_value(string, string) -> string - // FunctionSignatureId.FN_JSON_VALUE_JSON, // json_value(json, string) -> json - - // Net functions. These are all found in the "net.*" namespace. - // FunctionSignatureId.FN_NET_FORMAT_IP, - // FunctionSignatureId.FN_NET_PARSE_IP, - // FunctionSignatureId.FN_NET_FORMAT_PACKED_IP, - // FunctionSignatureId.FN_NET_PARSE_PACKED_IP, - // FunctionSignatureId.FN_NET_IP_IN_NET, - // FunctionSignatureId.FN_NET_MAKE_NET, - // FunctionSignatureId.FN_NET_HOST, // net.host(string) - // FunctionSignatureId.FN_NET_REG_DOMAIN, // net.reg_domain(string) - // FunctionSignatureId.FN_NET_PUBLIC_SUFFIX, // net.public_suffix(string) - // FunctionSignatureId.FN_NET_IP_FROM_STRING, // net.ip_from_string(string) - // FunctionSignatureId.FN_NET_SAFE_IP_FROM_STRING, // net.safe_ip_from_string(string) - // FunctionSignatureId.FN_NET_IP_TO_STRING, // net.ip_to_string(bytes) - // FunctionSignatureId.FN_NET_IP_NET_MASK, // net.ip_net_mask(int64, int64) - // FunctionSignatureId.FN_NET_IP_TRUNC, // net.ip_net_mask(bytes, int64) - // FunctionSignatureId.FN_NET_IPV4_FROM_INT64, // net.ipv4_from_int64(int64) - // FunctionSignatureId.FN_NET_IPV4_TO_INT64, // net.ipv4_to_int64(bytes) - - // Hashing functions. - FunctionSignatureId.FN_MD5_BYTES, // md5(bytes) - FunctionSignatureId.FN_MD5_STRING, // md5(string) - FunctionSignatureId.FN_SHA1_BYTES, // sha1(bytes) - FunctionSignatureId.FN_SHA1_STRING, // sha1(string) - FunctionSignatureId.FN_SHA256_BYTES, // sha256(bytes) - FunctionSignatureId.FN_SHA256_STRING, // sha256(string) - FunctionSignatureId.FN_SHA512_BYTES, // sha512(bytes) - FunctionSignatureId.FN_SHA512_STRING // sha512(string) - - // Fingerprinting functions - // FunctionSignatureId.FN_FARM_FINGERPRINT_BYTES, // farm_fingerprint(bytes) -> int64 - // FunctionSignatureId.FN_FARM_FINGERPRINT_STRING, // farm_fingerprint(string) -> int64 - - // Keyset management, encryption, and decryption functions - // Requires that FEATURE_ENCRYPTION is enabled. - // FunctionSignatureId.FN_KEYS_NEW_KEYSET, // keys.new_keyset(string) - // keys.add_key_from_raw_bytes(bytes, string, bytes) - // FunctionSignatureId.FN_KEYS_ADD_KEY_FROM_RAW_BYTES, - // FunctionSignatureId.FN_KEYS_ROTATE_KEYSET, // keys.rotate_keyset(bytes, string) - // FunctionSignatureId.FN_KEYS_KEYSET_LENGTH, // keys.keyset_length(bytes) - // FunctionSignatureId.FN_KEYS_KEYSET_TO_JSON, // keys.keyset_to_json(bytes) - // FunctionSignatureId.FN_KEYS_KEYSET_FROM_JSON, // keys.keyset_from_json(string) - // FunctionSignatureId.FN_AEAD_ENCRYPT_STRING, // aead.encrypt(bytes, string, string) - // FunctionSignatureId.FN_AEAD_ENCRYPT_BYTES, // aead.encrypt(bytes, bytes, bytes) - // FunctionSignatureId.FN_AEAD_DECRYPT_STRING,// aead.decrypt_string(bytes, bytes, string) - // FunctionSignatureId.FN_AEAD_DECRYPT_BYTES, // aead.decrypt_bytes(bytes, bytes, bytes) - // FunctionSignatureId.FN_KMS_ENCRYPT_STRING, // kms.encrypt(string, string) - // FunctionSignatureId.FN_KMS_ENCRYPT_BYTES, // kms.encrypt(string, bytes) - // FunctionSignatureId.FN_KMS_DECRYPT_STRING, // kms.decrypt_string(string, bytes) - // FunctionSignatureId.FN_KMS_DECRYPT_BYTES, // kms.decrypt_bytes(string, bytes) - - // ST_ family of functions (Geography related) - // Constructors - // FunctionSignatureId.FN_ST_GEOG_POINT, - // FunctionSignatureId.FN_ST_MAKE_LINE, - // FunctionSignatureId.FN_ST_MAKE_LINE_ARRAY, - // FunctionSignatureId.FN_ST_MAKE_POLYGON, - // FunctionSignatureId.FN_ST_MAKE_POLYGON_ORIENTED, - // Transformations - // FunctionSignatureId.FN_ST_INTERSECTION, - // FunctionSignatureId.FN_ST_UNION, - // FunctionSignatureId.FN_ST_UNION_ARRAY, - // FunctionSignatureId.FN_ST_DIFFERENCE, - // FunctionSignatureId.FN_ST_UNARY_UNION, - // FunctionSignatureId.FN_ST_CENTROID, - // FunctionSignatureId.FN_ST_BUFFER, - // FunctionSignatureId.FN_ST_BUFFER_WITH_TOLERANCE, - // FunctionSignatureId.FN_ST_SIMPLIFY, - // FunctionSignatureId.FN_ST_SNAP_TO_GRID, - // FunctionSignatureId.FN_ST_CLOSEST_POINT, - // FunctionSignatureId.FN_ST_BOUNDARY, - // FunctionSignatureId.FN_ST_CONVEXHULL, - // Predicates - // FunctionSignatureId.FN_ST_EQUALS, - // FunctionSignatureId.FN_ST_INTERSECTS, - // FunctionSignatureId.FN_ST_CONTAINS, - // FunctionSignatureId.FN_ST_COVERS, - // FunctionSignatureId.FN_ST_DISJOINT, - // FunctionSignatureId.FN_ST_INTERSECTS_BOX, - // FunctionSignatureId.FN_ST_DWITHIN, - // FunctionSignatureId.FN_ST_WITHIN, - // FunctionSignatureId.FN_ST_COVEREDBY, - // FunctionSignatureId.FN_ST_TOUCHES, - // Accessors - // FunctionSignatureId.FN_ST_IS_EMPTY, - // FunctionSignatureId.FN_ST_IS_COLLECTION, - // FunctionSignatureId.FN_ST_DIMENSION, - // FunctionSignatureId.FN_ST_NUM_POINTS, - // FunctionSignatureId.FN_ST_DUMP, - // Measures - // FunctionSignatureId.FN_ST_LENGTH, - // FunctionSignatureId.FN_ST_PERIMETER, - // FunctionSignatureId.FN_ST_AREA, - // FunctionSignatureId.FN_ST_DISTANCE, - // FunctionSignatureId.FN_ST_MAX_DISTANCE, - // Parsers/formatters - // FunctionSignatureId.FN_ST_GEOG_FROM_TEXT, - // FunctionSignatureId.FN_ST_GEOG_FROM_KML, - // FunctionSignatureId.FN_ST_GEOG_FROM_GEO_JSON, - // FunctionSignatureId.FN_ST_GEOG_FROM_WKB, - // FunctionSignatureId.FN_ST_AS_TEXT, - // FunctionSignatureId.FN_ST_AS_KML, - // FunctionSignatureId.FN_ST_AS_GEO_JSON, - // FunctionSignatureId.FN_ST_AS_BINARY, - // FunctionSignatureId.FN_ST_GEOHASH, - // FunctionSignatureId.FN_ST_GEOG_POINT_FROM_GEOHASH, - // Aggregate functions - // FunctionSignatureId.FN_ST_UNION_AGG, - // FunctionSignatureId.FN_ST_ACCUM, - // FunctionSignatureId.FN_ST_CENTROID_AGG, - // FunctionSignatureId.FN_ST_NEAREST_NEIGHBORS, - // Other geography functions - // FunctionSignatureId.FN_ST_X, - // FunctionSignatureId.FN_ST_Y, - // FunctionSignatureId.FN_ST_CLUSTERDBSCAN, - - // Array functions. - // FunctionSignatureId.FN_FLATTEN, // flatten(array path) -> array - // FunctionSignatureId.FN_ARRAY_AT_OFFSET, // $array_at_offset - // FunctionSignatureId.FN_ARRAY_AT_ORDINAL, // $array_at_ordinal - // FunctionSignatureId.FN_ARRAY_CONCAT, // array_concat(repeated array) -> array - // FunctionSignatureId.FN_ARRAY_CONCAT_OP, // array_concat(array, array) -> array - // FunctionSignatureId.FN_ARRAY_LENGTH, // array_length(array) -> int64 - // array_to_string(array, bytes[, bytes]) -> bytes - // FunctionSignatureId.FN_ARRAY_TO_BYTES, - // array_to_string(array, string[, string]) -> string - // FunctionSignatureId.FN_ARRAY_TO_STRING, - // FunctionSignatureId.FN_MAKE_ARRAY, // $make_array - // FunctionSignatureId.FN_SAFE_ARRAY_AT_OFFSET, // $safe_array_at_offset - // FunctionSignatureId.FN_SAFE_ARRAY_AT_ORDINAL, // $safe_array_at_ordinal - // FunctionSignatureId.FN_ARRAY_IS_DISTINCT, // array_is_distinct(array) -> bool - // FunctionSignatureId.FN_PROTO_MAP_AT_KEY, // $proto_map_at_key - // FunctionSignatureId.FN_SAFE_PROTO_MAP_AT_KEY, // $safe_proto_map_at_key - ); -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java deleted file mode 100644 index d077ef40ee64..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.SimpleTable; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; -import org.apache.beam.sdk.extensions.sql.impl.TableName; -import org.apache.beam.sdk.extensions.sql.meta.CustomTableResolver; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Schema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Table; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; - -/** Utility methods to resolve a table, given a top-level Calcite schema and a table path. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class TableResolution { - - /** - * Resolves {@code tablePath} according to the given {@code schemaPlus}. - * - *

{@code tablePath} represents a structured table name where the last component is the name of - * the table and all the preceding components are sub-schemas / namespaces within {@code - * schemaPlus}. - */ - public static Table resolveCalciteTable(SchemaPlus schemaPlus, List tablePath) { - Schema subSchema = schemaPlus; - - // subSchema.getSubschema() for all except last - for (int i = 0; i < tablePath.size() - 1; i++) { - subSchema = subSchema.getSubSchema(tablePath.get(i)); - if (subSchema == null) { - throw new IllegalStateException( - String.format( - "While resolving table path %s, no sub-schema found for component %s (\"%s\")", - tablePath, i, tablePath.get(i))); - } - } - - // for the final one call getTable() - return subSchema.getTable(Iterables.getLast(tablePath)); - } - - /** - * Registers tables that will be resolved during query analysis, so table providers can eagerly - * pre-load metadata. - */ - // TODO(https://issues.apache.org/jira/browse/BEAM-8817): share this logic between dialects - public static void registerTables(SchemaPlus schemaPlus, List> tables) { - Schema defaultSchema = CalciteSchema.from(schemaPlus).schema; - if (defaultSchema instanceof BeamCalciteSchema - && ((BeamCalciteSchema) defaultSchema).getTableProvider() instanceof CustomTableResolver) { - ((CustomTableResolver) ((BeamCalciteSchema) defaultSchema).getTableProvider()) - .registerKnownTableNames( - tables.stream().map(TableName::create).collect(Collectors.toList())); - } - - for (String subSchemaName : schemaPlus.getSubSchemaNames()) { - Schema subSchema = CalciteSchema.from(schemaPlus.getSubSchema(subSchemaName)).schema; - - if (subSchema instanceof BeamCalciteSchema - && ((BeamCalciteSchema) subSchema).getTableProvider() instanceof CustomTableResolver) { - ((CustomTableResolver) ((BeamCalciteSchema) subSchema).getTableProvider()) - .registerKnownTableNames( - tables.stream().map(TableName::create).collect(Collectors.toList())); - } - } - } - - /** - * Data class to store simple table, its full path (excluding top-level schema), and top-level - * schema. - */ - static class SimpleTableWithPath { - - SimpleTable table; - List path; - - static SimpleTableWithPath of(List path) { - SimpleTableWithPath tableWithPath = new SimpleTableWithPath(); - tableWithPath.table = new SimpleTable(Iterables.getLast(path)); - tableWithPath.path = path; - return tableWithPath; - } - - SimpleTable getTable() { - return table; - } - - List getPath() { - return path; - } - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java deleted file mode 100644 index c8a1097b54a2..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * 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.AnalyzerOptions; -import com.google.zetasql.LanguageOptions; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedQueryStmt; -import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner.QueryParameters; -import org.apache.beam.sdk.extensions.sql.zetasql.translation.ConversionContext; -import org.apache.beam.sdk.extensions.sql.zetasql.translation.ExpressionConverter; -import org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelRoot; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexExecutor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.FrameworkConfig; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Frameworks; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Program; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Util; - -/** ZetaSQLPlannerImpl. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class ZetaSQLPlannerImpl { - private final SchemaPlus defaultSchemaPlus; - - // variables that are used in Calcite's planner. - private final FrameworkConfig config; - private RelOptPlanner planner; - private JavaTypeFactory typeFactory; - private final RexExecutor executor; - private final ImmutableList programs; - - private String defaultTimezone = "UTC"; // choose UTC (offset 00:00) unless explicitly set - - ZetaSQLPlannerImpl(FrameworkConfig config) { - this.config = config; - this.executor = config.getExecutor(); - this.programs = config.getPrograms(); - - Frameworks.withPlanner( - (cluster, relOptSchema, rootSchema) -> { - Util.discard(rootSchema); // use our own defaultSchema - typeFactory = (JavaTypeFactory) cluster.getTypeFactory(); - planner = cluster.getPlanner(); - planner.setExecutor(executor); - return null; - }, - config); - - this.defaultSchemaPlus = config.getDefaultSchema(); - } - - public RelRoot rel(String sql, QueryParameters params) { - RelOptCluster cluster = RelOptCluster.create(planner, new RexBuilder(typeFactory)); - AnalyzerOptions options = SqlAnalyzer.getAnalyzerOptions(params, defaultTimezone); - BeamZetaSqlCatalog catalog = - BeamZetaSqlCatalog.create( - defaultSchemaPlus, (JavaTypeFactory) cluster.getTypeFactory(), options); - - // Set up table providers that need to be pre-registered - SqlAnalyzer analyzer = new SqlAnalyzer(); - List> tables = analyzer.extractTableNames(sql, options); - TableResolution.registerTables(this.defaultSchemaPlus, tables); - QueryTrait trait = new QueryTrait(); - catalog.addTables(tables, trait); - - ResolvedQueryStmt statement = analyzer.analyzeQuery(sql, options, catalog); - - ExpressionConverter expressionConverter = - new ExpressionConverter(cluster, params, catalog.getUserFunctionDefinitions()); - ConversionContext context = ConversionContext.of(config, expressionConverter, cluster, trait); - - RelNode convertedNode = QueryStatementConverter.convertRootQuery(context, statement); - return RelRoot.of(convertedNode, SqlKind.ALL); - } - - RelNode transform(int i, RelTraitSet relTraitSet, RelNode relNode) { - Program program = programs.get(i); - return program.run(planner, relNode, relTraitSet, ImmutableList.of(), ImmutableList.of()); - } - - String getDefaultTimezone() { - return defaultTimezone; - } - - void setDefaultTimezone(String timezone) { - defaultTimezone = timezone; - } - - static LanguageOptions getLanguageOptions() { - return SqlAnalyzer.baseAnalyzerOptions().getLanguageOptions(); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java deleted file mode 100644 index 8210afed8119..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java +++ /dev/null @@ -1,256 +0,0 @@ -/* - * 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.LanguageOptions; -import com.google.zetasql.Value; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions; -import org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner.NonCumulativeCostImpl; -import org.apache.beam.sdk.extensions.sql.impl.JdbcConnection; -import org.apache.beam.sdk.extensions.sql.impl.ParseException; -import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner; -import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException; -import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel; -import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelMetadataQuery; -import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets; -import org.apache.beam.sdk.extensions.sql.impl.planner.RelMdNodeStats; -import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; -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.extensions.sql.impl.rule.BeamCalcRule; -import org.apache.beam.sdk.extensions.sql.impl.rule.BeamUncollectRule; -import org.apache.beam.sdk.extensions.sql.impl.rule.BeamUnnestRule; -import org.apache.beam.sdk.extensions.sql.zetasql.unnest.BeamZetaSqlUncollectRule; -import org.apache.beam.sdk.extensions.sql.zetasql.unnest.BeamZetaSqlUnnestRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionConfig; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.ConventionTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.prepare.CalciteCatalogReader; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelRoot; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.ChainedRelMetadataProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rules.JoinCommuteRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperatorTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParser; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserImplFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.util.SqlOperatorTables; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.FrameworkConfig; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Frameworks; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSets; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** ZetaSQLQueryPlanner. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class ZetaSQLQueryPlanner implements QueryPlanner { - public static final Collection DEFAULT_CALC = - ImmutableList.builder().add(BeamZetaSqlCalcSplittingRule.INSTANCE).build(); - - private static final Logger LOG = LoggerFactory.getLogger(ZetaSQLQueryPlanner.class); - - private final ZetaSQLPlannerImpl plannerImpl; - - public ZetaSQLQueryPlanner(FrameworkConfig config) { - plannerImpl = new ZetaSQLPlannerImpl(config); - } - - /** - * Called by {@link org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv}.instantiatePlanner() - * reflectively. - */ - public ZetaSQLQueryPlanner(JdbcConnection jdbcConnection, Collection ruleSets) { - LOG.warn( - "Beam ZetaSQL has been deprecated. See https://github.com/apache/beam/issues/34423 for details."); - plannerImpl = - new ZetaSQLPlannerImpl( - defaultConfig(jdbcConnection, modifyRuleSetsForZetaSql(ruleSets, DEFAULT_CALC))); - setDefaultTimezone( - jdbcConnection - .getPipelineOptions() - .as(BeamSqlPipelineOptions.class) - .getZetaSqlDefaultTimezone()); - } - - public static final Factory FACTORY = ZetaSQLQueryPlanner::new; - - public static Collection getZetaSqlRuleSets() { - return modifyRuleSetsForZetaSql(BeamRuleSets.getRuleSets(), DEFAULT_CALC); - } - - public static Collection getZetaSqlRuleSets(Collection calc) { - return modifyRuleSetsForZetaSql(BeamRuleSets.getRuleSets(), calc); - } - - private static Collection modifyRuleSetsForZetaSql( - Collection ruleSets, Collection calc) { - ImmutableList.Builder ret = ImmutableList.builder(); - for (RuleSet ruleSet : ruleSets) { - ImmutableList.Builder bd = ImmutableList.builder(); - for (RelOptRule rule : ruleSet) { - // TODO[https://github.com/apache/beam/issues/20077]: Fix join re-ordering for ZetaSQL - // planner. Currently join re-ordering - // requires the JoinCommuteRule, which doesn't work without struct flattening. - if (rule instanceof JoinCommuteRule) { - continue; - } else if (rule instanceof BeamCalcRule) { - bd.addAll(calc); - } else if (rule instanceof BeamUnnestRule) { - bd.add(BeamZetaSqlUnnestRule.INSTANCE); - } else if (rule instanceof BeamUncollectRule) { - bd.add(BeamZetaSqlUncollectRule.INSTANCE); - } else { - bd.add(rule); - } - } - bd.add(BeamZetaSqlCalcMergeRule.INSTANCE); - ret.add(RuleSets.ofList(bd.build())); - } - return ret.build(); - } - - public String getDefaultTimezone() { - return plannerImpl.getDefaultTimezone(); - } - - public void setDefaultTimezone(String timezone) { - plannerImpl.setDefaultTimezone(timezone); - } - - public static LanguageOptions getLanguageOptions() { - return ZetaSQLPlannerImpl.getLanguageOptions(); - } - - public BeamRelNode convertToBeamRel(String sqlStatement) { - return convertToBeamRel(sqlStatement, QueryParameters.ofNone()); - } - - public BeamRelNode convertToBeamRel(String sqlStatement, Map queryParams) - throws ParseException, SqlConversionException { - return convertToBeamRel(sqlStatement, QueryParameters.ofNamed(queryParams)); - } - - public BeamRelNode convertToBeamRel(String sqlStatement, List queryParams) - throws ParseException, SqlConversionException { - return convertToBeamRel(sqlStatement, QueryParameters.ofPositional(queryParams)); - } - - @Override - public BeamRelNode convertToBeamRel(String sqlStatement, QueryParameters queryParameters) - throws ParseException, SqlConversionException { - return convertToBeamRelInternal(sqlStatement, queryParameters); - } - - @Override - public SqlNode parse(String sqlStatement) throws ParseException { - throw new UnsupportedOperationException( - String.format( - "%s.parse(String) is not implemented and should need be called", - this.getClass().getCanonicalName())); - } - - private BeamRelNode convertToBeamRelInternal(String sql, QueryParameters queryParams) { - RelRoot root = plannerImpl.rel(sql, queryParams); - RelTraitSet desiredTraits = - root.rel - .getTraitSet() - .replace(BeamLogicalConvention.INSTANCE) - .replace(root.collation) - .simplify(); - // beam physical plan - root.rel - .getCluster() - .setMetadataProvider( - ChainedRelMetadataProvider.of( - ImmutableList.of( - NonCumulativeCostImpl.SOURCE, - RelMdNodeStats.SOURCE, - root.rel.getCluster().getMetadataProvider()))); - - root.rel.getCluster().setMetadataQuerySupplier(BeamRelMetadataQuery::instance); - - RelMetadataQuery.THREAD_PROVIDERS.set( - JaninoRelMetadataProvider.of(root.rel.getCluster().getMetadataProvider())); - root.rel.getCluster().invalidateMetadataQuery(); - try { - BeamRelNode beamRelNode = (BeamRelNode) plannerImpl.transform(0, desiredTraits, root.rel); - LOG.info("BEAMPlan>\n{}", BeamSqlRelUtils.explainLazily(beamRelNode)); - return beamRelNode; - } catch (RelOptPlanner.CannotPlanException e) { - throw new SqlConversionException("Failed to produce plan for query " + sql, e); - } - } - - @SuppressWarnings({ - "rawtypes", // Frameworks.ConfigBuilder.traitDefs has method signature of raw type - }) - private static FrameworkConfig defaultConfig( - JdbcConnection connection, Collection ruleSets) { - final CalciteConnectionConfig config = connection.config(); - final SqlParser.ConfigBuilder parserConfig = - SqlParser.configBuilder() - .setQuotedCasing(config.quotedCasing()) - .setUnquotedCasing(config.unquotedCasing()) - .setQuoting(config.quoting()) - .setConformance(config.conformance()) - .setCaseSensitive(config.caseSensitive()); - final SqlParserImplFactory parserFactory = - config.parserFactory(SqlParserImplFactory.class, null); - if (parserFactory != null) { - parserConfig.setParserFactory(parserFactory); - } - - final SchemaPlus schema = connection.getRootSchema(); - final SchemaPlus defaultSchema = connection.getCurrentSchemaPlus(); - - final ImmutableList traitDefs = ImmutableList.of(ConventionTraitDef.INSTANCE); - - final CalciteCatalogReader catalogReader = - new CalciteCatalogReader( - CalciteSchema.from(schema), - ImmutableList.of(defaultSchema.getName()), - connection.getTypeFactory(), - connection.config()); - final SqlOperatorTable opTab0 = - connection.config().fun(SqlOperatorTable.class, SqlStdOperatorTable.instance()); - - return Frameworks.newConfigBuilder() - .parserConfig(parserConfig.build()) - .defaultSchema(defaultSchema) - .traitDefs(traitDefs) - .ruleSets(ruleSets.toArray(new RuleSet[0])) - .costFactory(BeamCostModel.FACTORY) - .typeSystem(connection.getTypeFactory().getTypeSystem()) - .operatorTable(SqlOperatorTables.chain(opTab0, catalogReader)) - .build(); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java deleted file mode 100644 index 318da1c41442..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java +++ /dev/null @@ -1,296 +0,0 @@ -/* - * 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.protobuf.ByteString; -import com.google.zetasql.ArrayType; -import com.google.zetasql.StructType; -import com.google.zetasql.StructType.StructField; -import com.google.zetasql.Type; -import com.google.zetasql.TypeFactory; -import com.google.zetasql.Value; -import com.google.zetasql.ZetaSQLType.TypeKind; -import java.math.BigDecimal; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.Schema.Field; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.logicaltypes.DateTime; -import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.math.LongMath; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Instant; - -/** - * Utility methods for ZetaSQL <=> Beam translation. - * - *

Unsupported ZetaSQL types: INT32, UINT32, UINT64, FLOAT, ENUM, PROTO, GEOGRAPHY - */ -@Internal -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public final class ZetaSqlBeamTranslationUtils { - - private static final long MICROS_PER_MILLI = 1000L; - - private ZetaSqlBeamTranslationUtils() {} - - // Type conversion: Beam => ZetaSQL - public static Type toZetaSqlType(FieldType fieldType) { - switch (fieldType.getTypeName()) { - case INT64: - return TypeFactory.createSimpleType(TypeKind.TYPE_INT64); - case DOUBLE: - return TypeFactory.createSimpleType(TypeKind.TYPE_DOUBLE); - case BOOLEAN: - return TypeFactory.createSimpleType(TypeKind.TYPE_BOOL); - case STRING: - return TypeFactory.createSimpleType(TypeKind.TYPE_STRING); - case BYTES: - return TypeFactory.createSimpleType(TypeKind.TYPE_BYTES); - case DECIMAL: - return TypeFactory.createSimpleType(TypeKind.TYPE_NUMERIC); - case DATETIME: - // TODO[https://github.com/apache/beam/issues/20364]: Mapping TIMESTAMP to a Beam - // LogicalType instead? - return TypeFactory.createSimpleType(TypeKind.TYPE_TIMESTAMP); - case LOGICAL_TYPE: - String identifier = fieldType.getLogicalType().getIdentifier(); - if (SqlTypes.DATE.getIdentifier().equals(identifier)) { - return TypeFactory.createSimpleType(TypeKind.TYPE_DATE); - } else if (SqlTypes.TIME.getIdentifier().equals(identifier)) { - return TypeFactory.createSimpleType(TypeKind.TYPE_TIME); - } else if (SqlTypes.DATETIME.getIdentifier().equals(identifier)) { - return TypeFactory.createSimpleType(TypeKind.TYPE_DATETIME); - } else { - throw new UnsupportedOperationException("Unknown Beam logical type: " + identifier); - } - case ARRAY: - return toZetaSqlArrayType(fieldType.getCollectionElementType()); - case ROW: - return toZetaSqlStructType(fieldType.getRowSchema()); - default: - throw new UnsupportedOperationException( - "Unknown Beam fieldType: " + fieldType.getTypeName()); - } - } - - private static ArrayType toZetaSqlArrayType(FieldType elementFieldType) { - return TypeFactory.createArrayType(toZetaSqlType(elementFieldType)); - } - - public static StructType toZetaSqlStructType(Schema schema) { - return TypeFactory.createStructType( - schema.getFields().stream() - .map(f -> new StructField(f.getName(), toZetaSqlType(f.getType()))) - .collect(Collectors.toList())); - } - - // Value conversion: Beam => ZetaSQL - public static Value toZetaSqlValue(@Nullable Object object, FieldType fieldType) { - if (object == null) { - return Value.createNullValue(toZetaSqlType(fieldType)); - } - switch (fieldType.getTypeName()) { - case INT64: - return Value.createInt64Value((Long) object); - case DOUBLE: - return Value.createDoubleValue((Double) object); - case BOOLEAN: - return Value.createBoolValue((Boolean) object); - case STRING: - return Value.createStringValue((String) object); - case BYTES: - return Value.createBytesValue(ByteString.copyFrom((byte[]) object)); - case DECIMAL: - return Value.createNumericValue((BigDecimal) object); - case DATETIME: - return Value.createTimestampValueFromUnixMicros( - LongMath.checkedMultiply(((Instant) object).getMillis(), MICROS_PER_MILLI)); - case LOGICAL_TYPE: - String identifier = fieldType.getLogicalType().getIdentifier(); - if (SqlTypes.DATE.getIdentifier().equals(identifier)) { - if (object instanceof Long) { // base type - return Value.createDateValue(((Long) object).intValue()); - } else { // input type - return Value.createDateValue((LocalDate) object); - } - } else if (SqlTypes.TIME.getIdentifier().equals(identifier)) { - LocalTime localTime; - if (object instanceof Long) { // base type - localTime = LocalTime.ofNanoOfDay((Long) object); - } else { // input type - localTime = (LocalTime) object; - } - return Value.createTimeValue(localTime); - } else if (SqlTypes.DATETIME.getIdentifier().equals(identifier)) { - LocalDateTime datetime; - if (object instanceof Row) { // base type - datetime = - LocalDateTime.of( - LocalDate.ofEpochDay(((Row) object).getInt64(DateTime.DATE_FIELD_NAME)), - LocalTime.ofNanoOfDay(((Row) object).getInt64(DateTime.TIME_FIELD_NAME))); - } else { // input type - datetime = (LocalDateTime) object; - } - return Value.createDatetimeValue(datetime); - } else { - throw new UnsupportedOperationException("Unknown Beam logical type: " + identifier); - } - case ARRAY: - return toZetaSqlArrayValue((List) object, fieldType.getCollectionElementType()); - case ROW: - return toZetaSqlStructValue((Row) object, fieldType.getRowSchema()); - default: - throw new UnsupportedOperationException( - "Unknown Beam fieldType: " + fieldType.getTypeName()); - } - } - - private static Value toZetaSqlArrayValue(List elements, FieldType elementFieldType) { - List values = - elements.stream() - .map(e -> toZetaSqlValue(e, elementFieldType)) - .collect(Collectors.toList()); - return Value.createArrayValue(toZetaSqlArrayType(elementFieldType), values); - } - - public static Value toZetaSqlStructValue(Row row, Schema schema) { - List values = new ArrayList<>(row.getFieldCount()); - - for (int i = 0; i < row.getFieldCount(); i++) { - values.add(toZetaSqlValue(row.getBaseValue(i, Object.class), schema.getField(i).getType())); - } - return Value.createStructValue(toZetaSqlStructType(schema), values); - } - - // Type conversion: ZetaSQL => Beam - public static FieldType toBeamType(Type type) { - switch (type.getKind()) { - case TYPE_INT64: - return FieldType.INT64.withNullable(true); - case TYPE_DOUBLE: - return FieldType.DOUBLE.withNullable(true); - case TYPE_BOOL: - return FieldType.BOOLEAN.withNullable(true); - case TYPE_STRING: - return FieldType.STRING.withNullable(true); - case TYPE_BYTES: - return FieldType.BYTES.withNullable(true); - case TYPE_NUMERIC: - return FieldType.DECIMAL.withNullable(true); - case TYPE_DATE: - return FieldType.logicalType(SqlTypes.DATE).withNullable(true); - case TYPE_TIME: - return FieldType.logicalType(SqlTypes.TIME).withNullable(true); - case TYPE_DATETIME: - return FieldType.logicalType(SqlTypes.DATETIME).withNullable(true); - case TYPE_TIMESTAMP: - return FieldType.DATETIME.withNullable(true); - case TYPE_ARRAY: - return FieldType.array(toBeamType(type.asArray().getElementType())).withNullable(true); - case TYPE_STRUCT: - return FieldType.row( - type.asStruct().getFieldList().stream() - .map(f -> Field.of(f.getName(), toBeamType(f.getType()))) - .collect(Schema.toSchema())) - .withNullable(true); - default: - throw new UnsupportedOperationException("Unknown ZetaSQL type: " + type.getKind()); - } - } - - // Value conversion: ZetaSQL => Beam (target Beam type unknown) - public static Object toBeamObject(Value value, boolean verifyValues) { - return toBeamObject(value, toBeamType(value.getType()), verifyValues); - } - - // Value conversion: ZetaSQL => Beam (target Beam type known) - public static Object toBeamObject(Value value, FieldType fieldType, boolean verifyValues) { - if (value.isNull()) { - return null; - } - switch (fieldType.getTypeName()) { - case INT64: - return value.getInt64Value(); - case DOUBLE: - // Floats with a floating part equal to zero are treated as whole (INT64). - // Cast to double when that happens. - if (value.getType().getKind().equals(TypeKind.TYPE_INT64)) { - return (double) value.getInt64Value(); - } - return value.getDoubleValue(); - case BOOLEAN: - return value.getBoolValue(); - case STRING: - return value.getStringValue(); - case BYTES: - return value.getBytesValue().toByteArray(); - case DECIMAL: - return value.getNumericValue(); - case DATETIME: - return Instant.ofEpochMilli(value.getTimestampUnixMicros() / MICROS_PER_MILLI); - case LOGICAL_TYPE: - String identifier = fieldType.getLogicalType().getIdentifier(); - if (SqlTypes.DATE.getIdentifier().equals(identifier)) { - return value.getLocalDateValue(); - } else if (SqlTypes.TIME.getIdentifier().equals(identifier)) { - return value.getLocalTimeValue(); - } else if (SqlTypes.DATETIME.getIdentifier().equals(identifier)) { - return value.getLocalDateTimeValue(); - } else { - throw new UnsupportedOperationException("Unknown Beam logical type: " + identifier); - } - case ARRAY: - return toBeamList(value, fieldType.getCollectionElementType(), verifyValues); - case ROW: - return toBeamRow(value, fieldType.getRowSchema(), verifyValues); - default: - throw new UnsupportedOperationException( - "Unknown Beam fieldType: " + fieldType.getTypeName()); - } - } - - private static List toBeamList( - Value arrayValue, FieldType elementType, boolean verifyValues) { - return arrayValue.getElementList().stream() - .map(e -> toBeamObject(e, elementType, verifyValues)) - .collect(Collectors.toList()); - } - - public static Row toBeamRow(Value structValue, Schema schema, boolean verifyValues) { - List objects = new ArrayList<>(schema.getFieldCount()); - List values = structValue.getFieldList(); - for (int i = 0; i < values.size(); i++) { - objects.add(toBeamObject(values.get(i), schema.getField(i).getType(), verifyValues)); - } - Row row = - verifyValues - ? Row.withSchema(schema).addValues(objects).build() - : Row.withSchema(schema).attachValues(objects); - return row; - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java deleted file mode 100644 index 965426db287f..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java +++ /dev/null @@ -1,366 +0,0 @@ -/* - * 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.StructType; -import com.google.zetasql.StructType.StructField; -import com.google.zetasql.Type; -import com.google.zetasql.TypeFactory; -import com.google.zetasql.Value; -import com.google.zetasql.ZetaSQLType.TypeKind; -import com.google.zetasql.functions.ZetaSQLDateTime.DateTimestampPart; -import java.math.BigDecimal; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BeamBigQuerySqlDialect; -import org.apache.beam.sdk.extensions.sql.zetasql.translation.SqlOperators; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.ByteString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.TimeUnit; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.avatica.util.TimeUnitRange; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.DateString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.TimeString; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.TimestampString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -/** - * Utility methods for ZetaSQL <=> Calcite translation. - * - *

Unsupported ZetaSQL types: INT32, UINT32, UINT64, FLOAT, ENUM (internal), PROTO, GEOGRAPHY - */ -@Internal -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public final class ZetaSqlCalciteTranslationUtils { - // Maximum and minimum allowed values for the NUMERIC/DECIMAL data type. - // https://github.com/google/zetasql/blob/master/docs/data-types.md#decimal-type - public static final BigDecimal ZETASQL_NUMERIC_MAX_VALUE = - new BigDecimal("99999999999999999999999999999.999999999"); - public static final BigDecimal ZETASQL_NUMERIC_MIN_VALUE = - new BigDecimal("-99999999999999999999999999999.999999999"); - // Number of digits after the decimal point supported by the NUMERIC data type. - public static final int ZETASQL_NUMERIC_SCALE = 9; - - private ZetaSqlCalciteTranslationUtils() {} - - // TODO[BEAM-9178]: support DateTimestampPart.WEEK and "WEEK with weekday"s - private static final ImmutableMap TIME_UNIT_CASTING_MAP = - ImmutableMap.builder() - .put(DateTimestampPart.YEAR.getNumber(), TimeUnit.YEAR) - .put(DateTimestampPart.MONTH.getNumber(), TimeUnit.MONTH) - .put(DateTimestampPart.DAY.getNumber(), TimeUnit.DAY) - .put(DateTimestampPart.DAYOFWEEK.getNumber(), TimeUnit.DOW) - .put(DateTimestampPart.DAYOFYEAR.getNumber(), TimeUnit.DOY) - .put(DateTimestampPart.QUARTER.getNumber(), TimeUnit.QUARTER) - .put(DateTimestampPart.HOUR.getNumber(), TimeUnit.HOUR) - .put(DateTimestampPart.MINUTE.getNumber(), TimeUnit.MINUTE) - .put(DateTimestampPart.SECOND.getNumber(), TimeUnit.SECOND) - .put(DateTimestampPart.MILLISECOND.getNumber(), TimeUnit.MILLISECOND) - .put(DateTimestampPart.MICROSECOND.getNumber(), TimeUnit.MICROSECOND) - .put(DateTimestampPart.ISOYEAR.getNumber(), TimeUnit.ISOYEAR) - .put(DateTimestampPart.ISOWEEK.getNumber(), TimeUnit.WEEK) - .build(); - - // Type conversion: Calcite => ZetaSQL - public static Type toZetaSqlType(RelDataType calciteType) { - switch (calciteType.getSqlTypeName()) { - case BIGINT: - return TypeFactory.createSimpleType(TypeKind.TYPE_INT64); - case DOUBLE: - return TypeFactory.createSimpleType(TypeKind.TYPE_DOUBLE); - case BOOLEAN: - return TypeFactory.createSimpleType(TypeKind.TYPE_BOOL); - case VARCHAR: - return TypeFactory.createSimpleType(TypeKind.TYPE_STRING); - case VARBINARY: - return TypeFactory.createSimpleType(TypeKind.TYPE_BYTES); - case DECIMAL: - return TypeFactory.createSimpleType(TypeKind.TYPE_NUMERIC); - case DATE: - return TypeFactory.createSimpleType(TypeKind.TYPE_DATE); - case TIME: - return TypeFactory.createSimpleType(TypeKind.TYPE_TIME); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return TypeFactory.createSimpleType(TypeKind.TYPE_DATETIME); - case TIMESTAMP: - return TypeFactory.createSimpleType(TypeKind.TYPE_TIMESTAMP); - case ARRAY: - return TypeFactory.createArrayType(toZetaSqlType(calciteType.getComponentType())); - case ROW: - return TypeFactory.createStructType( - calciteType.getFieldList().stream() - .map(f -> new StructField(f.getName(), toZetaSqlType(f.getType()))) - .collect(Collectors.toList())); - default: - throw new UnsupportedOperationException( - "Unknown Calcite type: " + calciteType.getSqlTypeName().getName()); - } - } - - // Type conversion: ZetaSQL => Calcite - public static RelDataType toCalciteType(Type type, boolean nullable, RexBuilder rexBuilder) { - RelDataType nonNullType; - switch (type.getKind()) { - case TYPE_INT64: - nonNullType = rexBuilder.getTypeFactory().createSqlType(SqlTypeName.BIGINT); - break; - case TYPE_DOUBLE: - nonNullType = rexBuilder.getTypeFactory().createSqlType(SqlTypeName.DOUBLE); - break; - case TYPE_BOOL: - nonNullType = rexBuilder.getTypeFactory().createSqlType(SqlTypeName.BOOLEAN); - break; - case TYPE_STRING: - nonNullType = rexBuilder.getTypeFactory().createSqlType(SqlTypeName.VARCHAR); - break; - case TYPE_BYTES: - nonNullType = rexBuilder.getTypeFactory().createSqlType(SqlTypeName.VARBINARY); - break; - case TYPE_NUMERIC: - nonNullType = rexBuilder.getTypeFactory().createSqlType(SqlTypeName.DECIMAL); - break; - case TYPE_DATE: - nonNullType = rexBuilder.getTypeFactory().createSqlType(SqlTypeName.DATE); - break; - case TYPE_TIME: - nonNullType = rexBuilder.getTypeFactory().createSqlType(SqlTypeName.TIME); - break; - case TYPE_DATETIME: - nonNullType = - rexBuilder.getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE); - break; - case TYPE_TIMESTAMP: - nonNullType = rexBuilder.getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP); - break; - case TYPE_ARRAY: - // TODO: Should element type has the same nullability as the array type? - nonNullType = toCalciteArrayType(type.asArray().getElementType(), nullable, rexBuilder); - break; - case TYPE_STRUCT: - // TODO: Should field type has the same nullability as the struct type? - nonNullType = toCalciteStructType(type.asStruct(), nullable, rexBuilder); - break; - default: - throw new UnsupportedOperationException("Unknown ZetaSQL type: " + type.getKind().name()); - } - return rexBuilder.getTypeFactory().createTypeWithNullability(nonNullType, nullable); - } - - private static RelDataType toCalciteArrayType( - Type elementType, boolean nullable, RexBuilder rexBuilder) { - return rexBuilder - .getTypeFactory() - // -1 cardinality means unlimited array size - .createArrayType(toCalciteType(elementType, nullable, rexBuilder), -1); - } - - private static RelDataType toCalciteStructType( - StructType structType, boolean nullable, RexBuilder rexBuilder) { - List fields = structType.getFieldList(); - List fieldNames = getFieldNameList(fields); - List fieldTypes = - fields.stream() - .map(f -> toCalciteType(f.getType(), nullable, rexBuilder)) - .collect(Collectors.toList()); - return rexBuilder.getTypeFactory().createStructType(fieldTypes, fieldNames); - } - - private static List getFieldNameList(List fields) { - ImmutableList.Builder b = ImmutableList.builder(); - Set usedName = new HashSet<>(); - for (int i = 0; i < fields.size(); i++) { - String name = fields.get(i).getName(); - // Follow the same way that BigQuery handles unspecified or duplicate field name - if ("".equals(name) || name.startsWith("_field_") || usedName.contains(name)) { - name = "_field_" + (i + 1); // BigQuery uses 1-based default field name - } - b.add(name); - usedName.add(name); - } - return b.build(); - } - - // Value conversion: ZetaSQL => Calcite - public static RexNode toRexNode(Value value, RexBuilder rexBuilder) { - Type type = value.getType(); - if (value.isNull()) { - return rexBuilder.makeNullLiteral(toCalciteType(type, true, rexBuilder)); - } - - switch (type.getKind()) { - case TYPE_INT64: - return rexBuilder.makeExactLiteral( - new BigDecimal(value.getInt64Value()), toCalciteType(type, false, rexBuilder)); - case TYPE_DOUBLE: - // Cannot simply call makeApproxLiteral() because +inf, -inf, and NaN cannot be represented - // as BigDecimal. So we create wrapper functions here for these three cases such that we can - // later recognize it and customize its unparsing in BeamBigQuerySqlDialect. - double val = value.getDoubleValue(); - String wrapperFun = null; - if (val == Double.POSITIVE_INFINITY) { - wrapperFun = BeamBigQuerySqlDialect.DOUBLE_POSITIVE_INF_WRAPPER; - } else if (val == Double.NEGATIVE_INFINITY) { - wrapperFun = BeamBigQuerySqlDialect.DOUBLE_NEGATIVE_INF_WRAPPER; - } else if (Double.isNaN(val)) { - wrapperFun = BeamBigQuerySqlDialect.DOUBLE_NAN_WRAPPER; - } - - RelDataType returnType = toCalciteType(type, false, rexBuilder); - if (wrapperFun == null) { - return rexBuilder.makeApproxLiteral(new BigDecimal(val), returnType); - } else if (BeamBigQuerySqlDialect.DOUBLE_NAN_WRAPPER.equals(wrapperFun)) { - // TODO[https://github.com/apache/beam/issues/20354]: Update the temporary workaround - // below after vendored Calcite version. - // Adding an additional random parameter for the wrapper function of NaN, to avoid - // triggering Calcite operation simplification. (e.g. 'NaN == NaN' would be simplify to - // 'null or NaN is not null' in Calcite. This would miscalculate the expression to be - // true, which should be false.) - return rexBuilder.makeCall( - SqlOperators.createZetaSqlFunction(wrapperFun, returnType.getSqlTypeName()), - rexBuilder.makeApproxLiteral(BigDecimal.valueOf(Math.random()), returnType)); - } else { - return rexBuilder.makeCall( - SqlOperators.createZetaSqlFunction(wrapperFun, returnType.getSqlTypeName())); - } - case TYPE_BOOL: - return rexBuilder.makeLiteral(value.getBoolValue()); - case TYPE_STRING: - // Has to allow CAST because Calcite create CHAR type first and does a CAST to VARCHAR. - // If not allow cast, rexBuilder() will only build a literal with CHAR type. - return rexBuilder.makeLiteral( - value.getStringValue(), toCalciteType(type, false, rexBuilder), true); - case TYPE_BYTES: - return rexBuilder.makeBinaryLiteral(new ByteString(value.getBytesValue().toByteArray())); - case TYPE_NUMERIC: - // Cannot simply call makeExactLiteral() because later it will be unparsed to the string - // representation of the BigDecimal itself (e.g. "SELECT NUMERIC '0'" will be unparsed to - // "SELECT 0E-9"), and Calcite does not allow customize unparsing of SqlNumericLiteral. - // So we create a wrapper function here such that we can later recognize it and customize - // its unparsing in BeamBigQuerySqlDialect. - return rexBuilder.makeCall( - SqlOperators.createZetaSqlFunction( - BeamBigQuerySqlDialect.NUMERIC_LITERAL_WRAPPER, - toCalciteType(type, false, rexBuilder).getSqlTypeName()), - rexBuilder.makeExactLiteral( - value.getNumericValue(), toCalciteType(type, false, rexBuilder))); - case TYPE_DATE: - return rexBuilder.makeDateLiteral(dateValueToDateString(value)); - case TYPE_TIME: - return rexBuilder.makeTimeLiteral( - timeValueToTimeString(value), - rexBuilder.getTypeFactory().getTypeSystem().getMaxPrecision(SqlTypeName.TIME)); - case TYPE_DATETIME: - return rexBuilder.makeTimestampWithLocalTimeZoneLiteral( - datetimeValueToTimestampString(value), - rexBuilder - .getTypeFactory() - .getTypeSystem() - .getMaxPrecision(SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE)); - case TYPE_TIMESTAMP: - return rexBuilder.makeTimestampLiteral( - timestampValueToTimestampString(value), - rexBuilder.getTypeFactory().getTypeSystem().getMaxPrecision(SqlTypeName.TIMESTAMP)); - case TYPE_ARRAY: - return arrayValueToRexNode(value, rexBuilder); - case TYPE_STRUCT: - return structValueToRexNode(value, rexBuilder); - case TYPE_ENUM: // internal only, used for DateTimestampPart - return enumValueToRexNode(value, rexBuilder); - default: - throw new UnsupportedOperationException("Unknown ZetaSQL type: " + type.getKind().name()); - } - } - - private static RexNode arrayValueToRexNode(Value value, RexBuilder rexBuilder) { - return rexBuilder.makeCall( - toCalciteArrayType( - value.getType().asArray().getElementType(), - value.getElementList().stream().anyMatch(v -> v.isNull()), - rexBuilder), - SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, - value.getElementList().stream() - .map(v -> toRexNode(v, rexBuilder)) - .collect(Collectors.toList())); - } - - private static RexNode structValueToRexNode(Value value, RexBuilder rexBuilder) { - return rexBuilder.makeCall( - toCalciteStructType(value.getType().asStruct(), false, rexBuilder), - SqlStdOperatorTable.ROW, - value.getFieldList().stream() - .map(v -> toRexNode(v, rexBuilder)) - .collect(Collectors.toList())); - } - - // internal only, used for DateTimestampPart - private static RexNode enumValueToRexNode(Value value, RexBuilder rexBuilder) { - String enumDescriptorName = value.getType().asEnum().getDescriptor().getFullName(); - if (!"zetasql.functions.DateTimestampPart".equals(enumDescriptorName)) { - throw new UnsupportedOperationException("Unknown ZetaSQL Enum type: " + enumDescriptorName); - } - TimeUnit timeUnit = TIME_UNIT_CASTING_MAP.get(value.getEnumValue()); - if (timeUnit == null) { - throw new UnsupportedOperationException("Unknown ZetaSQL Enum value: " + value.getEnumName()); - } - return rexBuilder.makeFlag(TimeUnitRange.of(timeUnit, null)); - } - - private static DateString dateValueToDateString(Value value) { - return DateString.fromDaysSinceEpoch(value.getDateValue()); - } - - private static TimeString timeValueToTimeString(Value value) { - LocalTime localTime = value.getLocalTimeValue(); - return new TimeString(localTime.getHour(), localTime.getMinute(), localTime.getSecond()) - .withNanos(localTime.getNano()); - } - - private static TimestampString datetimeValueToTimestampString(Value value) { - LocalDateTime dateTime = value.getLocalDateTimeValue(); - return new TimestampString( - dateTime.getYear(), - dateTime.getMonthValue(), - dateTime.getDayOfMonth(), - dateTime.getHour(), - dateTime.getMinute(), - dateTime.getSecond()) - .withNanos(dateTime.getNano()); - } - - private static TimestampString timestampValueToTimestampString(Value value) { - long micros = value.getTimestampUnixMicros(); - if (micros % 1000L != 0) { - throw new UnsupportedOperationException( - String.format( - "%s has sub-millisecond precision, which Beam ZetaSQL does not currently support.", - micros)); - } - return TimestampString.fromMillisSinceEpoch(micros / 1000L); - } -} 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 deleted file mode 100644 index 7a948efb4b74..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlException.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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. - * - *

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/package-info.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/package-info.java deleted file mode 100644 index 0400e76ca028..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/package-info.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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. - */ - -/** - * ZetaSQL Dialect package. - * - *

- * - * @deprecated Use Calcite SQL dialect. Beam ZetaSQL has been deprecated. - */ -package org.apache.beam.sdk.extensions.sql.zetasql; diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java deleted file mode 100644 index 412cd46001f8..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java +++ /dev/null @@ -1,283 +0,0 @@ -/* - * 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.translation; - -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_CAST; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_COLUMN_REF; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_GET_STRUCT_FIELD; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_LITERAL; - -import com.google.zetasql.FunctionSignature; -import com.google.zetasql.ZetaSQLResolvedNodeKind; -import com.google.zetasql.ZetaSQLType.TypeKind; -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedAggregateFunctionCall; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedAggregateScan; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedComputedColumn; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedComputedColumnBase; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedExpr; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.beam.sdk.extensions.sql.impl.UdafImpl; -import org.apache.beam.sdk.extensions.sql.zetasql.BeamZetaSqlCatalog; -import org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlCalciteTranslationUtils; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollations; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.AggregateCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalAggregate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalProject; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlAggFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ImmutableBitSet; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -/** Converts aggregate calls. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class AggregateScanConverter extends RelConverter { - private static final String AVG_ILLEGAL_LONG_INPUT_TYPE = - "AVG(INT64) is not supported. You might want to use AVG(CAST(expression AS FLOAT64)."; - - AggregateScanConverter(ConversionContext context) { - super(context); - } - - @Override - public List getInputs(ResolvedAggregateScan zetaNode) { - return Collections.singletonList(zetaNode.getInputScan()); - } - - @Override - public RelNode convert(ResolvedAggregateScan zetaNode, List inputs) { - LogicalProject input = convertAggregateScanInputScanToLogicalProject(zetaNode, inputs.get(0)); - - // Calcite LogicalAggregate's GroupSet is indexes of group fields starting from 0. - int groupFieldsListSize = zetaNode.getGroupByList().size(); - ImmutableBitSet groupSet; - if (groupFieldsListSize != 0) { - groupSet = - ImmutableBitSet.of( - IntStream.rangeClosed(0, groupFieldsListSize - 1) - .boxed() - .collect(Collectors.toList())); - } else { - groupSet = ImmutableBitSet.of(); - } - - // TODO: add support for indicator - - List aggregateCalls; - if (zetaNode.getAggregateList().isEmpty()) { - aggregateCalls = ImmutableList.of(); - } else { - aggregateCalls = new ArrayList<>(); - // For aggregate calls, their input ref follow after GROUP BY input ref. - int columnRefoff = groupFieldsListSize; - for (ResolvedComputedColumnBase computedColumn : zetaNode.getAggregateList()) { - AggregateCall aggCall = - convertAggCall(computedColumn, columnRefoff, groupSet.size(), input); - aggregateCalls.add(aggCall); - if (!aggCall.getArgList().isEmpty()) { - // Only increment column reference offset when aggregates use them (BEAM-8042). - // Ex: COUNT(*) does not have arguments, while COUNT(`field`) does. - columnRefoff++; - } - } - } - - LogicalAggregate logicalAggregate = - new LogicalAggregate( - getCluster(), - input.getTraitSet(), - input, - groupSet, - ImmutableList.of(groupSet), - aggregateCalls); - - return logicalAggregate; - } - - private LogicalProject convertAggregateScanInputScanToLogicalProject( - ResolvedAggregateScan node, RelNode input) { - // AggregateScan's input is the source of data (e.g. TableScan), which is different from the - // design of CalciteSQL, in which the LogicalAggregate's input is a LogicalProject, whose input - // is a LogicalTableScan. When AggregateScan's input is WithRefScan, the WithRefScan is - // ebullient to a LogicalTableScan. So it's still required to build another LogicalProject as - // the input of LogicalAggregate. - List projects = new ArrayList<>(); - List fieldNames = new ArrayList<>(); - - // LogicalProject has a list of expr, which including UDF in GROUP BY clause for - // LogicalAggregate. - for (ResolvedComputedColumn computedColumn : node.getGroupByList()) { - projects.add( - getExpressionConverter() - .convertRexNodeFromResolvedExpr( - computedColumn.getExpr(), - node.getInputScan().getColumnList(), - input.getRowType().getFieldList(), - ImmutableMap.of())); - fieldNames.add(getTrait().resolveAlias(computedColumn.getColumn())); - } - - // LogicalProject should also include columns used by aggregate functions. These columns should - // follow after GROUP BY columns. - // TODO: remove duplicate columns in projects. - for (ResolvedComputedColumnBase resolvedComputedColumn : node.getAggregateList()) { - // Should create Calcite's RexInputRef from ResolvedColumn from ResolvedComputedColumn. - // TODO: handle aggregate function with more than one argument and handle OVER - // TODO: is there is general way for column reference tracking and deduplication for - // aggregation? - ResolvedAggregateFunctionCall aggregateFunctionCall = - ((ResolvedAggregateFunctionCall) resolvedComputedColumn.getExpr()); - if (aggregateFunctionCall.getArgumentList() != null - && aggregateFunctionCall.getArgumentList().size() >= 1) { - ResolvedExpr resolvedExpr = aggregateFunctionCall.getArgumentList().get(0); - for (int i = 0; i < aggregateFunctionCall.getArgumentList().size(); i++) { - if (i == 0) { - // TODO: assume aggregate function's input is either a ColumnRef or a cast(ColumnRef). - // TODO: user might use multiple CAST so we need to handle this rare case. - projects.add( - getExpressionConverter() - .convertRexNodeFromResolvedExpr( - resolvedExpr, - node.getInputScan().getColumnList(), - input.getRowType().getFieldList(), - ImmutableMap.of())); - } else { - projects.add( - getExpressionConverter() - .convertRexNodeFromResolvedExpr( - aggregateFunctionCall.getArgumentList().get(i))); - } - fieldNames.add(getTrait().resolveAlias(resolvedComputedColumn.getColumn())); - } - } - } - - return LogicalProject.create(input, ImmutableList.of(), projects, fieldNames); - } - - private AggregateCall convertAggCall( - ResolvedComputedColumnBase computedColumn, int columnRefOff, int groupCount, RelNode input) { - ResolvedAggregateFunctionCall aggregateFunctionCall = - (ResolvedAggregateFunctionCall) computedColumn.getExpr(); - - // Reject AVG(INT64) - if (aggregateFunctionCall.getFunction().getName().equals("avg")) { - FunctionSignature signature = aggregateFunctionCall.getSignature(); - if (signature - .getFunctionArgumentList() - .get(0) - .getType() - .getKind() - .equals(TypeKind.TYPE_INT64)) { - throw new UnsupportedOperationException(AVG_ILLEGAL_LONG_INPUT_TYPE); - } - } - - // Reject aggregation DISTINCT - if (aggregateFunctionCall.getDistinct()) { - throw new UnsupportedOperationException( - "Does not support " - + aggregateFunctionCall.getFunction().getSqlName() - + " DISTINCT. 'SELECT DISTINCT' syntax could be used to deduplicate before" - + " aggregation."); - } - - final SqlAggFunction sqlAggFunction; - if (aggregateFunctionCall - .getFunction() - .getGroup() - .equals(BeamZetaSqlCatalog.USER_DEFINED_JAVA_AGGREGATE_FUNCTIONS)) { - // Create a new operator for user-defined functions. - SqlReturnTypeInference typeInference = - x -> - ZetaSqlCalciteTranslationUtils.toCalciteType( - aggregateFunctionCall - .getFunction() - .getSignatureList() - .get(0) - .getResultType() - .getType(), - // TODO(BEAM-9514) set nullable=true - false, - getCluster().getRexBuilder()); - UdafImpl impl = - new UdafImpl<>( - getExpressionConverter() - .userFunctionDefinitions - .javaAggregateFunctions() - .get(aggregateFunctionCall.getFunction().getNamePath())); - sqlAggFunction = - SqlOperators.createUdafOperator( - aggregateFunctionCall.getFunction().getName(), typeInference, impl); - } else { - // Look up builtin functions in SqlOperatorMappingTable. - sqlAggFunction = (SqlAggFunction) SqlOperatorMappingTable.create(aggregateFunctionCall); - if (sqlAggFunction == null) { - throw new UnsupportedOperationException( - "Does not support ZetaSQL aggregate function: " - + aggregateFunctionCall.getFunction().getName()); - } - } - - List argList = new ArrayList<>(); - ResolvedAggregateFunctionCall expr = ((ResolvedAggregateFunctionCall) computedColumn.getExpr()); - List resolvedNodeKinds = - Arrays.asList(RESOLVED_CAST, RESOLVED_COLUMN_REF, RESOLVED_GET_STRUCT_FIELD); - for (int i = 0; i < expr.getArgumentList().size(); i++) { - // Throw an error if aggregate function's input isn't either a ColumnRef or a cast(ColumnRef). - // TODO: is there a general way to handle aggregation calls conversion? - ZetaSQLResolvedNodeKind.ResolvedNodeKind resolvedNodeKind = - expr.getArgumentList().get(i).nodeKind(); - if (i == 0 && resolvedNodeKinds.contains(resolvedNodeKind)) { - argList.add(columnRefOff); - } else if (i > 0 && resolvedNodeKind == RESOLVED_LITERAL) { - continue; - } else { - throw new UnsupportedOperationException( - "Aggregate function only accepts Column Reference or CAST(Column Reference) as the first argument and " - + "Literals as subsequent arguments as its inputs"); - } - } - - String aggName = getTrait().resolveAlias(computedColumn.getColumn()); - return AggregateCall.create( - sqlAggFunction, - false, - false, - false, - argList, - -1, - null, - RelCollations.EMPTY, - groupCount, - input, - // When we pass null as the return type, Calcite infers it for us. - null, - aggName); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java deleted file mode 100644 index ac3de648b52d..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes; -import java.util.Collections; -import java.util.List; -import org.apache.beam.sdk.extensions.sql.zetasql.unnest.ZetaSqlUnnest; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.CorrelationId; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalCorrelate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalProject; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.ImmutableBitSet; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -/** - * Converts array scan that represents a reference to an array column, or an (possibly nested) array - * field of an struct column to uncollect. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class ArrayScanColumnRefToUncollect extends RelConverter { - ArrayScanColumnRefToUncollect(ConversionContext context) { - super(context); - } - - @Override - public boolean canConvert(ResolvedNodes.ResolvedArrayScan zetaNode) { - return zetaNode.getInputScan() != null - && getColumnRef(zetaNode.getArrayExpr()) != null - && zetaNode.getJoinExpr() == null; - } - - @Override - public List getInputs(ResolvedNodes.ResolvedArrayScan zetaNode) { - return ImmutableList.of(zetaNode.getInputScan()); - } - - @Override - public RelNode convert(ResolvedNodes.ResolvedArrayScan zetaNode, List inputs) { - assert inputs.size() == 1; - RelNode input = inputs.get(0); - RexInputRef columnRef = - (RexInputRef) - getExpressionConverter() - .convertRexNodeFromResolvedExpr( - getColumnRef(zetaNode.getArrayExpr()), - zetaNode.getInputScan().getColumnList(), - input.getRowType().getFieldList(), - ImmutableMap.of()); - - CorrelationId correlationId = new CorrelationId(0); - RexNode convertedColumnRef = - getCluster() - .getRexBuilder() - .makeFieldAccess( - getCluster().getRexBuilder().makeCorrel(input.getRowType(), correlationId), - columnRef.getIndex()); - - String fieldName = - String.format( - "%s%s", - zetaNode.getElementColumn().getTableName(), zetaNode.getElementColumn().getName()); - - RelNode projectNode = - LogicalProject.create( - createOneRow(getCluster()), - ImmutableList.of(), - Collections.singletonList( - convertArrayExpr( - zetaNode.getArrayExpr(), getCluster().getRexBuilder(), convertedColumnRef)), - ImmutableList.of(fieldName)); - - boolean ordinality = (zetaNode.getArrayOffsetColumn() != null); - RelNode uncollect = ZetaSqlUnnest.create(projectNode.getTraitSet(), projectNode, ordinality); - - return LogicalCorrelate.create( - input, - uncollect, - correlationId, - ImmutableBitSet.of(columnRef.getIndex()), - JoinRelType.INNER); - } - - private static ResolvedNodes.ResolvedColumnRef getColumnRef(ResolvedNode arrayExpr) { - while (arrayExpr instanceof ResolvedNodes.ResolvedGetStructField) { - arrayExpr = ((ResolvedNodes.ResolvedGetStructField) arrayExpr).getExpr(); - } - return arrayExpr instanceof ResolvedNodes.ResolvedColumnRef - ? (ResolvedNodes.ResolvedColumnRef) arrayExpr - : null; - } - - private static RexNode convertArrayExpr( - ResolvedNodes.ResolvedExpr expr, RexBuilder builder, RexNode convertedColumnRef) { - if (expr instanceof ResolvedNodes.ResolvedColumnRef) { - return convertedColumnRef; - } - ResolvedNodes.ResolvedGetStructField getStructField = - (ResolvedNodes.ResolvedGetStructField) expr; - return builder.makeFieldAccess( - convertArrayExpr(getStructField.getExpr(), builder, convertedColumnRef), - (int) getStructField.getFieldIdx()); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanLiteralToUncollectConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanLiteralToUncollectConverter.java deleted file mode 100644 index 902b7a762fe5..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanLiteralToUncollectConverter.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedArrayScan; -import java.util.Collections; -import java.util.List; -import org.apache.beam.sdk.extensions.sql.zetasql.unnest.ZetaSqlUnnest; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalProject; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; - -/** Converts array scan that represents an array literal to uncollect. */ -class ArrayScanLiteralToUncollectConverter extends RelConverter { - - ArrayScanLiteralToUncollectConverter(ConversionContext context) { - super(context); - } - - @Override - public boolean canConvert(ResolvedArrayScan zetaNode) { - return zetaNode.getInputScan() == null; - } - - @Override - public RelNode convert(ResolvedArrayScan zetaNode, List inputs) { - RexNode arrayLiteralExpression = - getExpressionConverter().convertRexNodeFromResolvedExpr(zetaNode.getArrayExpr()); - - String fieldName = - String.format( - "%s%s", - zetaNode.getElementColumn().getTableName(), zetaNode.getElementColumn().getName()); - - RelNode projectNode = - LogicalProject.create( - createOneRow(getCluster()), - ImmutableList.of(), - Collections.singletonList(arrayLiteralExpression), - ImmutableList.of(fieldName)); - - boolean ordinality = (zetaNode.getArrayOffsetColumn() != null); - - // These asserts guaranteed by the parser code, but not the data structure. - // If they aren't true we need to add a Project to reorder columns. - assert zetaNode.getElementColumn().getId() == 1; - assert !ordinality || zetaNode.getArrayOffsetColumn().getColumn().getId() == 2; - return ZetaSqlUnnest.create(projectNode.getTraitSet(), projectNode, ordinality); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanToJoinConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanToJoinConverter.java deleted file mode 100644 index 79984ea877e0..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanToJoinConverter.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedArrayScan; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedColumnRef; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import org.apache.beam.sdk.extensions.sql.zetasql.unnest.ZetaSqlUnnest; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.CorrelationId; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalJoin; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalProject; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; - -/** Converts array scan that represents join of an uncollect(array_field) to uncollect. */ -class ArrayScanToJoinConverter extends RelConverter { - - ArrayScanToJoinConverter(ConversionContext context) { - super(context); - } - - /** This is the case of {@code table [LEFT|INNER] JOIN UNNEST(table.array_field) on join_expr}. */ - @Override - public boolean canConvert(ResolvedArrayScan zetaNode) { - return zetaNode.getArrayExpr() instanceof ResolvedColumnRef - && zetaNode.getInputScan() != null - && zetaNode.getJoinExpr() != null; - } - - /** Left input is converted from input scan. */ - @Override - public List getInputs(ResolvedArrayScan zetaNode) { - return Collections.singletonList(zetaNode.getInputScan()); - } - - /** Returns a LogicJoin. */ - @Override - public RelNode convert(ResolvedArrayScan zetaNode, List inputs) { - List projects = new ArrayList<>(); - - RelNode leftInput = inputs.get(0); - - ResolvedColumnRef columnRef = (ResolvedColumnRef) zetaNode.getArrayExpr(); - CorrelationId correlationId = getCluster().createCorrel(); - getCluster().getQuery().mapCorrel(correlationId.getName(), leftInput); - String columnName = - String.format( - "%s%s", - zetaNode.getElementColumn().getTableName(), zetaNode.getElementColumn().getName()); - - projects.add( - getCluster() - .getRexBuilder() - .makeFieldAccess( - getCluster().getRexBuilder().makeCorrel(leftInput.getRowType(), correlationId), - getExpressionConverter() - .indexOfProjectionColumnRef( - columnRef.getColumn().getId(), zetaNode.getInputScan().getColumnList()))); - - RelNode projectNode = - LogicalProject.create( - createOneRow(getCluster()), ImmutableList.of(), projects, ImmutableList.of(columnName)); - - // Create an UnCollect - boolean ordinality = (zetaNode.getArrayOffsetColumn() != null); - - // These asserts guaranteed by the parser code, but not the data structure. - // If they aren't true we need the Project to reorder columns. - assert zetaNode.getElementColumn().getId() == 1; - assert !ordinality || zetaNode.getArrayOffsetColumn().getColumn().getId() == 2; - ZetaSqlUnnest uncollectNode = - ZetaSqlUnnest.create(projectNode.getTraitSet(), projectNode, ordinality); - - List rightProjects = new ArrayList<>(); - List rightNames = new ArrayList<>(); - rightProjects.add(getCluster().getRexBuilder().makeInputRef(uncollectNode, 0)); - rightNames.add(columnName); - if (ordinality) { - rightProjects.add(getCluster().getRexBuilder().makeInputRef(uncollectNode, 1)); - rightNames.add( - String.format( - zetaNode.getArrayOffsetColumn().getColumn().getTableName(), - zetaNode.getArrayOffsetColumn().getColumn().getName())); - } - - RelNode rightInput = - LogicalProject.create(uncollectNode, ImmutableList.of(), rightProjects, rightNames); - - // Join condition should be a RexNode converted from join_expr. - RexNode condition = - getExpressionConverter().convertRexNodeFromResolvedExpr(zetaNode.getJoinExpr()); - JoinRelType joinRelType = zetaNode.getIsOuter() ? JoinRelType.LEFT : JoinRelType.INNER; - - return LogicalJoin.create( - leftInput, - rightInput, - ImmutableList.of(), - condition, - ImmutableSet.of(), - joinRelType, - false, - ImmutableList.of()); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java deleted file mode 100644 index 01cf2d1c20f5..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNode; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.extensions.sql.zetasql.QueryTrait; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.FrameworkConfig; - -/** Conversion context, some rules need this data to convert the nodes. */ -@Internal -public class ConversionContext { - private final FrameworkConfig config; - private final ExpressionConverter expressionConverter; - private final RelOptCluster cluster; - private final QueryTrait trait; - - // SQL native user-defined table-valued function can be resolved by Analyzer. Its sql body is - // converted to ResolvedNode, in which function parameters are replaced with ResolvedArgumentRef. - // Meanwhile, Analyzer provides values for function parameters because it looks ahead to find - // the SELECT query. Thus keep the argument name to values (converted to RexNode) mapping in - // Context for future usage in plan conversion. - private Map functionArgumentRefMapping; - - public static ConversionContext of( - FrameworkConfig config, - ExpressionConverter expressionConverter, - RelOptCluster cluster, - QueryTrait trait) { - return new ConversionContext(config, expressionConverter, cluster, trait); - } - - private ConversionContext( - FrameworkConfig config, - ExpressionConverter expressionConverter, - RelOptCluster cluster, - QueryTrait trait) { - this.config = config; - this.expressionConverter = expressionConverter; - this.cluster = cluster; - this.trait = trait; - this.functionArgumentRefMapping = new HashMap<>(); - } - - FrameworkConfig getConfig() { - return config; - } - - ExpressionConverter getExpressionConverter() { - return expressionConverter; - } - - RelOptCluster cluster() { - return cluster; - } - - QueryTrait getTrait() { - return trait; - } - - Map, ResolvedNode> getUserDefinedTableValuedFunctions() { - return getExpressionConverter().userFunctionDefinitions.sqlTableValuedFunctions(); - } - - Map getFunctionArgumentRefMapping() { - return functionArgumentRefMapping; - } - - void addToFunctionArgumentRefMapping(String s, RexNode r) { - getFunctionArgumentRefMapping().put(s, r); - } - - void clearFunctionArgumentRefMapping() { - getFunctionArgumentRefMapping().clear(); - } -} 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 deleted file mode 100644 index 0f32451504b3..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java +++ /dev/null @@ -1,965 +0,0 @@ -/* - * 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.translation; - -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_FUNCTION_CALL; -import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_BOOL; -import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_BYTES; -import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_DOUBLE; -import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_INT64; -import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_NUMERIC; -import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_STRING; -import static com.google.zetasql.ZetaSQLType.TypeKind.TYPE_TIMESTAMP; -import static org.apache.beam.sdk.extensions.sql.zetasql.BeamZetaSqlCatalog.PRE_DEFINED_WINDOW_FUNCTIONS; -import static org.apache.beam.sdk.extensions.sql.zetasql.BeamZetaSqlCatalog.USER_DEFINED_JAVA_SCALAR_FUNCTIONS; -import static org.apache.beam.sdk.extensions.sql.zetasql.BeamZetaSqlCatalog.USER_DEFINED_SQL_FUNCTIONS; -import static org.apache.beam.sdk.extensions.sql.zetasql.BeamZetaSqlCatalog.ZETASQL_FUNCTION_GROUP_NAME; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - -import com.google.common.base.Ascii; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.zetasql.TVFRelation; -import com.google.zetasql.TVFRelation.Column; -import com.google.zetasql.TableValuedFunction; -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; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedArgumentRef; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedCast; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedColumnRef; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedComputedColumn; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedCreateFunctionStmt; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedExpr; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedFunctionCall; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedGetStructField; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedLiteral; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedOrderByScan; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedParameter; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedProjectScan; -import java.math.BigDecimal; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.beam.repackaged.core.org.apache.commons.lang3.reflect.FieldUtils; -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.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_28_0.org.apache.calcite.avatica.util.TimeUnit; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFieldImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelRecordType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIntervalQualifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlRowOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * Extracts expressions (function calls, field accesses) from the resolve query nodes, converts them - * to RexNodes. - */ -@Internal -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class ExpressionConverter { - - // Constants of pre-defined functions. - private static final String WINDOW_START = "_START"; - private static final String WINDOW_END = "_END"; - private static final String FIXED_WINDOW = "TUMBLE"; - private static final String FIXED_WINDOW_START = FIXED_WINDOW + WINDOW_START; - private static final String FIXED_WINDOW_END = FIXED_WINDOW + WINDOW_END; - private static final String SLIDING_WINDOW = "HOP"; - private static final String SLIDING_WINDOW_START = SLIDING_WINDOW + WINDOW_START; - private static final String SLIDING_WINDOW_END = SLIDING_WINDOW + WINDOW_END; - private static final String SESSION_WINDOW = "SESSION"; - private static final String SESSION_WINDOW_START = SESSION_WINDOW + WINDOW_START; - private static final String SESSION_WINDOW_END = SESSION_WINDOW + WINDOW_END; - - private static final ImmutableMap WINDOW_START_END_TO_WINDOW_MAP = - ImmutableMap.builder() - .put(FIXED_WINDOW_START, FIXED_WINDOW) - .put(FIXED_WINDOW_END, FIXED_WINDOW) - .put(SLIDING_WINDOW_START, SLIDING_WINDOW) - .put(SLIDING_WINDOW_END, SLIDING_WINDOW) - .put(SESSION_WINDOW_START, SESSION_WINDOW) - .put(SESSION_WINDOW_END, SESSION_WINDOW) - .build(); - - private static final ImmutableSet WINDOW_START_END_FUNCTION_SET = - ImmutableSet.of( - FIXED_WINDOW_START, - FIXED_WINDOW_END, - SLIDING_WINDOW_START, - SLIDING_WINDOW_END, - SESSION_WINDOW_START, - SESSION_WINDOW_END); - - private static final ImmutableMap> UNSUPPORTED_CASTING = - ImmutableMap.>builder() - .put(TYPE_INT64, ImmutableSet.of(TYPE_DOUBLE)) - .put(TYPE_BOOL, ImmutableSet.of(TYPE_STRING)) - .put(TYPE_STRING, ImmutableSet.of(TYPE_BOOL, TYPE_DOUBLE)) - .build(); - - private static final ImmutableSet DATE_PART_UNITS_TO_MILLIS = - ImmutableSet.of("DAY", "HOUR", "MINUTE", "SECOND"); - private static final ImmutableSet DATE_PART_UNITS_TO_MONTHS = ImmutableSet.of("YEAR"); - - private static final long ONE_SECOND_IN_MILLIS = 1000L; - private static final long ONE_MINUTE_IN_MILLIS = 60L * ONE_SECOND_IN_MILLIS; - private static final long ONE_HOUR_IN_MILLIS = 60L * ONE_MINUTE_IN_MILLIS; - private static final long ONE_DAY_IN_MILLIS = 24L * ONE_HOUR_IN_MILLIS; - - @SuppressWarnings("unused") - private static final long ONE_MONTH_IN_MILLIS = 30L * ONE_DAY_IN_MILLIS; - - @SuppressWarnings("unused") - private static final long ONE_YEAR_IN_MILLIS = 365L * ONE_DAY_IN_MILLIS; - - // Constants of error messages. - private static final String INTERVAL_DATE_PART_MSG = - "YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE, SECOND, MILLISECOND"; - private static final String INTERVAL_FORMAT_MSG = - "INTERVAL should be set as a STRING in the specific format: \"INTERVAL int64 date_part\"." - + " The date_part includes: " - + INTERVAL_DATE_PART_MSG; - - private final RelOptCluster cluster; - private final QueryParameters queryParams; - private int nullParamCount = 0; - final UserFunctionDefinitions userFunctionDefinitions; - - public ExpressionConverter( - RelOptCluster cluster, - QueryParameters params, - UserFunctionDefinitions userFunctionDefinitions) { - this.cluster = cluster; - this.queryParams = params; - this.userFunctionDefinitions = userFunctionDefinitions; - } - - /** Extract expressions from a project scan node. */ - public List retrieveRexNode(ResolvedProjectScan node, List fieldList) { - List ret = new ArrayList<>(); - - for (ResolvedColumn column : node.getColumnList()) { - int index = -1; - if ((index = indexOfResolvedColumnInExprList(node.getExprList(), column)) != -1) { - ResolvedComputedColumn computedColumn = node.getExprList().get(index); - int windowFieldIndex = -1; - if (computedColumn.getExpr().nodeKind() == RESOLVED_FUNCTION_CALL) { - String functionName = - ((ResolvedFunctionCall) computedColumn.getExpr()).getFunction().getName(); - if (WINDOW_START_END_FUNCTION_SET.contains(functionName)) { - ResolvedAggregateScan resolvedAggregateScan = - (ResolvedAggregateScan) node.getInputScan(); - windowFieldIndex = - indexOfWindowField( - resolvedAggregateScan.getGroupByList(), - resolvedAggregateScan.getColumnList(), - WINDOW_START_END_TO_WINDOW_MAP.get(functionName)); - } - } - ret.add( - convertRexNodeFromComputedColumnWithFieldList( - computedColumn, node.getInputScan().getColumnList(), fieldList, windowFieldIndex)); - } else { - // ResolvedColumn is not a expression, which means it has to be an input column reference. - index = indexOfProjectionColumnRef(column.getId(), node.getInputScan().getColumnList()); - if (index < 0 || index >= node.getInputScan().getColumnList().size()) { - throw new IllegalStateException( - String.format("Cannot find %s in fieldList %s", column, fieldList)); - } - - ret.add(rexBuilder().makeInputRef(fieldList.get(index).getType(), index)); - } - } - return ret; - } - - /** Extract expressions from order by scan node. */ - public List retrieveRexNodeFromOrderByScan( - RelOptCluster cluster, ResolvedOrderByScan node, List fieldList) { - final RexBuilder rexBuilder = cluster.getRexBuilder(); - List ret = new ArrayList<>(); - - for (ResolvedColumn column : node.getColumnList()) { - int index = indexOfProjectionColumnRef(column.getId(), node.getInputScan().getColumnList()); - ret.add(rexBuilder.makeInputRef(fieldList.get(index).getType(), index)); - } - - return ret; - } - - private static int indexOfResolvedColumnInExprList( - ImmutableList exprList, ResolvedColumn column) { - if (exprList == null || exprList.isEmpty()) { - return -1; - } - - for (int i = 0; i < exprList.size(); i++) { - ResolvedComputedColumn computedColumn = exprList.get(i); - if (computedColumn.getColumn().equals(column)) { - return i; - } - } - - return -1; - } - - private static int indexOfWindowField( - List groupByList, List columnList, String windowFn) { - for (ResolvedComputedColumn groupByComputedColumn : groupByList) { - if (groupByComputedColumn.getExpr().nodeKind() == RESOLVED_FUNCTION_CALL) { - ResolvedFunctionCall functionCall = (ResolvedFunctionCall) groupByComputedColumn.getExpr(); - if (functionCall.getFunction().getName().equals(windowFn)) { - int ret = - indexOfResolvedColumnInColumnList(columnList, groupByComputedColumn.getColumn()); - if (ret == -1) { - throw new IllegalStateException("Cannot find " + windowFn + " in " + groupByList); - } else { - return ret; - } - } - } - } - - throw new IllegalStateException("Cannot find " + windowFn + " in " + groupByList); - } - - private static int indexOfResolvedColumnInColumnList( - List columnList, ResolvedColumn column) { - if (columnList == null || columnList.isEmpty()) { - return -1; - } - - for (int i = 0; i < columnList.size(); i++) { - if (columnList.get(i).equals(column)) { - return i; - } - } - - return -1; - } - - /** Create a RexNode for a corresponding resolved expression node. */ - public RexNode convertRexNodeFromResolvedExpr( - ResolvedExpr expr, - List columnList, - List fieldList, - Map functionArguments) { - if (columnList == null || fieldList == null) { - return convertRexNodeFromResolvedExpr(expr); - } - - RexNode ret; - - switch (expr.nodeKind()) { - case RESOLVED_LITERAL: - ret = convertResolvedLiteral((ResolvedLiteral) expr); - break; - case RESOLVED_COLUMN_REF: - ret = convertResolvedColumnRef((ResolvedColumnRef) expr, columnList, fieldList); - break; - case RESOLVED_FUNCTION_CALL: - ret = - convertResolvedFunctionCall( - (ResolvedFunctionCall) expr, columnList, fieldList, functionArguments); - break; - case RESOLVED_CAST: - ret = convertResolvedCast((ResolvedCast) expr, columnList, fieldList, functionArguments); - break; - case RESOLVED_PARAMETER: - ret = convertResolvedParameter((ResolvedParameter) expr); - break; - case RESOLVED_GET_STRUCT_FIELD: - ret = - convertResolvedStructFieldAccess( - (ResolvedGetStructField) expr, columnList, fieldList, functionArguments); - break; - case RESOLVED_ARGUMENT_REF: - ret = convertResolvedArgumentRef((ResolvedArgumentRef) expr, functionArguments); - break; - default: - ret = convertRexNodeFromResolvedExpr(expr); - } - - return ret; - } - - public RexNode convertRelNodeToRexRangeRef(RelNode rel) { - return rexBuilder().makeRangeReference(rel); - } - - /** Create a RexNode for a corresponding resolved expression. */ - public RexNode convertRexNodeFromResolvedExpr(ResolvedExpr expr) { - RexNode ret; - - switch (expr.nodeKind()) { - case RESOLVED_LITERAL: - ret = convertResolvedLiteral((ResolvedLiteral) expr); - break; - case RESOLVED_COLUMN_REF: - ret = convertResolvedColumnRef((ResolvedColumnRef) expr); - break; - case RESOLVED_FUNCTION_CALL: - // TODO: is there a better way to shared code for different cases of - // convertResolvedFunctionCall than passing nulls? - ret = - convertResolvedFunctionCall((ResolvedFunctionCall) expr, null, null, ImmutableMap.of()); - break; - case RESOLVED_CAST: - ret = convertResolvedCast((ResolvedCast) expr, null, null, ImmutableMap.of()); - break; - case RESOLVED_PARAMETER: - ret = convertResolvedParameter((ResolvedParameter) expr); - break; - case RESOLVED_GET_STRUCT_FIELD: - ret = convertResolvedStructFieldAccess((ResolvedGetStructField) expr); - break; - case RESOLVED_SUBQUERY_EXPR: - throw new UnsupportedOperationException("Does not support sub-queries"); - default: - throw new UnsupportedOperationException( - "Does not support expr node kind " + expr.nodeKind()); - } - - return ret; - } - - private RexNode convertRexNodeFromComputedColumnWithFieldList( - ResolvedComputedColumn column, - List columnList, - List fieldList, - int windowFieldIndex) { - if (column.getExpr().nodeKind() != RESOLVED_FUNCTION_CALL) { - return convertRexNodeFromResolvedExpr( - column.getExpr(), columnList, fieldList, ImmutableMap.of()); - } - - ResolvedFunctionCall functionCall = (ResolvedFunctionCall) column.getExpr(); - - // TODO: is there any other illegal case? - if (functionCall.getFunction().getName().equals(FIXED_WINDOW) - || functionCall.getFunction().getName().equals(SLIDING_WINDOW) - || functionCall.getFunction().getName().equals(SESSION_WINDOW)) { - throw new ZetaSqlException( - functionCall.getFunction().getName() + " shouldn't appear in SELECT exprlist."); - } - - if (!functionCall.getFunction().getGroup().equals(PRE_DEFINED_WINDOW_FUNCTIONS)) { - // non-window function should still go through normal FunctionCall conversion process. - return convertRexNodeFromResolvedExpr( - column.getExpr(), columnList, fieldList, ImmutableMap.of()); - } - - // ONLY window_start and window_end should arrive here. - // TODO: Have extra verification here to make sure window start/end functions have the same - // parameter with window function. - List operands = new ArrayList<>(); - switch (functionCall.getFunction().getName()) { - case FIXED_WINDOW_START: - case SLIDING_WINDOW_START: - case SESSION_WINDOW_START: - // TODO: in Calcite implementation, session window's start is equal to end. Need to fix it - // in Calcite. - case SESSION_WINDOW_END: - return rexBuilder() - .makeInputRef(fieldList.get(windowFieldIndex).getType(), windowFieldIndex); - case FIXED_WINDOW_END: - operands.add( - rexBuilder().makeInputRef(fieldList.get(windowFieldIndex).getType(), windowFieldIndex)); - // TODO: check window_end 's duration is the same as it's aggregate window. - operands.add( - convertIntervalToRexIntervalLiteral( - (ResolvedLiteral) functionCall.getArgumentList().get(0))); - return rexBuilder().makeCall(SqlOperators.ZETASQL_TIMESTAMP_ADD, operands); - case SLIDING_WINDOW_END: - operands.add( - rexBuilder().makeInputRef(fieldList.get(windowFieldIndex).getType(), windowFieldIndex)); - operands.add( - convertIntervalToRexIntervalLiteral( - (ResolvedLiteral) functionCall.getArgumentList().get(1))); - return rexBuilder().makeCall(SqlOperators.ZETASQL_TIMESTAMP_ADD, operands); - default: - throw new UnsupportedOperationException( - "Does not support window start/end: " + functionCall.getFunction().getName()); - } - } - - public RexNode trueLiteral() { - return rexBuilder().makeLiteral(true); - } - - /** Convert a resolved literal to a RexNode. */ - public RexNode convertResolvedLiteral(ResolvedLiteral resolvedLiteral) { - return ZetaSqlCalciteTranslationUtils.toRexNode(resolvedLiteral.getValue(), rexBuilder()); - } - - /** Convert a TableValuedFunction in ZetaSQL to a RexCall in Calcite. */ - public RexCall convertTableValuedFunction( - RelNode input, - TableValuedFunction tvf, - List argumentList, - List inputTableColumns) { - ResolvedColumn wmCol; - // Handle builtin windowing TVF. - switch (tvf.getName()) { - case TVFStreamingUtils.FIXED_WINDOW_TVF: - // TUMBLE tvf's second argument is descriptor. - wmCol = extractWatermarkColumnFromDescriptor(argumentList.get(1).getDescriptorArg()); - - return (RexCall) - rexBuilder() - .makeCall( - new SqlWindowTableFunction(SqlKind.TUMBLE.name()), - convertRelNodeToRexRangeRef(input), - convertResolvedColumnToRexInputRef(wmCol, inputTableColumns), - convertIntervalToRexIntervalLiteral( - (ResolvedLiteral) argumentList.get(2).getExpr())); - - case TVFStreamingUtils.SLIDING_WINDOW_TVF: - // HOP tvf's second argument is descriptor. - wmCol = extractWatermarkColumnFromDescriptor(argumentList.get(1).getDescriptorArg()); - return (RexCall) - rexBuilder() - .makeCall( - new SqlWindowTableFunction(SqlKind.HOP.name()), - convertRelNodeToRexRangeRef(input), - convertResolvedColumnToRexInputRef(wmCol, inputTableColumns), - convertIntervalToRexIntervalLiteral( - (ResolvedLiteral) argumentList.get(2).getExpr()), - convertIntervalToRexIntervalLiteral( - (ResolvedLiteral) argumentList.get(3).getExpr())); - - case TVFStreamingUtils.SESSION_WINDOW_TVF: - // SESSION tvf's second argument is descriptor. - wmCol = extractWatermarkColumnFromDescriptor(argumentList.get(1).getDescriptorArg()); - // SESSION tvf's third argument is descriptor. - List keyCol = - extractSessionKeyColumnFromDescriptor(argumentList.get(2).getDescriptorArg()); - List operands = new ArrayList<>(); - operands.add(convertRelNodeToRexRangeRef(input)); - operands.add(convertResolvedColumnToRexInputRef(wmCol, inputTableColumns)); - operands.add( - convertIntervalToRexIntervalLiteral((ResolvedLiteral) argumentList.get(3).getExpr())); - operands.addAll(convertResolvedColumnsToRexInputRef(keyCol, inputTableColumns)); - return (RexCall) - rexBuilder().makeCall(new SqlWindowTableFunction(SqlKind.SESSION.name()), operands); - } - - if (tvf instanceof FixedOutputSchemaTVF) { - FixedOutputSchemaTVF fixedOutputSchemaTVF = (FixedOutputSchemaTVF) tvf; - return (RexCall) - rexBuilder() - .makeCall( - new ZetaSqlUserDefinedSQLNativeTableValuedFunction( - new SqlIdentifier(tvf.getName(), SqlParserPos.ZERO), - opBinding -> { - TVFRelation rel = fixedOutputSchemaTVF.getOutputSchema(); - // TODO(yathu) revert this workaround when ZetaSQL adds back this API. - List cols; - try { - cols = (List) FieldUtils.readField(rel, "columns", true); - } catch (IllegalAccessException e) { - throw new RuntimeException(e); - } - List relDataTypeFields = - convertTVFRelationColumnsToRelDataTypeFields(cols); - return new RelRecordType(relDataTypeFields); - }, - null, - null, - null, - null)); - } - - throw new UnsupportedOperationException( - "Does not support table-valued function: " + tvf.getName()); - } - - private List convertTVFRelationColumnsToRelDataTypeFields( - List columns) { - return IntStream.range(0, columns.size()) - .mapToObj( - i -> - new RelDataTypeFieldImpl( - columns.get(i).getName(), - i, - ZetaSqlCalciteTranslationUtils.toCalciteType( - columns.get(i).getType(), false, rexBuilder()))) - .collect(Collectors.toList()); - } - - private List convertResolvedColumnsToRexInputRef( - List columns, List inputTableColumns) { - List ret = new ArrayList<>(); - for (ResolvedColumn column : columns) { - ret.add(convertResolvedColumnToRexInputRef(column, inputTableColumns)); - } - return ret; - } - - private RexInputRef convertResolvedColumnToRexInputRef( - ResolvedColumn column, List inputTableColumns) { - for (int i = 0; i < inputTableColumns.size(); i++) { - if (inputTableColumns.get(i).equals(column)) { - return rexBuilder() - .makeInputRef( - ZetaSqlCalciteTranslationUtils.toCalciteType(column.getType(), false, rexBuilder()), - i); - } - } - - throw new IllegalArgumentException( - "ZetaSQL parser guarantees that wmCol can be found from inputTableColumns so it shouldn't reach here."); - } - - private ResolvedColumn extractWatermarkColumnFromDescriptor( - ResolvedNodes.ResolvedDescriptor descriptor) { - ResolvedColumn wmCol = descriptor.getDescriptorColumnList().get(0); - checkArgument( - wmCol.getType().getKind() == TYPE_TIMESTAMP, - "Watermarked column should be TIMESTAMP type: %s", - descriptor.getDescriptorColumnNameList().get(0)); - return wmCol; - } - - private List extractSessionKeyColumnFromDescriptor( - ResolvedNodes.ResolvedDescriptor descriptor) { - checkArgument( - descriptor.getDescriptorColumnNameList().size() > 0, - "Session key descriptor should not be empty"); - - return descriptor.getDescriptorColumnList(); - } - - private RexNode convertResolvedColumnRef( - ResolvedColumnRef columnRef, - List columnList, - List fieldList) { - int index = indexOfProjectionColumnRef(columnRef.getColumn().getId(), columnList); - if (index < 0 || index >= columnList.size()) { - throw new IllegalStateException( - String.format("Cannot find %s in fieldList %s", columnRef.getColumn(), fieldList)); - } - return rexBuilder().makeInputRef(fieldList.get(index).getType(), index); - } - - private RexNode convertResolvedColumnRef(ResolvedColumnRef columnRef) { - // TODO: id - 1 might be only correct if the columns read from TableScan. - // What if the columns come from other scans (which means their id are not indexed from 0), - // and what if there are some mis-order? - // TODO: can join key be NULL? - return rexBuilder() - .makeInputRef( - ZetaSqlCalciteTranslationUtils.toCalciteType(columnRef.getType(), false, rexBuilder()), - (int) columnRef.getColumn().getId() - 1); - } - - /** Return an index of the projection column reference. */ - public int indexOfProjectionColumnRef(long colId, List columnList) { - int ret = -1; - for (int i = 0; i < columnList.size(); i++) { - if (columnList.get(i).getId() == colId) { - ret = i; - break; - } - } - - return ret; - } - - private RexNode convertResolvedFunctionCall( - ResolvedFunctionCall functionCall, - @Nullable List columnList, - @Nullable List fieldList, - Map outerFunctionArguments) { - final String funGroup = functionCall.getFunction().getGroup(); - final String funName = functionCall.getFunction().getName(); - SqlOperator op = SqlOperatorMappingTable.create(functionCall); - List operands = new ArrayList<>(); - - if (PRE_DEFINED_WINDOW_FUNCTIONS.equals(funGroup)) { - switch (funName) { - case FIXED_WINDOW: - case SESSION_WINDOW: - // TODO: check size and type of window function argument list. - // Add ts column reference to operands. - operands.add( - convertRexNodeFromResolvedExpr( - functionCall.getArgumentList().get(0), - columnList, - fieldList, - outerFunctionArguments)); - // Add fixed window size or session window gap to operands. - operands.add( - convertIntervalToRexIntervalLiteral( - (ResolvedLiteral) functionCall.getArgumentList().get(1))); - break; - case SLIDING_WINDOW: - // Add ts column reference to operands. - operands.add( - convertRexNodeFromResolvedExpr( - functionCall.getArgumentList().get(0), - columnList, - fieldList, - outerFunctionArguments)); - // add sliding window emit frequency to operands. - operands.add( - convertIntervalToRexIntervalLiteral( - (ResolvedLiteral) functionCall.getArgumentList().get(1))); - // add sliding window size to operands. - operands.add( - convertIntervalToRexIntervalLiteral( - (ResolvedLiteral) functionCall.getArgumentList().get(2))); - break; - default: - throw new UnsupportedOperationException( - "Unsupported function: " + funName + ". Only support TUMBLE, HOP, and SESSION now."); - } - } else if (ZETASQL_FUNCTION_GROUP_NAME.equals(funGroup)) { - if (op == null) { - Type returnType = functionCall.getSignature().getResultType().getType(); - if (returnType != null) { - op = - SqlOperators.createZetaSqlFunction( - funName, - ZetaSqlCalciteTranslationUtils.toCalciteType(returnType, false, rexBuilder()) - .getSqlTypeName()); - } else { - throw new UnsupportedOperationException("Does not support ZetaSQL function: " + funName); - } - } - - for (ResolvedExpr expr : functionCall.getArgumentList()) { - operands.add( - convertRexNodeFromResolvedExpr(expr, columnList, fieldList, outerFunctionArguments)); - } - } else if (USER_DEFINED_JAVA_SCALAR_FUNCTIONS.equals(funGroup)) { - UserFunctionDefinitions.JavaScalarFunction javaScalarFunction = - userFunctionDefinitions - .javaScalarFunctions() - .get(functionCall.getFunction().getNamePath()); - ArrayList innerFunctionArguments = new ArrayList<>(); - for (int i = 0; i < functionCall.getArgumentList().size(); i++) { - ResolvedExpr argExpr = functionCall.getArgumentList().get(i); - RexNode argNode = - convertRexNodeFromResolvedExpr(argExpr, columnList, fieldList, outerFunctionArguments); - innerFunctionArguments.add(argNode); - } - return rexBuilder() - .makeCall( - SqlOperators.createUdfOperator( - functionCall.getFunction().getName(), - javaScalarFunction.method(), - USER_DEFINED_JAVA_SCALAR_FUNCTIONS, - javaScalarFunction.jarPath()), - innerFunctionArguments); - } else if (USER_DEFINED_SQL_FUNCTIONS.equals(funGroup)) { - ResolvedCreateFunctionStmt createFunctionStmt = - userFunctionDefinitions - .sqlScalarFunctions() - .get(functionCall.getFunction().getNamePath()); - ResolvedExpr functionExpression = createFunctionStmt.getFunctionExpression(); - ImmutableMap.Builder innerFunctionArguments = ImmutableMap.builder(); - for (int i = 0; i < functionCall.getArgumentList().size(); i++) { - String argName = createFunctionStmt.getArgumentNameList().get(i); - ResolvedExpr argExpr = functionCall.getArgumentList().get(i); - RexNode argNode = - convertRexNodeFromResolvedExpr(argExpr, columnList, fieldList, outerFunctionArguments); - innerFunctionArguments.put(argName, argNode); - } - return this.convertRexNodeFromResolvedExpr( - functionExpression, columnList, fieldList, innerFunctionArguments.build()); - } else { - throw new UnsupportedOperationException("Does not support function group: " + funGroup); - } - - SqlOperatorRewriter rewriter = - SqlOperatorMappingTable.ZETASQL_FUNCTION_TO_CALCITE_SQL_OPERATOR_REWRITER.get(funName); - - if (rewriter != null) { - return rewriter.apply(rexBuilder(), operands); - } else { - return rexBuilder().makeCall(op, operands); - } - } - - private RexNode convertIntervalToRexIntervalLiteral(ResolvedLiteral resolvedLiteral) { - if (resolvedLiteral.getType().getKind() != TYPE_STRING) { - throw new ZetaSqlException(INTERVAL_FORMAT_MSG); - } - - String valStr = resolvedLiteral.getValue().getStringValue(); - List stringList = - Arrays.stream(valStr.split(" ")).filter(s -> !s.isEmpty()).collect(Collectors.toList()); - - if (stringList.size() != 3) { - throw new ZetaSqlException(INTERVAL_FORMAT_MSG); - } - - if (!Ascii.toUpperCase(stringList.get(0)).equals("INTERVAL")) { - throw new ZetaSqlException(INTERVAL_FORMAT_MSG); - } - - long intervalValue; - try { - intervalValue = Long.parseLong(stringList.get(1)); - } catch (NumberFormatException e) { - throw new ZetaSqlException( - Status.UNIMPLEMENTED - .withDescription(INTERVAL_FORMAT_MSG) - .withCause(e) - .asRuntimeException()); - } - - String intervalDatepart = Ascii.toUpperCase(stringList.get(2)); - return createCalciteIntervalRexLiteral(intervalValue, intervalDatepart); - } - - private RexLiteral createCalciteIntervalRexLiteral(long intervalValue, String intervalTimeUnit) { - SqlIntervalQualifier sqlIntervalQualifier = - convertIntervalDatepartToSqlIntervalQualifier(intervalTimeUnit); - BigDecimal decimalValue; - if (DATE_PART_UNITS_TO_MILLIS.contains(intervalTimeUnit)) { - decimalValue = convertIntervalValueToMillis(sqlIntervalQualifier, intervalValue); - } else if (DATE_PART_UNITS_TO_MONTHS.contains(intervalTimeUnit)) { - decimalValue = new BigDecimal(intervalValue * 12); - } else { - decimalValue = new BigDecimal(intervalValue); - } - return rexBuilder().makeIntervalLiteral(decimalValue, sqlIntervalQualifier); - } - - private static BigDecimal convertIntervalValueToMillis( - SqlIntervalQualifier qualifier, long value) { - switch (qualifier.typeName()) { - case INTERVAL_DAY: - return new BigDecimal(value * ONE_DAY_IN_MILLIS); - case INTERVAL_HOUR: - return new BigDecimal(value * ONE_HOUR_IN_MILLIS); - case INTERVAL_MINUTE: - return new BigDecimal(value * ONE_MINUTE_IN_MILLIS); - case INTERVAL_SECOND: - return new BigDecimal(value * ONE_SECOND_IN_MILLIS); - default: - throw new ZetaSqlException(qualifier.typeName().toString()); - } - } - - private static SqlIntervalQualifier convertIntervalDatepartToSqlIntervalQualifier( - String datePart) { - switch (datePart) { - case "YEAR": - return new SqlIntervalQualifier(TimeUnit.YEAR, null, SqlParserPos.ZERO); - case "MONTH": - return new SqlIntervalQualifier(TimeUnit.MONTH, null, SqlParserPos.ZERO); - case "DAY": - return new SqlIntervalQualifier(TimeUnit.DAY, null, SqlParserPos.ZERO); - case "HOUR": - return new SqlIntervalQualifier(TimeUnit.HOUR, null, SqlParserPos.ZERO); - case "MINUTE": - return new SqlIntervalQualifier(TimeUnit.MINUTE, null, SqlParserPos.ZERO); - case "SECOND": - return new SqlIntervalQualifier(TimeUnit.SECOND, null, SqlParserPos.ZERO); - case "WEEK": - return new SqlIntervalQualifier(TimeUnit.WEEK, null, SqlParserPos.ZERO); - case "QUARTER": - return new SqlIntervalQualifier(TimeUnit.QUARTER, null, SqlParserPos.ZERO); - case "MILLISECOND": - return new SqlIntervalQualifier(TimeUnit.MILLISECOND, null, SqlParserPos.ZERO); - default: - throw new ZetaSqlException( - String.format( - "Received an undefined INTERVAL unit: %s. Please specify unit from the following" - + " list: %s.", - datePart, INTERVAL_DATE_PART_MSG)); - } - } - - private RexNode convertResolvedCast( - ResolvedCast resolvedCast, - List columnList, - List fieldList, - Map functionArguments) { - return convertResolvedCast( - resolvedCast, - convertRexNodeFromResolvedExpr( - resolvedCast.getExpr(), columnList, fieldList, functionArguments)); - } - - private RexNode convertResolvedCast(ResolvedCast resolvedCast, RexNode input) { - TypeKind fromType = resolvedCast.getExpr().getType().getKind(); - TypeKind toType = resolvedCast.getType().getKind(); - isCastingSupported(fromType, toType, input); - - // nullability of the output type should match that of the input node's type - RelDataType outputType = - ZetaSqlCalciteTranslationUtils.toCalciteType( - resolvedCast.getType(), input.getType().isNullable(), rexBuilder()); - - if (isZetaSQLCast(fromType, toType)) { - return rexBuilder().makeCall(outputType, SqlOperators.CAST_OP, ImmutableList.of(input)); - } else { - return rexBuilder().makeCast(outputType, input); - } - } - - private static void isCastingSupported(TypeKind fromType, TypeKind toType, RexNode input) { - if (UNSUPPORTED_CASTING.containsKey(toType) - && UNSUPPORTED_CASTING.get(toType).contains(fromType)) { - throw new UnsupportedOperationException( - "Does not support CAST(" + fromType + " AS " + toType + ")"); - } - if (fromType.equals(TYPE_DOUBLE) - && toType.equals(TYPE_NUMERIC) - && input instanceof RexLiteral) { - BigDecimal value = (BigDecimal) ((RexLiteral) input).getValue(); - if (value.compareTo(ZetaSqlCalciteTranslationUtils.ZETASQL_NUMERIC_MAX_VALUE) > 0) { - 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 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 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()); - } - } - } - - private static boolean isZetaSQLCast(TypeKind fromType, TypeKind toType) { - // TODO: Structure CAST_OP so that we don't have to repeat the supported types - // here - return (fromType.equals(TYPE_BYTES) && toType.equals(TYPE_STRING)) - || (fromType.equals(TYPE_INT64) && toType.equals(TYPE_BOOL)) - || (fromType.equals(TYPE_BOOL) && toType.equals(TYPE_INT64)) - || (fromType.equals(TYPE_TIMESTAMP) && toType.equals(TYPE_STRING)); - } - - private RexNode convertResolvedParameter(ResolvedParameter parameter) { - Value value; - switch (queryParams.getKind()) { - case NAMED: - value = ((Map) queryParams.named()).get(parameter.getName()); - break; - case POSITIONAL: - // parameter is 1-indexed, while parameter list is 0-indexed. - value = ((List) queryParams.positional()).get((int) parameter.getPosition() - 1); - break; - default: - throw new IllegalArgumentException("Found unexpected parameter " + parameter); - } - Preconditions.checkState(parameter.getType().equals(value.getType())); - if (value.isNull()) { - // In some cases NULL parameter cannot be substituted with NULL literal - // Therefore we create a dynamic parameter placeholder here for each NULL parameter - return rexBuilder() - .makeDynamicParam( - ZetaSqlCalciteTranslationUtils.toCalciteType(value.getType(), true, rexBuilder()), - nullParamCount++); - } else { - // Substitute non-NULL parameter with literal - return ZetaSqlCalciteTranslationUtils.toRexNode(value, rexBuilder()); - } - } - - private RexNode convertResolvedArgumentRef( - ResolvedArgumentRef resolvedArgumentRef, Map functionArguments) { - return functionArguments.get(resolvedArgumentRef.getName()); - } - - private RexNode convertResolvedStructFieldAccess(ResolvedGetStructField resolvedGetStructField) { - RexNode referencedExpr = convertRexNodeFromResolvedExpr(resolvedGetStructField.getExpr()); - return convertResolvedStructFieldAccessInternal( - referencedExpr, (int) resolvedGetStructField.getFieldIdx()); - } - - private RexNode convertResolvedStructFieldAccess( - ResolvedGetStructField resolvedGetStructField, - List columnList, - List fieldList, - Map functionArguments) { - RexNode referencedExpr = - convertRexNodeFromResolvedExpr( - resolvedGetStructField.getExpr(), columnList, fieldList, functionArguments); - return convertResolvedStructFieldAccessInternal( - referencedExpr, (int) resolvedGetStructField.getFieldIdx()); - } - - private RexNode convertResolvedStructFieldAccessInternal(RexNode referencedExpr, int fieldIdx) { - // Calcite SQL does not allow the ROW constructor to be dereferenced directly, so do it here. - if (referencedExpr instanceof RexCall - && ((RexCall) referencedExpr).getOperator() instanceof SqlRowOperator) { - return ((RexCall) referencedExpr).getOperands().get(fieldIdx); - } - return rexBuilder().makeFieldAccess(referencedExpr, fieldIdx); - } - - private RexBuilder rexBuilder() { - return cluster.getRexBuilder(); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/FilterScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/FilterScanConverter.java deleted file mode 100644 index 6a4208e808ca..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/FilterScanConverter.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedFilterScan; -import java.util.Collections; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalFilter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; - -/** Converts filter. */ -class FilterScanConverter extends RelConverter { - - FilterScanConverter(ConversionContext context) { - super(context); - } - - @Override - public List getInputs(ResolvedFilterScan zetaNode) { - return Collections.singletonList(zetaNode.getInputScan()); - } - - @Override - public RelNode convert(ResolvedFilterScan zetaNode, List inputs) { - RelNode input = inputs.get(0); - RexNode condition = - getExpressionConverter() - .convertRexNodeFromResolvedExpr( - zetaNode.getFilterExpr(), - zetaNode.getInputScan().getColumnList(), - input.getRowType().getFieldList(), - context.getFunctionArgumentRefMapping()); - - return LogicalFilter.create(input, condition); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/JoinScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/JoinScanConverter.java deleted file mode 100644 index 085395725fbb..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/JoinScanConverter.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedColumn; -import com.google.zetasql.resolvedast.ResolvedJoinScanEnums.JoinType; -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedJoinScan; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalJoin; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; - -/** Converts joins if neither side of the join is a WithRefScan. */ -class JoinScanConverter extends RelConverter { - - private static final ImmutableMap JOIN_TYPES = - ImmutableMap.of( - JoinType.INNER, - JoinRelType.INNER, - JoinType.FULL, - JoinRelType.FULL, - JoinType.LEFT, - JoinRelType.LEFT, - JoinType.RIGHT, - JoinRelType.RIGHT); - - JoinScanConverter(ConversionContext context) { - super(context); - } - - @Override - public boolean canConvert(ResolvedJoinScan zetaNode) { - return true; - } - - @Override - public List getInputs(ResolvedJoinScan zetaNode) { - return ImmutableList.of(zetaNode.getLeftScan(), zetaNode.getRightScan()); - } - - @Override - public RelNode convert(ResolvedJoinScan zetaNode, List inputs) { - RelNode convertedLeftInput = inputs.get(0); - RelNode convertedRightInput = inputs.get(1); - - List combinedZetaFieldsList = - ImmutableList.builder() - .addAll(zetaNode.getLeftScan().getColumnList()) - .addAll(zetaNode.getRightScan().getColumnList()) - .build(); - - List combinedCalciteFieldsList = - ImmutableList.builder() - .addAll(convertedLeftInput.getRowType().getFieldList()) - .addAll(convertedRightInput.getRowType().getFieldList()) - .build(); - - final RexNode condition; - if (zetaNode.getJoinExpr() == null) { - condition = getExpressionConverter().trueLiteral(); - } else { - condition = - getExpressionConverter() - .convertRexNodeFromResolvedExpr( - zetaNode.getJoinExpr(), - combinedZetaFieldsList, - combinedCalciteFieldsList, - ImmutableMap.of()); - } - - return LogicalJoin.create( - convertedLeftInput, - convertedRightInput, - ImmutableList.of(), - condition, - ImmutableSet.of(), - convertResolvedJoinType(zetaNode.getJoinType())); - } - - static JoinRelType convertResolvedJoinType(JoinType joinType) { - if (!JOIN_TYPES.containsKey(joinType)) { - throw new UnsupportedOperationException("JOIN type: " + joinType + " is unsupported."); - } - - return JOIN_TYPES.get(joinType); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java deleted file mode 100644 index 5cb9569ba074..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedLimitOffsetScan; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedOrderByScan; -import java.util.Collections; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollations; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalSort; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexDynamicParam; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -/** Converts LIMIT without ORDER BY. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class LimitOffsetScanToLimitConverter extends RelConverter { - - LimitOffsetScanToLimitConverter(ConversionContext context) { - super(context); - } - - @Override - public boolean canConvert(ResolvedLimitOffsetScan zetaNode) { - return !(zetaNode.getInputScan() instanceof ResolvedOrderByScan); - } - - @Override - public List getInputs(ResolvedLimitOffsetScan zetaNode) { - return Collections.singletonList(zetaNode.getInputScan()); - } - - @Override - public RelNode convert(ResolvedLimitOffsetScan zetaNode, List inputs) { - RelNode input = inputs.get(0); - RelCollation relCollation = RelCollations.of(ImmutableList.of()); - RexNode offset = - zetaNode.getOffset() == null - ? null - : getExpressionConverter().convertRexNodeFromResolvedExpr(zetaNode.getOffset()); - RexNode fetch = - getExpressionConverter() - .convertRexNodeFromResolvedExpr( - zetaNode.getLimit(), - zetaNode.getColumnList(), - input.getRowType().getFieldList(), - ImmutableMap.of()); - - // offset or fetch being RexDynamicParam means it is NULL (the only param supported currently) - if (offset instanceof RexDynamicParam - || RexLiteral.isNullLiteral(offset) - || fetch instanceof RexDynamicParam - || RexLiteral.isNullLiteral(fetch)) { - throw new UnsupportedOperationException("Limit requires non-null count and offset"); - } - - return LogicalSort.create(input, relCollation, offset, fetch); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java deleted file mode 100644 index 10ff26a9bb4c..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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.translation; - -import static java.util.stream.Collectors.toList; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING; -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelFieldCollation.Direction.DESCENDING; - -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedLimitOffsetScan; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedOrderByItem; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedOrderByScan; -import java.util.Collections; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelCollationImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelFieldCollation; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelFieldCollation.Direction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalProject; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalSort; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -/** Converts ORDER BY LIMIT. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class LimitOffsetScanToOrderByLimitConverter extends RelConverter { - - LimitOffsetScanToOrderByLimitConverter(ConversionContext context) { - super(context); - } - - @Override - public boolean canConvert(ResolvedLimitOffsetScan zetaNode) { - return zetaNode.getInputScan() instanceof ResolvedOrderByScan; - } - - @Override - public List getInputs(ResolvedLimitOffsetScan zetaNode) { - // The immediate input is the ORDER BY scan which we don't support, - // but we can handle the ORDER BY LIMIT if we know the underlying projection, for example. - return Collections.singletonList( - ((ResolvedOrderByScan) zetaNode.getInputScan()).getInputScan()); - } - - @Override - public RelNode convert(ResolvedLimitOffsetScan zetaNode, List inputs) { - ResolvedOrderByScan inputOrderByScan = (ResolvedOrderByScan) zetaNode.getInputScan(); - RelNode input = inputs.get(0); - RelCollation relCollation = getRelCollation(inputOrderByScan); - - RexNode offset = - zetaNode.getOffset() == null - ? null - : getExpressionConverter().convertRexNodeFromResolvedExpr(zetaNode.getOffset()); - RexNode fetch = - getExpressionConverter() - .convertRexNodeFromResolvedExpr( - zetaNode.getLimit(), - zetaNode.getColumnList(), - input.getRowType().getFieldList(), - ImmutableMap.of()); - - if (RexLiteral.isNullLiteral(offset) || RexLiteral.isNullLiteral(fetch)) { - throw new UnsupportedOperationException("Limit requires non-null count and offset"); - } - - RelNode sorted = LogicalSort.create(input, relCollation, offset, fetch); - return convertOrderByScanToLogicalScan(inputOrderByScan, sorted); - } - - /** Collation is a sort order, as in ORDER BY DESCENDING/ASCENDING. */ - private static RelCollation getRelCollation(ResolvedOrderByScan node) { - final long inputOffset = node.getInputScan().getColumnList().get(0).getId(); - List fieldCollations = - node.getOrderByItemList().stream() - .map(item -> orderByItemToFieldCollation(item, inputOffset)) - .collect(toList()); - return RelCollationImpl.of(fieldCollations); - } - - private static RelFieldCollation orderByItemToFieldCollation( - ResolvedOrderByItem item, long inputOffset) { - Direction sortDirection = item.getIsDescending() ? DESCENDING : ASCENDING; - final long fieldIndex = item.getColumnRef().getColumn().getId() - inputOffset; - return new RelFieldCollation((int) fieldIndex, sortDirection); - } - - private RelNode convertOrderByScanToLogicalScan(ResolvedOrderByScan node, RelNode input) { - List projects = - getExpressionConverter() - .retrieveRexNodeFromOrderByScan(getCluster(), node, input.getRowType().getFieldList()); - List fieldNames = getTrait().retrieveFieldNames(node.getColumnList()); - - return LogicalProject.create(input, ImmutableList.of(), projects, fieldNames); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/OrderByScanUnsupportedConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/OrderByScanUnsupportedConverter.java deleted file mode 100644 index 63bc77cc6812..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/OrderByScanUnsupportedConverter.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedOrderByScan; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; - -/** - * Always throws exception, represents the case when order by is used without limit. - * - *

Order by limit is a special case that is handled in {@link LimitOffsetScanToLimitConverter}. - */ -class OrderByScanUnsupportedConverter extends RelConverter { - - OrderByScanUnsupportedConverter(ConversionContext context) { - super(context); - } - - @Override - public RelNode convert(ResolvedOrderByScan zetaNode, List inputs) { - throw new UnsupportedOperationException("ORDER BY without a LIMIT is not supported."); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ProjectScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ProjectScanConverter.java deleted file mode 100644 index 49a1f2dbd4d9..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ProjectScanConverter.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedProjectScan; -import java.util.Collections; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalProject; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; - -/** Converts projection. */ -class ProjectScanConverter extends RelConverter { - - ProjectScanConverter(ConversionContext context) { - super(context); - } - - @Override - public List getInputs(ResolvedProjectScan zetaNode) { - return Collections.singletonList(zetaNode.getInputScan()); - } - - @Override - public RelNode convert(ResolvedProjectScan zetaNode, List inputs) { - RelNode input = inputs.get(0); - - List projects = - getExpressionConverter().retrieveRexNode(zetaNode, input.getRowType().getFieldList()); - List fieldNames = getTrait().retrieveFieldNames(zetaNode.getColumnList()); - return LogicalProject.create(input, ImmutableList.of(), projects, fieldNames); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/QueryStatementConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/QueryStatementConverter.java deleted file mode 100644 index e3d9042dcfe1..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/QueryStatementConverter.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * 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.translation; - -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_AGGREGATE_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_ARRAY_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_FILTER_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_JOIN_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_LIMIT_OFFSET_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_ORDER_BY_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_PROJECT_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_SET_OPERATION_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_SINGLE_ROW_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_TABLE_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_TVFSCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_WITH_REF_SCAN; -import static com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind.RESOLVED_WITH_SCAN; -import static java.util.stream.Collectors.toList; - -import com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind; -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedQueryStmt; -import java.util.Collections; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMultimap; - -/** - * Converts a resolved Zeta SQL query represented by a tree to corresponding Calcite representation. - */ -@SuppressWarnings({ - "rawtypes" // TODO(https://github.com/apache/beam/issues/20447) -}) -public class QueryStatementConverter extends RelConverter { - - /** Conversion rules, multimap from node kind to conversion rule. */ - private final ImmutableMultimap rules; - - public static RelNode convertRootQuery(ConversionContext context, ResolvedQueryStmt query) { - return new QueryStatementConverter(context).convert(query, Collections.emptyList()); - } - - private QueryStatementConverter(ConversionContext context) { - super(context); - this.rules = - ImmutableMultimap.builder() - .put(RESOLVED_AGGREGATE_SCAN, new AggregateScanConverter(context)) - .put(RESOLVED_ARRAY_SCAN, new ArrayScanToJoinConverter(context)) - .put(RESOLVED_ARRAY_SCAN, new ArrayScanLiteralToUncollectConverter(context)) - .put(RESOLVED_ARRAY_SCAN, new ArrayScanColumnRefToUncollect(context)) - .put(RESOLVED_FILTER_SCAN, new FilterScanConverter(context)) - .put(RESOLVED_JOIN_SCAN, new JoinScanConverter(context)) - .put(RESOLVED_LIMIT_OFFSET_SCAN, new LimitOffsetScanToLimitConverter(context)) - .put(RESOLVED_LIMIT_OFFSET_SCAN, new LimitOffsetScanToOrderByLimitConverter(context)) - .put(RESOLVED_ORDER_BY_SCAN, new OrderByScanUnsupportedConverter(context)) - .put(RESOLVED_PROJECT_SCAN, new ProjectScanConverter(context)) - .put(RESOLVED_SET_OPERATION_SCAN, new SetOperationScanConverter(context)) - .put(RESOLVED_SINGLE_ROW_SCAN, new SingleRowScanConverter(context)) - .put(RESOLVED_TABLE_SCAN, new TableScanConverter(context)) - .put(RESOLVED_WITH_REF_SCAN, new WithRefScanConverter(context)) - .put(RESOLVED_WITH_SCAN, new WithScanConverter(context)) - .put(RESOLVED_TVFSCAN, new TVFScanConverter(context)) - .build(); - } - - @Override - public RelNode convert(ResolvedQueryStmt zetaNode, List inputs) { - if (zetaNode.getIsValueTable()) { - throw new UnsupportedOperationException("Value Tables are not supported"); - } - - getTrait().addOutputColumnList(zetaNode.getOutputColumnList()); - - return convertNode(zetaNode.getQuery()); - } - - /** - * Convert node. - * - *

Finds a matching rule, uses the rule to extract inputs from the node, then converts the - * inputs (recursively), then converts the node using the converted inputs. - */ - private RelNode convertNode(ResolvedNode zetaNode) { - RelConverter nodeConverter = getConverterRule(zetaNode); - List inputs = nodeConverter.getInputs(zetaNode); - List convertedInputs = inputs.stream().map(this::convertNode).collect(toList()); - return nodeConverter.convert(zetaNode, convertedInputs); - } - - private RelConverter getConverterRule(ResolvedNode zetaNode) { - if (!rules.containsKey(zetaNode.nodeKind())) { - throw new UnsupportedOperationException( - String.format("Conversion of %s is not supported", zetaNode.nodeKind())); - } - - return rules.get(zetaNode.nodeKind()).stream() - .filter(relConverter -> relConverter.canConvert(zetaNode)) - .findFirst() - .orElseThrow( - () -> - new UnsupportedOperationException( - String.format("Cannot find a conversion rule for: %s", zetaNode))); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/RelConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/RelConverter.java deleted file mode 100644 index 3111a8c2e1f3..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/RelConverter.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNode; -import java.math.BigDecimal; -import java.util.Collections; -import java.util.List; -import org.apache.beam.sdk.extensions.sql.zetasql.QueryTrait; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalValues; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.FrameworkConfig; - -/** A rule that converts Zeta SQL resolved relational node to corresponding Calcite rel node. */ -abstract class RelConverter { - - /** - * Conversion context, contains things like FrameworkConfig, QueryTrait and other state used - * during conversion. - */ - protected ConversionContext context; - - RelConverter(ConversionContext context) { - this.context = context; - } - - /** Whether this rule can handle the conversion of the specific node. */ - public boolean canConvert(T zetaNode) { - return true; - } - - /** Extract Zeta SQL resolved nodes that correspond to the inputs of the current node. */ - public List getInputs(T zetaNode) { - return Collections.emptyList(); - } - - /** - * Converts given Zeta SQL node to corresponding Calcite node. - * - *

{@code inputs} are node inputs that have already been converter to Calcite versions. They - * correspond to the nodes in {@link #getInputs(ResolvedNode)}. - */ - public abstract RelNode convert(T zetaNode, List inputs); - - protected RelOptCluster getCluster() { - return context.cluster(); - } - - protected FrameworkConfig getConfig() { - return context.getConfig(); - } - - protected ExpressionConverter getExpressionConverter() { - return context.getExpressionConverter(); - } - - protected QueryTrait getTrait() { - return context.getTrait(); - } - - // This function creates a single dummy input row for queries that don't read from a table. - // For example: SELECT "hello" - // The code is copy-pasted from Calcite's LogicalValues.createOneRow() with a single line - // change: SqlTypeName.INTEGER replaced by SqlTypeName.BIGINT. - // Would like to call LogicalValues.createOneRow() directly, but it uses type SqlTypeName.INTEGER - // which corresponds to TypeKind.TYPE_INT32 in ZetaSQL, a type not supported in ZetaSQL - // PRODUCT_EXTERNAL mode. See - // https://github.com/google/zetasql/blob/c610a21ffdc110293c1c7bd255a2674ebc7ec7a8/java/com/google/zetasql/TypeFactory.java#L61 - static LogicalValues createOneRow(RelOptCluster cluster) { - final RelDataType rowType = - cluster.getTypeFactory().builder().add("ZERO", SqlTypeName.BIGINT).nullable(false).build(); - final ImmutableList> tuples = - ImmutableList.of( - ImmutableList.of( - cluster - .getRexBuilder() - .makeExactLiteral(BigDecimal.ZERO, rowType.getFieldList().get(0).getType()))); - return LogicalValues.create(cluster, rowType, tuples); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SetOperationScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SetOperationScanConverter.java deleted file mode 100644 index f8ac7ccdd299..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SetOperationScanConverter.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.translation; - -import static com.google.zetasql.resolvedast.ResolvedSetOperationScanEnums.SetOperationType.EXCEPT_ALL; -import static com.google.zetasql.resolvedast.ResolvedSetOperationScanEnums.SetOperationType.EXCEPT_DISTINCT; -import static com.google.zetasql.resolvedast.ResolvedSetOperationScanEnums.SetOperationType.INTERSECT_ALL; -import static com.google.zetasql.resolvedast.ResolvedSetOperationScanEnums.SetOperationType.INTERSECT_DISTINCT; -import static com.google.zetasql.resolvedast.ResolvedSetOperationScanEnums.SetOperationType.UNION_ALL; -import static com.google.zetasql.resolvedast.ResolvedSetOperationScanEnums.SetOperationType.UNION_DISTINCT; -import static java.util.stream.Collectors.toList; - -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedSetOperationItem; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedSetOperationScan; -import com.google.zetasql.resolvedast.ResolvedSetOperationScanEnums.SetOperationType; -import java.util.List; -import java.util.function.BiFunction; -import java.util.function.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalIntersect; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalMinus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalUnion; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -/** Converts set operations. */ -class SetOperationScanConverter extends RelConverter { - private enum Type { - DISTINCT, - ALL - } - - private static final ImmutableMap, RelNode>> - SET_OPERATION_FACTORIES = - ImmutableMap., RelNode>>builder() - .put(UNION_ALL, createFactoryFor(LogicalUnion::create, Type.ALL)) - .put(UNION_DISTINCT, createFactoryFor(LogicalUnion::create, Type.DISTINCT)) - .put(INTERSECT_ALL, createFactoryFor(LogicalIntersect::create, Type.ALL)) - .put(INTERSECT_DISTINCT, createFactoryFor(LogicalIntersect::create, Type.DISTINCT)) - .put(EXCEPT_ALL, createFactoryFor(LogicalMinus::create, Type.ALL)) - .put(EXCEPT_DISTINCT, createFactoryFor(LogicalMinus::create, Type.DISTINCT)) - .build(); - - /** - * A little closure to wrap the invocation of the factory method (e.g. LogicalUnion::create) for - * the set operation node. - */ - private static Function, RelNode> createFactoryFor( - BiFunction, Boolean, RelNode> setOperationFactory, Type type) { - return (List inputs) -> createRel(setOperationFactory, type == Type.ALL, inputs); - } - - SetOperationScanConverter(ConversionContext context) { - super(context); - } - - @Override - public List getInputs(ResolvedSetOperationScan zetaNode) { - return zetaNode.getInputItemList().stream() - .map(ResolvedSetOperationItem::getScan) - .collect(toList()); - } - - @Override - public RelNode convert(ResolvedSetOperationScan zetaNode, List inputs) { - if (!SET_OPERATION_FACTORIES.containsKey(zetaNode.getOpType())) { - throw new UnsupportedOperationException( - "Operation " + zetaNode.getOpType() + " is unsupported"); - } - - return SET_OPERATION_FACTORIES.get(zetaNode.getOpType()).apply(inputs); - } - - /** Beam set operations rel expects two inputs, so we are constructing a binary tree here. */ - private static RelNode createRel( - BiFunction, Boolean, RelNode> factory, boolean all, List inputs) { - return inputs.stream() - .skip(2) - .reduce( - // start with creating a set node for two first inputs - invokeFactory(factory, inputs.get(0), inputs.get(1), all), - // create another operation node with previous op node and the next input - (setOpNode, nextInput) -> invokeFactory(factory, setOpNode, nextInput, all)); - } - - /** - * Creates a set operation rel with two inputs. - * - *

Factory is, for example, LogicalUnion::create. - */ - private static RelNode invokeFactory( - BiFunction, Boolean, RelNode> factory, - RelNode input1, - RelNode input2, - boolean all) { - return factory.apply(ImmutableList.of(input1, input2), all); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java deleted file mode 100644 index e339b0ce6552..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedSingleRowScan; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; - -/** Converts a single row value. */ -class SingleRowScanConverter extends RelConverter { - - SingleRowScanConverter(ConversionContext context) { - super(context); - } - - @Override - public boolean canConvert(ResolvedSingleRowScan zetaNode) { - return zetaNode.getColumnList() == null || zetaNode.getColumnList().isEmpty(); - } - - @Override - public RelNode convert(ResolvedSingleRowScan zetaNode, List inputs) { - return createOneRow(getCluster()); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCaseWithValueOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCaseWithValueOperatorRewriter.java deleted file mode 100644 index 02ce2d57b9c4..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCaseWithValueOperatorRewriter.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.translation; - -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; - -/** - * Rewrites $case_with_value calls as $case_no_value calls. - * - *

Turns: - * - *

CASE x
- *   WHEN w1 THEN t1
- *   WHEN w2 THEN t2
- *   ELSE e
- *   END
- * - *

into: - * - *

CASE
- *   WHEN x == w1 THEN t1
- *   WHEN x == w2 THEN t2
- *   ELSE expr
- *   END
- * - *

Note that the ELSE statement is actually optional, but we don't need to worry about that here - * because the ZetaSQL analyzer populates the ELSE argument as a NULL literal if it's not specified. - */ -class SqlCaseWithValueOperatorRewriter implements SqlOperatorRewriter { - @Override - public RexNode apply(RexBuilder rexBuilder, List operands) { - Preconditions.checkArgument( - operands.size() % 2 == 0 && !operands.isEmpty(), - "$case_with_value should have an even number of arguments greater than 0 in function call" - + " (The value operand, the else operand, and paired when/then operands)."); - SqlOperator op = SqlStdOperatorTable.CASE; - - List newOperands = new ArrayList<>(); - RexNode value = operands.get(0); - - for (int i = 1; i < operands.size() - 2; i += 2) { - RexNode when = operands.get(i); - RexNode then = operands.get(i + 1); - newOperands.add( - rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, ImmutableList.of(value, when))); - newOperands.add(then); - } - - RexNode elseOperand = Iterables.getLast(operands); - newOperands.add(elseOperand); - - return rexBuilder.makeCall(op, newOperands); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCoalesceOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCoalesceOperatorRewriter.java deleted file mode 100644 index df1217fa15bd..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCoalesceOperatorRewriter.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.translation; - -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Util; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; - -/** - * Rewrites COALESCE calls as CASE ($case_no_value) calls. - * - *

Turns COALESCE(a, b, c) into: - * - *

CASE
- *   WHEN a IS NOT NULL THEN a
- *   WHEN b IS NOT NULL THEN b
- *   ELSE c
- *   END
- * - *

There is also a special case for the single-argument case: COALESCE(a) becomes - * just a. - */ -class SqlCoalesceOperatorRewriter implements SqlOperatorRewriter { - @Override - public RexNode apply(RexBuilder rexBuilder, List operands) { - Preconditions.checkArgument( - operands.size() >= 1, "COALESCE should have at least one argument in function call."); - - // No need for a case operator if there's only one operand - if (operands.size() == 1) { - return operands.get(0); - } - - SqlOperator op = SqlStdOperatorTable.CASE; - - List newOperands = new ArrayList<>(); - for (RexNode operand : Util.skipLast(operands)) { - newOperands.add( - rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, ImmutableList.of(operand))); - newOperands.add(operand); - } - newOperands.add(Util.last(operands)); - - return rexBuilder.makeCall(op, newOperands); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlIfNullOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlIfNullOperatorRewriter.java deleted file mode 100644 index ecab784a5e57..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlIfNullOperatorRewriter.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.translation; - -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; - -/** - * Rewrites IFNULL calls as CASE ($case_no_value) calls. - * - *

Turns IFNULL(expr, null_result) into:

CASE
- *   WHEN expr IS NULL THEN null_result
- *   ELSE expr
- *   END
- */ -class SqlIfNullOperatorRewriter implements SqlOperatorRewriter { - @Override - public RexNode apply(RexBuilder rexBuilder, List operands) { - Preconditions.checkArgument( - operands.size() == 2, "IFNULL should have two arguments in function call."); - - SqlOperator op = SqlStdOperatorTable.CASE; - List newOperands = - ImmutableList.of( - rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, ImmutableList.of(operands.get(0))), - operands.get(1), - operands.get(0)); - - return rexBuilder.makeCall(op, newOperands); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlInOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlInOperatorRewriter.java deleted file mode 100644 index a73d0abde14d..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlInOperatorRewriter.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.translation; - -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; - -/** Rewrites $in calls as SEARCH calls. */ -class SqlInOperatorRewriter implements SqlOperatorRewriter { - @Override - public RexNode apply(RexBuilder rexBuilder, List operands) { - Preconditions.checkArgument( - operands.size() >= 2, "IN should have at least two arguments in function call."); - final RexNode arg = operands.get(0); - final List ranges = ImmutableList.copyOf(operands.subList(1, operands.size())); - - // ZetaSQL has weird behavior for NULL... - for (RexNode node : ranges) { - if (node instanceof RexLiteral && ((RexLiteral) node).isNull()) { - throw new UnsupportedOperationException("IN NULL unsupported"); - } - } - - return rexBuilder.makeIn(arg, ranges); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java deleted file mode 100644 index 99dd23fff82d..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.translation; - -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; - -/** - * Rewrites NULLIF calls as CASE ($case_no_value) calls. - * - *

Turns NULLIF(expression, expression_to_match) into:

CASE
- *   WHEN expression == expression_to_match THEN NULL
- *   ELSE expression
- *   END
- */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class SqlNullIfOperatorRewriter implements SqlOperatorRewriter { - @Override - public RexNode apply(RexBuilder rexBuilder, List operands) { - Preconditions.checkArgument( - operands.size() == 2, "NULLIF should have two arguments in function call."); - - SqlOperator op = - SqlOperatorMappingTable.ZETASQL_FUNCTION_TO_CALCITE_SQL_OPERATOR - .get("$case_no_value") - .apply(null); - List newOperands = - ImmutableList.of( - rexBuilder.makeCall( - SqlStdOperatorTable.EQUALS, ImmutableList.of(operands.get(0), operands.get(1))), - rexBuilder.makeNullLiteral(operands.get(1).getType()), - operands.get(0)); - - return rexBuilder.makeCall(op, newOperands); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorMappingTable.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorMappingTable.java deleted file mode 100644 index c3b0c6376871..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorMappingTable.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNodes; -import java.util.Map; -import java.util.function.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** SqlOperatorMappingTable. */ -class SqlOperatorMappingTable { - - // todo: Some of operators defined here are later overridden in ZetaSQLPlannerImpl. - // We should remove them from this table and add generic way to provide custom - // implementation. (Ex.: timestamp_add) - static final Map> - ZETASQL_FUNCTION_TO_CALCITE_SQL_OPERATOR = - ImmutableMap - .>builder() - // grouped window function - .put("TUMBLE", resolvedFunction -> SqlStdOperatorTable.TUMBLE_OLD) - .put("HOP", resolvedFunction -> SqlStdOperatorTable.HOP_OLD) - .put("SESSION", resolvedFunction -> SqlStdOperatorTable.SESSION_OLD) - - // ZetaSQL functions - .put("$and", resolvedFunction -> SqlStdOperatorTable.AND) - .put("$or", resolvedFunction -> SqlStdOperatorTable.OR) - .put("$not", resolvedFunction -> SqlStdOperatorTable.NOT) - .put("$equal", resolvedFunction -> SqlStdOperatorTable.EQUALS) - .put("$not_equal", resolvedFunction -> SqlStdOperatorTable.NOT_EQUALS) - .put("$greater", resolvedFunction -> SqlStdOperatorTable.GREATER_THAN) - .put( - "$greater_or_equal", - resolvedFunction -> SqlStdOperatorTable.GREATER_THAN_OR_EQUAL) - .put("$less", resolvedFunction -> SqlStdOperatorTable.LESS_THAN) - .put("$less_or_equal", resolvedFunction -> SqlStdOperatorTable.LESS_THAN_OR_EQUAL) - .put("$like", resolvedFunction -> SqlOperators.LIKE) - .put("$is_null", resolvedFunction -> SqlStdOperatorTable.IS_NULL) - .put("$is_true", resolvedFunction -> SqlStdOperatorTable.IS_TRUE) - .put("$is_false", resolvedFunction -> SqlStdOperatorTable.IS_FALSE) - .put("$add", resolvedFunction -> SqlStdOperatorTable.PLUS) - .put("$subtract", resolvedFunction -> SqlStdOperatorTable.MINUS) - .put("$multiply", resolvedFunction -> SqlStdOperatorTable.MULTIPLY) - .put("$unary_minus", resolvedFunction -> SqlStdOperatorTable.UNARY_MINUS) - .put("$divide", resolvedFunction -> SqlStdOperatorTable.DIVIDE) - .put("concat", resolvedFunction -> SqlOperators.CONCAT) - .put("substr", resolvedFunction -> SqlOperators.SUBSTR) - .put("substring", resolvedFunction -> SqlOperators.SUBSTR) - .put("trim", resolvedFunction -> SqlOperators.TRIM) - .put("replace", resolvedFunction -> SqlOperators.REPLACE) - .put("char_length", resolvedFunction -> SqlOperators.CHAR_LENGTH) - .put("starts_with", resolvedFunction -> SqlOperators.START_WITHS) - .put("ends_with", resolvedFunction -> SqlOperators.ENDS_WITH) - .put("ltrim", resolvedFunction -> SqlOperators.LTRIM) - .put("rtrim", resolvedFunction -> SqlOperators.RTRIM) - .put("reverse", resolvedFunction -> SqlOperators.REVERSE) - .put("$count_star", resolvedFunction -> SqlStdOperatorTable.COUNT) - .put("max", resolvedFunction -> SqlStdOperatorTable.MAX) - .put("min", resolvedFunction -> SqlStdOperatorTable.MIN) - .put("avg", resolvedFunction -> SqlStdOperatorTable.AVG) - .put("sum", resolvedFunction -> SqlStdOperatorTable.SUM) - .put("any_value", resolvedFunction -> SqlStdOperatorTable.ANY_VALUE) - .put("count", resolvedFunction -> SqlStdOperatorTable.COUNT) - .put("bit_and", resolvedFunction -> SqlStdOperatorTable.BIT_AND) - .put("string_agg", SqlOperators::createStringAggOperator) // NULL values not supported - .put("array_agg", resolvedFunction -> SqlOperators.ARRAY_AGG_FN) - .put("bit_or", resolvedFunction -> SqlStdOperatorTable.BIT_OR) - .put("bit_xor", resolvedFunction -> SqlOperators.BIT_XOR) - .put("ceil", resolvedFunction -> SqlStdOperatorTable.CEIL) - .put("floor", resolvedFunction -> SqlStdOperatorTable.FLOOR) - .put("mod", resolvedFunction -> SqlStdOperatorTable.MOD) - .put("timestamp", resolvedFunction -> SqlOperators.TIMESTAMP_OP) - .put("$case_no_value", resolvedFunction -> SqlStdOperatorTable.CASE) - - // if operator - IF(cond, pos, neg) can actually be mapped directly to `CASE WHEN cond - // THEN pos ELSE neg` - .put("if", resolvedFunction -> SqlStdOperatorTable.CASE) - - // $case_no_value specializations - // all of these operators can have their operands adjusted to achieve the same thing - // with - // a call to $case_with_value - .put("$case_with_value", resolvedFunction -> SqlStdOperatorTable.CASE) - .put("coalesce", resolvedFunction -> SqlStdOperatorTable.CASE) - .put("ifnull", resolvedFunction -> SqlStdOperatorTable.CASE) - .put("nullif", resolvedFunction -> SqlStdOperatorTable.CASE) - .put("countif", resolvedFunction -> SqlOperators.COUNTIF) - .build(); - - static final Map ZETASQL_FUNCTION_TO_CALCITE_SQL_OPERATOR_REWRITER = - ImmutableMap.builder() - .put("$case_with_value", new SqlCaseWithValueOperatorRewriter()) - .put("coalesce", new SqlCoalesceOperatorRewriter()) - .put("ifnull", new SqlIfNullOperatorRewriter()) - .put("nullif", new SqlNullIfOperatorRewriter()) - .put("$in", new SqlInOperatorRewriter()) - .build(); - - static @Nullable SqlOperator create( - ResolvedNodes.ResolvedFunctionCallBase aggregateFunctionCall) { - - Function sqlOperatorFactory = - ZETASQL_FUNCTION_TO_CALCITE_SQL_OPERATOR.get(aggregateFunctionCall.getFunction().getName()); - - if (sqlOperatorFactory != null) { - return sqlOperatorFactory.apply(aggregateFunctionCall); - } - return null; - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorRewriter.java deleted file mode 100644 index e283a7238727..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorRewriter.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.translation; - -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexBuilder; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; - -/** Interface for rewriting calls a specific ZetaSQL operator. */ -interface SqlOperatorRewriter { - /** - * Create and return a new {@link RexNode} that represents a call to this operator with the - * specified operands. - * - * @param rexBuilder A {@link RexBuilder} instance to use for creating new {@link RexNode}s - * @param operands The original list of {@link RexNode} operands passed to this operator call - * @return The created RexNode - */ - RexNode apply(RexBuilder rexBuilder, List operands); -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java deleted file mode 100644 index 0f6dcea2f692..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java +++ /dev/null @@ -1,352 +0,0 @@ -/* - * 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.translation; - -import static org.apache.beam.sdk.extensions.sql.zetasql.BeamZetaSqlCatalog.ZETASQL_FUNCTION_GROUP_NAME; - -import com.google.zetasql.Value; -import com.google.zetasql.io.grpc.Status; -import com.google.zetasql.io.grpc.StatusRuntimeException; -import com.google.zetasql.resolvedast.ResolvedNodes; -import java.lang.reflect.Method; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.extensions.sql.impl.ScalarFunctionImpl; -import org.apache.beam.sdk.extensions.sql.impl.UdafImpl; -import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelDataTypeSystem; -import org.apache.beam.sdk.extensions.sql.impl.transform.BeamBuiltinAggregations; -import org.apache.beam.sdk.extensions.sql.impl.transform.agg.CountIf; -import org.apache.beam.sdk.extensions.sql.impl.udaf.ArrayAgg; -import org.apache.beam.sdk.extensions.sql.impl.udaf.StringAgg; -import org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils; -import org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlException; -import org.apache.beam.sdk.extensions.sql.zetasql.translation.impl.BeamBuiltinMethods; -import org.apache.beam.sdk.extensions.sql.zetasql.translation.impl.CastFunctionImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactoryImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.AggregateFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.FunctionParameter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.ScalarFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlSyntax; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.ArraySqlType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.FamilyOperandTypeChecker; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.InferTypes; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.OperandTypes; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeFactoryImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeFamily; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Optionality; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.util.Util; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; - -/** - * A separate SqlOperators table for those functions that do not exist or not compatible with - * Calcite. Most of functions within this class is copied from Calcite. - */ -@Internal -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class SqlOperators { - public static final SqlOperator ZETASQL_TIMESTAMP_ADD = - createZetaSqlFunction("timestamp_add", SqlTypeName.TIMESTAMP); - - private static final RelDataType OTHER = createSqlType(SqlTypeName.OTHER, false); - private static final RelDataType TIMESTAMP = createSqlType(SqlTypeName.TIMESTAMP, false); - private static final RelDataType NULLABLE_TIMESTAMP = createSqlType(SqlTypeName.TIMESTAMP, true); - private static final RelDataType BIGINT = createSqlType(SqlTypeName.BIGINT, false); - private static final RelDataType NULLABLE_BIGINT = createSqlType(SqlTypeName.BIGINT, true); - - public static final SqlOperator ARRAY_AGG_FN = - createUdafOperator( - "array_agg", - x -> new ArraySqlType(x.getOperandType(0), true), - new UdafImpl<>(new ArrayAgg.ArrayAggArray<>())); - - public static final SqlOperator START_WITHS = - createUdfOperator( - "STARTS_WITH", BeamBuiltinMethods.STARTS_WITH_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator CONCAT = - createUdfOperator("CONCAT", BeamBuiltinMethods.CONCAT_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator REPLACE = - createUdfOperator("REPLACE", BeamBuiltinMethods.REPLACE_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator TRIM = - createUdfOperator("TRIM", BeamBuiltinMethods.TRIM_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator LTRIM = - createUdfOperator("LTRIM", BeamBuiltinMethods.LTRIM_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator RTRIM = - createUdfOperator("RTRIM", BeamBuiltinMethods.RTRIM_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator SUBSTR = - createUdfOperator("SUBSTR", BeamBuiltinMethods.SUBSTR_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator REVERSE = - createUdfOperator("REVERSE", BeamBuiltinMethods.REVERSE_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator CHAR_LENGTH = - createUdfOperator( - "CHAR_LENGTH", BeamBuiltinMethods.CHAR_LENGTH_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator ENDS_WITH = - createUdfOperator( - "ENDS_WITH", BeamBuiltinMethods.ENDS_WITH_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator LIKE = - createUdfOperator( - "LIKE", - BeamBuiltinMethods.LIKE_METHOD, - SqlSyntax.BINARY, - ZETASQL_FUNCTION_GROUP_NAME, - ""); - - public static final SqlOperator VALIDATE_TIMESTAMP = - createUdfOperator( - "validateTimestamp", - DateTimeUtils.class, - "validateTimestamp", - x -> NULLABLE_TIMESTAMP, - ImmutableList.of(TIMESTAMP), - ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator VALIDATE_TIME_INTERVAL = - createUdfOperator( - "validateIntervalArgument", - DateTimeUtils.class, - "validateTimeInterval", - x -> NULLABLE_BIGINT, - ImmutableList.of(BIGINT, OTHER), - ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator TIMESTAMP_OP = - createUdfOperator( - "TIMESTAMP", BeamBuiltinMethods.TIMESTAMP_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator DATE_OP = - createUdfOperator("DATE", BeamBuiltinMethods.DATE_METHOD, ZETASQL_FUNCTION_GROUP_NAME); - - public static final SqlOperator BIT_XOR = - createUdafOperator( - "BIT_XOR", - x -> NULLABLE_BIGINT, - new UdafImpl<>(new BeamBuiltinAggregations.BitXOr())); - - public static final SqlOperator COUNTIF = - createUdafOperator( - "countif", - x -> createTypeFactory().createSqlType(SqlTypeName.BIGINT), - new UdafImpl<>(new CountIf.CountIfFn())); - - public static final SqlUserDefinedFunction CAST_OP = - new SqlUserDefinedFunction( - new SqlIdentifier("CAST", SqlParserPos.ZERO), - SqlKind.OTHER_FUNCTION, - null, - null, - null, - new CastFunctionImpl()); - - public static SqlOperator createStringAggOperator( - ResolvedNodes.ResolvedFunctionCallBase aggregateFunctionCall) { - List args = aggregateFunctionCall.getArgumentList(); - String inputType = args.get(0).getType().typeName(); - Value delimiter = null; - if (args.size() == 2) { - ResolvedNodes.ResolvedExpr resolvedExpr = args.get(1); - if (resolvedExpr instanceof ResolvedNodes.ResolvedLiteral) { - delimiter = ((ResolvedNodes.ResolvedLiteral) resolvedExpr).getValue(); - } else { - // TODO(https://github.com/apache/beam/issues/21283) Add support for params - throw new ZetaSqlException( - new StatusRuntimeException( - Status.INVALID_ARGUMENT.withDescription( - String.format( - "STRING_AGG only supports ResolvedLiteral as delimiter, provided %s", - resolvedExpr.getClass().getName())))); - } - } - - switch (inputType) { - case "BYTES": - return SqlOperators.createUdafOperator( - "string_agg", - x -> SqlOperators.createTypeFactory().createSqlType(SqlTypeName.VARBINARY), - new UdafImpl<>( - new StringAgg.StringAggByte( - delimiter == null - ? ",".getBytes(StandardCharsets.UTF_8) - : delimiter.getBytesValue().toByteArray()))); - case "STRING": - return SqlOperators.createUdafOperator( - "string_agg", - x -> SqlOperators.createTypeFactory().createSqlType(SqlTypeName.VARCHAR), - new UdafImpl<>( - new StringAgg.StringAggString( - delimiter == null ? "," : delimiter.getStringValue()))); - default: - throw new UnsupportedOperationException( - String.format("[%s] is not supported in STRING_AGG", inputType)); - } - } - - /** - * Create a dummy SqlFunction of type OTHER_FUNCTION from given function name and return type. - * These functions will be unparsed in either {@link - * org.apache.beam.sdk.extensions.sql.zetasql.BeamZetaSqlCalcRel} (for built-in functions) or - * {@link org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel} (for user-defined functions). - */ - public static SqlFunction createZetaSqlFunction(String name, SqlTypeName returnType) { - return new SqlFunction( - name, - SqlKind.OTHER_FUNCTION, - x -> createSqlType(returnType, true), - null, // operandTypeInference - null, // operandTypeChecker - SqlFunctionCategory.USER_DEFINED_FUNCTION); - } - - static SqlUserDefinedAggFunction createUdafOperator( - String name, SqlReturnTypeInference returnTypeInference, AggregateFunction function) { - return new SqlUserDefinedAggFunction( - new SqlIdentifier(name, SqlParserPos.ZERO), - returnTypeInference, - null, - null, - function, - false, - false, - Optionality.FORBIDDEN, - createTypeFactory()); - } - - private static SqlUserDefinedFunction createUdfOperator( - String name, - Class methodClass, - String methodName, - SqlReturnTypeInference returnTypeInference, - List paramTypes, - String funGroup) { - return new SqlUserDefinedFunction( - new SqlIdentifier(name, SqlParserPos.ZERO), - returnTypeInference, - null, - null, - paramTypes, - ZetaSqlScalarFunctionImpl.create(methodClass, methodName, funGroup, "")); - } - - static SqlUserDefinedFunction createUdfOperator( - String name, Method method, String funGroup, String jarPath) { - return createUdfOperator(name, method, SqlSyntax.FUNCTION, funGroup, jarPath); - } - - static SqlUserDefinedFunction createUdfOperator(String name, Method method, String funGroup) { - return createUdfOperator(name, method, SqlSyntax.FUNCTION, funGroup, ""); - } - - private static SqlUserDefinedFunction createUdfOperator( - String name, Method method, final SqlSyntax syntax, String funGroup, String jarPath) { - Function function = ZetaSqlScalarFunctionImpl.create(method, funGroup, jarPath); - final RelDataTypeFactory typeFactory = createTypeFactory(); - - List argTypes = new ArrayList<>(); - List typeFamilies = new ArrayList<>(); - for (FunctionParameter o : function.getParameters()) { - final RelDataType type = o.getType(typeFactory); - argTypes.add(type); - typeFamilies.add(Util.first(type.getSqlTypeName().getFamily(), SqlTypeFamily.ANY)); - } - - final FamilyOperandTypeChecker typeChecker = - OperandTypes.family(typeFamilies, i -> function.getParameters().get(i).isOptional()); - - final List paramTypes = toSql(typeFactory, argTypes); - - return new SqlUserDefinedFunction( - new SqlIdentifier(name, SqlParserPos.ZERO), - infer((ScalarFunction) function), - InferTypes.explicit(argTypes), - typeChecker, - paramTypes, - function) { - @Override - public SqlSyntax getSyntax() { - return syntax; - } - }; - } - - private static RelDataType createSqlType(SqlTypeName typeName, boolean withNullability) { - final RelDataTypeFactory typeFactory = createTypeFactory(); - RelDataType type = typeFactory.createSqlType(typeName); - if (withNullability) { - type = typeFactory.createTypeWithNullability(type, true); - } - return type; - } - - private static RelDataTypeFactory createTypeFactory() { - return new SqlTypeFactoryImpl(BeamRelDataTypeSystem.INSTANCE); - } - - private static SqlReturnTypeInference infer(final ScalarFunction function) { - return opBinding -> { - final RelDataTypeFactory typeFactory = opBinding.getTypeFactory(); - final RelDataType type; - if (function instanceof ScalarFunctionImpl) { - type = ((ScalarFunctionImpl) function).getReturnType(typeFactory, opBinding); - } else { - type = function.getReturnType(typeFactory); - } - return toSql(typeFactory, type); - }; - } - - private static List toSql( - final RelDataTypeFactory typeFactory, List types) { - return Lists.transform(types, type -> toSql(typeFactory, type)); - } - - private static RelDataType toSql(RelDataTypeFactory typeFactory, RelDataType type) { - if (type instanceof RelDataTypeFactoryImpl.JavaType - && ((RelDataTypeFactoryImpl.JavaType) type).getJavaClass() == Object.class) { - return typeFactory.createTypeWithNullability( - typeFactory.createSqlType(SqlTypeName.ANY), true); - } - return JavaTypeFactoryImpl.toSql(typeFactory, type); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java deleted file mode 100644 index cc40c43b91d7..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * 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.translation; - -import java.util.ArrayList; -import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.utils.TVFStreamingUtils; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFieldImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelRecordType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlCallBinding; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlOperandCountRange; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlOperandCountRanges; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlValidator; - -/** Base class for table-valued function windowing operator (TUMBLE, HOP and SESSION). */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class SqlWindowTableFunction extends SqlFunction { - public SqlWindowTableFunction(String name) { - super( - name, - SqlKind.OTHER_FUNCTION, - ARG0_TABLE_FUNCTION_WINDOWING, - null, - null, - SqlFunctionCategory.SYSTEM); - } - - @Override - public SqlOperandCountRange getOperandCountRange() { - return SqlOperandCountRanges.of(3); - } - - @Override - public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) { - // There should only be three operands, and number of operands are checked before - // this call. - final SqlNode operand0 = callBinding.operand(0); - final SqlValidator validator = callBinding.getValidator(); - final RelDataType type = validator.getValidatedNodeType(operand0); - if (type.getSqlTypeName() != SqlTypeName.ROW) { - return throwValidationSignatureErrorOrReturnFalse(callBinding, throwOnFailure); - } - return true; - } - - private boolean throwValidationSignatureErrorOrReturnFalse( - SqlCallBinding callBinding, boolean throwOnFailure) { - if (throwOnFailure) { - throw callBinding.newValidationSignatureError(); - } else { - return false; - } - } - - @Override - public String getAllowedSignatures(String opNameToUse) { - return getName() + "(TABLE table_name, DESCRIPTOR(col1, col2 ...), datetime interval)"; - } - - /** - * The first parameter of table-value function windowing is a TABLE parameter, which is not - * scalar. So need to override SqlOperator.argumentMustBeScalar. - */ - @Override - public boolean argumentMustBeScalar(int ordinal) { - return ordinal != 0; - } - - /** - * Type-inference strategy whereby the result type of a table function call is a ROW, which is - * combined from the operand #0(TABLE parameter)'s schema and two additional fields: - * - *
    - *
  1. window_start. TIMESTAMP type to indicate a window's start. - *
  2. window_end. TIMESTAMP type to indicate a window's end. - *
- */ - public static final SqlReturnTypeInference ARG0_TABLE_FUNCTION_WINDOWING = - opBinding -> { - RelDataType inputRowType = opBinding.getOperandType(0); - List newFields = new ArrayList<>(inputRowType.getFieldList()); - RelDataType timestampType = opBinding.getTypeFactory().createSqlType(SqlTypeName.TIMESTAMP); - - RelDataTypeField windowStartField = - new RelDataTypeFieldImpl( - TVFStreamingUtils.WINDOW_START, newFields.size(), timestampType); - newFields.add(windowStartField); - RelDataTypeField windowEndField = - new RelDataTypeFieldImpl(TVFStreamingUtils.WINDOW_END, newFields.size(), timestampType); - newFields.add(windowEndField); - - return new RelRecordType(inputRowType.getStructKind(), newFields); - }; -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java deleted file mode 100644 index 298e7ab35a88..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.FileDescriptorSetsBuilder; -import com.google.zetasql.FunctionProtos.TableValuedFunctionProto; -import com.google.zetasql.TableValuedFunction.FixedOutputSchemaTVF; -import com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind; -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedFunctionArgument; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedLiteral; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedTVFScan; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalTableFunctionScan; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; - -/** Converts TVFScan. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class TVFScanConverter extends RelConverter { - - TVFScanConverter(ConversionContext context) { - super(context); - } - - @Override - public RelNode convert(ResolvedTVFScan zetaNode, List inputs) { - RelNode input = inputs.get(0); - RexCall call = - getExpressionConverter() - .convertTableValuedFunction( - input, - zetaNode.getTvf(), - zetaNode.getArgumentList(), - zetaNode.getArgumentList().size() > 0 - && zetaNode.getArgumentList().get(0).getScan() != null - ? zetaNode.getArgumentList().get(0).getScan().getColumnList() - : Collections.emptyList()); - RelNode tableFunctionScan = - LogicalTableFunctionScan.create( - getCluster(), inputs, call, null, call.getType(), Collections.EMPTY_SET); - - // Pure SQL UDF's language body is built bottom up, so FunctionArgumentRefMapping should be - // already consumed thus it can be cleared now. - context.clearFunctionArgumentRefMapping(); - return tableFunctionScan; - } - - @Override - public List getInputs(ResolvedTVFScan zetaNode) { - List inputs = new ArrayList<>(); - if (zetaNode.getTvf() != null - && context - .getUserDefinedTableValuedFunctions() - .containsKey(zetaNode.getTvf().getNamePath())) { - inputs.add(context.getUserDefinedTableValuedFunctions().get(zetaNode.getTvf().getNamePath())); - } - - for (ResolvedFunctionArgument argument : zetaNode.getArgumentList()) { - if (argument.getScan() != null) { - inputs.add(argument.getScan()); - } - } - - // Extract ResolvedArguments for solving ResolvedArgumentRef in later conversion. - if (zetaNode.getTvf() instanceof FixedOutputSchemaTVF) { - FileDescriptorSetsBuilder temp = new FileDescriptorSetsBuilder(); - // TODO: migrate to public Java API to retrieve FunctionSignature. - TableValuedFunctionProto tableValuedFunctionProto = zetaNode.getTvf().serialize(temp); - for (int i = 0; i < tableValuedFunctionProto.getSignature().getArgumentList().size(); i++) { - String argumentName = - tableValuedFunctionProto.getSignature().getArgument(i).getOptions().getArgumentName(); - if (zetaNode.getArgumentList().get(i).nodeKind() - == ResolvedNodeKind.RESOLVED_FUNCTION_ARGUMENT) { - ResolvedFunctionArgument resolvedTVFArgument = zetaNode.getArgumentList().get(i); - if (resolvedTVFArgument.getExpr().nodeKind() == ResolvedNodeKind.RESOLVED_LITERAL) { - ResolvedLiteral literal = (ResolvedLiteral) resolvedTVFArgument.getExpr(); - context.addToFunctionArgumentRefMapping( - argumentName, getExpressionConverter().convertResolvedLiteral(literal)); - } - } - } - } - return inputs; - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TableScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TableScanConverter.java deleted file mode 100644 index 8b7af956ec9a..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TableScanConverter.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.translation; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; - -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedTableScan; -import java.util.List; -import java.util.Properties; -import org.apache.beam.sdk.extensions.sql.zetasql.TableResolution; -import org.apache.beam.vendor.calcite.v1_28_0.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.config.CalciteConnectionConfigImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.jdbc.CalciteSchema; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.prepare.CalciteCatalogReader; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.prepare.RelOptTableImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelRoot; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.hint.RelHint; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Table; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.TranslatableTable; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** Converts table scan. */ -class TableScanConverter extends RelConverter { - - TableScanConverter(ConversionContext context) { - super(context); - } - - @Override - public RelNode convert(ResolvedTableScan zetaNode, List inputs) { - - List tablePath = getTablePath(zetaNode.getTable()); - - SchemaPlus defaultSchemaPlus = getConfig().getDefaultSchema(); - if (defaultSchemaPlus == null) { - throw new AssertionError("Default schema is null."); - } - // TODO: reject incorrect top-level schema - - Table calciteTable = TableResolution.resolveCalciteTable(defaultSchemaPlus, tablePath); - - // we already resolved the table before passing the query to Analyzer, so it should be there - checkNotNull( - calciteTable, - "Unable to resolve the table path %s in schema %s", - tablePath, - defaultSchemaPlus.getName()); - - String defaultSchemaName = defaultSchemaPlus.getName(); - - final CalciteCatalogReader catalogReader = - new CalciteCatalogReader( - CalciteSchema.from(defaultSchemaPlus), - ImmutableList.of(defaultSchemaName), - getCluster().getTypeFactory(), - new CalciteConnectionConfigImpl(new Properties())); - - RelOptTableImpl relOptTable = - RelOptTableImpl.create( - catalogReader, - calciteTable.getRowType(getCluster().getTypeFactory()), - calciteTable, - ImmutableList.builder().add(defaultSchemaName).addAll(tablePath).build()); - - if (calciteTable instanceof TranslatableTable) { - return ((TranslatableTable) calciteTable).toRel(createToRelContext(), relOptTable); - } else { - throw new UnsupportedOperationException("Does not support non TranslatableTable type table!"); - } - } - - private List getTablePath(com.google.zetasql.Table table) { - if (!getTrait().isTableResolved(table)) { - throw new IllegalArgumentException( - "Unexpected table found when converting to Calcite rel node: " + table); - } - - return getTrait().getTablePath(table); - } - - private RelOptTable.ToRelContext createToRelContext() { - return new RelOptTable.ToRelContext() { - @Override - public RelRoot expandView( - RelDataType relDataType, String s, List list, @Nullable List list1) { - throw new UnsupportedOperationException("This RelContext does not support expandView"); - } - - @Override - public RelOptCluster getCluster() { - return TableScanConverter.this.getCluster(); - } - - @Override - public List getTableHints() { - return ImmutableList.of(); - } - }; - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/UserFunctionDefinitions.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/UserFunctionDefinitions.java deleted file mode 100644 index b1891337a550..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/UserFunctionDefinitions.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.translation; - -import com.google.auto.value.AutoValue; -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes; -import java.lang.reflect.Method; -import java.util.List; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; - -/** Holds user defined function definitions. */ -@AutoValue -public abstract class UserFunctionDefinitions { - public abstract ImmutableMap, ResolvedNodes.ResolvedCreateFunctionStmt> - sqlScalarFunctions(); - - /** - * SQL native user-defined table-valued function can be resolved by Analyzer. Keeping the function - * name to its ResolvedNode mapping so during Plan conversion, UDTVF implementation can replace - * inputs of TVFScanConverter. - */ - public abstract ImmutableMap, ResolvedNode> sqlTableValuedFunctions(); - - public abstract ImmutableMap, JavaScalarFunction> javaScalarFunctions(); - - public abstract ImmutableMap, Combine.CombineFn> javaAggregateFunctions(); - - @AutoValue - public abstract static class JavaScalarFunction { - public static JavaScalarFunction create(Method method, String jarPath) { - return new AutoValue_UserFunctionDefinitions_JavaScalarFunction(method, jarPath); - } - - public abstract Method method(); - - /** The Beam filesystem path to the jar where the method was defined. */ - public abstract String jarPath(); - } - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setSqlScalarFunctions( - ImmutableMap, ResolvedNodes.ResolvedCreateFunctionStmt> sqlScalarFunctions); - - public abstract Builder setSqlTableValuedFunctions( - ImmutableMap, ResolvedNode> sqlTableValuedFunctions); - - public abstract Builder setJavaScalarFunctions( - ImmutableMap, JavaScalarFunction> javaScalarFunctions); - - public abstract Builder setJavaAggregateFunctions( - ImmutableMap, Combine.CombineFn> javaAggregateFunctions); - - public abstract UserFunctionDefinitions build(); - } - - public static Builder newBuilder() { - return new AutoValue_UserFunctionDefinitions.Builder() - .setSqlScalarFunctions(ImmutableMap.of()) - .setSqlTableValuedFunctions(ImmutableMap.of()) - .setJavaScalarFunctions(ImmutableMap.of()) - .setJavaAggregateFunctions(ImmutableMap.of()); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java deleted file mode 100644 index f33b10f70586..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedWithRefScan; -import java.util.Collections; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; - -/** Converts a call-site reference to a named WITH subquery. */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class WithRefScanConverter extends RelConverter { - - WithRefScanConverter(ConversionContext context) { - super(context); - } - - @Override - public List getInputs(ResolvedWithRefScan zetaNode) { - // WithRefScan contains only a name of a WITH query, - // but to actually convert it to the node we need to get the resolved node representation - // of the query. Here we take it from the trait, where it was persisted previously - // in WithScanConverter that actually parses the WITH query part. - // - // This query node returned from here will be converted by some other converter, - // (e.g. if the WITH query root is a projection it will go through ProjectScanConverter) - // and will reach the convert() method below as an already converted rel node. - return Collections.singletonList( - getTrait().withEntries.get(zetaNode.getWithQueryName()).getWithSubquery()); - } - - @Override - public RelNode convert(ResolvedWithRefScan zetaNode, List inputs) { - // Here the actual WITH query body has already been converted by, e.g. a ProjectScnaConverter, - // so to resolve the reference we just return that converter rel node. - return inputs.get(0); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithScanConverter.java deleted file mode 100644 index 0e80ddbdb260..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithScanConverter.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.translation; - -import com.google.zetasql.resolvedast.ResolvedNode; -import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedWithScan; -import java.util.Collections; -import java.util.List; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; - -/** Converts a named WITH. */ -class WithScanConverter extends RelConverter { - - WithScanConverter(ConversionContext context) { - super(context); - } - - @Override - public List getInputs(ResolvedWithScan zetaNode) { - // We must persist the named WITH queries nodes, - // so that when they are referenced by name (e.g. in FROM/JOIN), we can - // resolve them. We need this because the nodes that represent the references (WithRefScan) - // only contain the names of the queries, so we need to keep this map for resolution of the - // names. - zetaNode - .getWithEntryList() - .forEach(withEntry -> getTrait().withEntries.put(withEntry.getWithQueryName(), withEntry)); - - // Returning the body of the query, it is something like ProjectScan that will be converted - // by ProjectScanConverter before it reaches the convert() method below. - return Collections.singletonList(zetaNode.getQuery()); - } - - @Override - public RelNode convert(ResolvedWithScan zetaNode, List inputs) { - // The body of the WITH query is already converted at this point so we just - // return it, nothing else is needed. - return inputs.get(0); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ZetaSqlScalarFunctionImpl.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ZetaSqlScalarFunctionImpl.java deleted file mode 100644 index 1ed6939f9549..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ZetaSqlScalarFunctionImpl.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.translation; - -import java.lang.reflect.Method; -import org.apache.beam.sdk.extensions.sql.impl.ScalarFunctionImpl; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.CallImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.ScalarFunction; - -/** ZetaSQL-specific extension to {@link ScalarFunctionImpl}. */ -public class ZetaSqlScalarFunctionImpl extends ScalarFunctionImpl { - /** - * ZetaSQL function group identifier. Different function groups may have divergent translation - * paths. - */ - public final String functionGroup; - - private ZetaSqlScalarFunctionImpl( - Method method, CallImplementor implementor, String functionGroup, String jarPath) { - super(method, implementor, jarPath); - this.functionGroup = functionGroup; - } - - /** - * Creates {@link org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function} from - * given class. - * - *

If a method of the given name is not found or it does not suit, returns {@code null}. - * - * @param clazz class that is used to implement the function - * @param methodName Method name (typically "eval") - * @param functionGroup ZetaSQL function group identifier. Different function groups may have - * divergent translation paths. - * @return created {@link ScalarFunction} or null - */ - public static Function create( - Class clazz, String methodName, String functionGroup, String jarPath) { - return create(findMethod(clazz, methodName), functionGroup, jarPath); - } - - /** - * Creates {@link org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.Function} from - * given method. When {@code eval} method does not suit, {@code null} is returned. - * - * @param method method that is used to implement the function - * @param functionGroup ZetaSQL function group identifier. Different function groups may have - * divergent translation paths. - * @return created {@link Function} or null - */ - public static Function create(Method method, String functionGroup, String jarPath) { - validateMethod(method); - CallImplementor implementor = createImplementor(method); - return new ZetaSqlScalarFunctionImpl(method, implementor, functionGroup, jarPath); - } - - /* - * Finds a method in a given class by name. - * @param clazz class to search method in - * @param name name of the method to find - * @return the first method with matching name or null when no method found - */ - private static Method findMethod(Class clazz, String name) { - for (Method method : clazz.getMethods()) { - if (method.getName().equals(name) && !method.isBridge()) { - return method; - } - } - throw new NoSuchMethodError( - String.format("Method %s not found in class %s.", name, clazz.getName())); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamBuiltinMethods.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamBuiltinMethods.java deleted file mode 100644 index 79adab4a01e8..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamBuiltinMethods.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.translation.impl; - -import java.lang.reflect.Method; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Types; - -/** BeamBuiltinMethods. */ -@Internal -public class BeamBuiltinMethods { - public static final Method STARTS_WITH_METHOD = - Types.lookupMethod(StringFunctions.class, "startsWith", String.class, String.class); - - public static final Method ENDS_WITH_METHOD = - Types.lookupMethod(StringFunctions.class, "endsWith", String.class, String.class); - - public static final Method LIKE_METHOD = - Types.lookupMethod(StringFunctions.class, "like", String.class, String.class); - - public static final Method CONCAT_METHOD = - Types.lookupMethod( - StringFunctions.class, - "concat", - String.class, - String.class, - String.class, - String.class, - String.class); - - public static final Method REPLACE_METHOD = - Types.lookupMethod( - StringFunctions.class, "replace", String.class, String.class, String.class); - - public static final Method TRIM_METHOD = - Types.lookupMethod(StringFunctions.class, "trim", String.class, String.class); - - public static final Method LTRIM_METHOD = - Types.lookupMethod(StringFunctions.class, "ltrim", String.class, String.class); - - public static final Method RTRIM_METHOD = - Types.lookupMethod(StringFunctions.class, "rtrim", String.class, String.class); - - public static final Method SUBSTR_METHOD = - Types.lookupMethod(StringFunctions.class, "substr", String.class, long.class, long.class); - - public static final Method REVERSE_METHOD = - Types.lookupMethod(StringFunctions.class, "reverse", String.class); - - public static final Method CHAR_LENGTH_METHOD = - Types.lookupMethod(StringFunctions.class, "charLength", String.class); - - public static final Method TIMESTAMP_METHOD = - Types.lookupMethod(TimestampFunctions.class, "timestamp", String.class, String.class); - - public static final Method DATE_METHOD = - Types.lookupMethod(DateFunctions.class, "date", Integer.class, Integer.class, Integer.class); -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamCodegenUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamCodegenUtils.java deleted file mode 100644 index 029e3caa0904..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamCodegenUtils.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.translation.impl; - -import java.io.UnsupportedEncodingException; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils; -import org.joda.time.DateTime; - -/** BeamCodegenUtils. */ -@Internal -public class BeamCodegenUtils { - // convert bytes to String in UTF8 encoding. - public static String toStringUTF8(byte[] bytes) { - try { - return new String(bytes, "UTF8"); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException(e); - } - } - - public static String toStringTimestamp(long timestamp) { - return DateTimeUtils.formatTimestampWithTimeZone(new DateTime(timestamp)); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/CastFunctionImpl.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/CastFunctionImpl.java deleted file mode 100644 index 1bcd73479bce..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/CastFunctionImpl.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.translation.impl; - -import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.RexImpTable.createImplementor; - -import java.util.Collections; -import java.util.List; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.CallImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.NotNullImplementor; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.NullPolicy; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.RexImpTable; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expression; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.tree.Expressions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.FunctionParameter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.ImplementableFunction; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; - -/** ZetaSQLCastFunctionImpl. */ -@Internal -public class CastFunctionImpl implements ImplementableFunction { - @Override - public CallImplementor getImplementor() { - return createImplementor(new ZetaSQLCastCallNotNullImplementor(), NullPolicy.STRICT, false); - } - - @Override - public List getParameters() { - return Collections.emptyList(); - } - - private static class ZetaSQLCastCallNotNullImplementor implements NotNullImplementor { - - @Override - public Expression implement( - RexToLixTranslator rexToLixTranslator, RexCall rexCall, List list) { - if (rexCall.getOperands().size() != 1 || list.size() != 1) { - throw new IllegalArgumentException("CAST should have one operand."); - } - SqlTypeName toType = rexCall.getType().getSqlTypeName(); - SqlTypeName fromType = rexCall.getOperands().get(0).getType().getSqlTypeName(); - - Expression translatedOperand = list.get(0); - Expression convertedOperand; - // CAST(BYTES AS STRING) - BINARY to VARCHAR in Calcite - if (fromType == SqlTypeName.BINARY && toType == SqlTypeName.VARCHAR) { - // operand is literal, which is bytes wrapped in ByteString. - // this piece of code is same as - // BeamCodegenUtils.toStringUTF8(ByeString.getBytes()); - convertedOperand = - Expressions.call( - BeamCodegenUtils.class, - "toStringUTF8", - Expressions.call(translatedOperand, "getBytes")); - } else if (fromType == SqlTypeName.VARBINARY && toType == SqlTypeName.VARCHAR) { - // translatedOperand is a byte[] - // this piece of code is same as - // BeamCodegenUtils.toStringUTF8(byte[]); - convertedOperand = - Expressions.call(BeamCodegenUtils.class, "toStringUTF8", translatedOperand); - } else if (fromType == SqlTypeName.BOOLEAN && toType == SqlTypeName.BIGINT) { - convertedOperand = - Expressions.condition( - translatedOperand, - Expressions.constant(1L, Long.class), - Expressions.constant(0L, Long.class)); - } else if (fromType == SqlTypeName.BIGINT && toType == SqlTypeName.BOOLEAN) { - convertedOperand = Expressions.notEqual(translatedOperand, Expressions.constant(0)); - } else if (fromType == SqlTypeName.TIMESTAMP && toType == SqlTypeName.VARCHAR) { - convertedOperand = - Expressions.call(BeamCodegenUtils.class, "toStringTimestamp", translatedOperand); - } else { - throw new UnsupportedOperationException( - "Unsupported CAST: " + fromType.name() + " to " + toType.name()); - } - - // If operand is nullable, wrap in a null check - if (rexCall.getOperands().get(0).getType().isNullable()) { - convertedOperand = - Expressions.condition( - Expressions.equal(translatedOperand, RexImpTable.NULL_EXPR), - RexImpTable.NULL_EXPR, - convertedOperand); - } - - return convertedOperand; - } - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/DateFunctions.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/DateFunctions.java deleted file mode 100644 index e5fda88327d3..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/DateFunctions.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.translation.impl; - -import java.util.TimeZone; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; - -/** DateFunctions. */ -@Internal -public class DateFunctions { - public DateTime date(Integer year, Integer month, Integer day) { - return DateTimeUtils.parseDate( - String.join("-", year.toString(), month.toString(), day.toString())); - } - - public DateTime date(DateTime ts) { - return date(ts, "UTC"); - } - - public DateTime date(DateTime ts, String timezone) { - return ts.withZoneRetainFields(DateTimeZone.forTimeZone(TimeZone.getTimeZone(timezone))); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/StringFunctions.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/StringFunctions.java deleted file mode 100644 index ac32fd32c46e..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/StringFunctions.java +++ /dev/null @@ -1,184 +0,0 @@ -/* - * 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.translation.impl; - -import java.util.regex.Pattern; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.function.Strict; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.SqlFunctions; - -/** StringFunctions. */ -@Internal -public class StringFunctions { - public static final String SUBSTR_PARAMETER_EXCEED_INTEGER = - "SUBSTR function only allows: " - + Integer.MIN_VALUE - + " <= position or length <= " - + Integer.MAX_VALUE; - - @Strict - public static Boolean startsWith(String str1, String str2) { - return str1.startsWith(str2); - } - - @Strict - public static Boolean endsWith(String str1, String str2) { - return str1.endsWith(str2); - } - - @Strict - public static String concat(String arg) { - return arg; - } - - @Strict - public static String concat(String arg1, String arg2) { - return concatIfNotIncludeNull(arg1, arg2); - } - - @Strict - public static String concat(String arg1, String arg2, String arg3) { - return concatIfNotIncludeNull(arg1, arg2, arg3); - } - - @Strict - public static String concat(String arg1, String arg2, String arg3, String arg4) { - return concatIfNotIncludeNull(arg1, arg2, arg3, arg4); - } - - @Strict - public static String concat(String arg1, String arg2, String arg3, String arg4, String arg5) { - return concatIfNotIncludeNull(arg1, arg2, arg3, arg4, arg5); - } - - @Strict - private static String concatIfNotIncludeNull(String... args) { - return String.join("", args); - } - - // https://jira.apache.org/jira/browse/CALCITE-2889 - // public static String concat(String... args) { - // StringBuilder stringBuilder = new StringBuilder(); - // for (String arg : args) { - // stringBuilder.append(arg); - // } - // return stringBuilder.toString(); - // } - - @Strict - public static String replace(String origin, String target, String replacement) { - // Java's string.replace behaves differently when target = "". When target = "", - // Java's replace function replace every character in origin with replacement, - // while origin value should not be changed is expected in SQL. - if (target.length() == 0) { - return origin; - } - - return origin.replace(target, replacement); - } - - public static String trim(String str) { - return trim(str, " "); - } - - @Strict - public static String trim(String str, String seek) { - return SqlFunctions.trim(true, true, seek, str, false); - } - - public static String ltrim(String str) { - return ltrim(str, " "); - } - - @Strict - public static String ltrim(String str, String seek) { - return SqlFunctions.trim(true, false, seek, str, false); - } - - public static String rtrim(String str) { - return rtrim(str, " "); - } - - @Strict - public static String rtrim(String str, String seek) { - return SqlFunctions.trim(false, true, seek, str, false); - } - - public static String substr(String str, long from, long len) { - if (from > Integer.MAX_VALUE - || len > Integer.MAX_VALUE - || from < Integer.MIN_VALUE - || len < Integer.MIN_VALUE) { - throw new RuntimeException(SUBSTR_PARAMETER_EXCEED_INTEGER); - } - return SqlFunctions.substring(str, (int) from, (int) len); - } - - @Strict - public static String reverse(String str) { - return new StringBuilder(str).reverse().toString(); - } - - @Strict - public static Long charLength(String str) { - return (long) str.length(); - } - - // ZetaSQL's LIKE statement does not support the ESCAPE clause. Instead it - // always uses \ as an escape character. - @Strict - public static Boolean like(String s, String pattern) { - String regex = sqlToRegexLike(pattern, '\\'); - return Pattern.matches(regex, s); - } - - private static final String JAVA_REGEX_SPECIALS = "[]()|^-+*?{}$\\."; - - /** - * Translates a SQL LIKE pattern to Java regex pattern. Modified from Apache Calcite's - * Like.sqlToRegexLike - */ - private static String sqlToRegexLike(String sqlPattern, char escapeChar) { - int i; - final int len = sqlPattern.length(); - final StringBuilder javaPattern = new StringBuilder(len + len); - for (i = 0; i < len; i++) { - char c = sqlPattern.charAt(i); - if (c == escapeChar) { - if (i == (sqlPattern.length() - 1)) { - throw new IllegalArgumentException("LIKE pattern ends with a backslash"); - } - char nextChar = sqlPattern.charAt(++i); - if (JAVA_REGEX_SPECIALS.indexOf(nextChar) >= 0) { - javaPattern.append('\\'); - } - javaPattern.append(nextChar); - } else if (c == '_') { - javaPattern.append('.'); - } else if (c == '%') { - javaPattern.append("(?s:.*)"); - } else { - if (JAVA_REGEX_SPECIALS.indexOf(c) >= 0) { - javaPattern.append('\\'); - } - javaPattern.append(c); - } - } - return javaPattern.toString(); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/TimestampFunctions.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/TimestampFunctions.java deleted file mode 100644 index 904b192b2a5b..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/TimestampFunctions.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.translation.impl; - -import java.util.TimeZone; -import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.linq4j.function.Strict; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; - -/** TimestampFunctions. */ -@Internal -public class TimestampFunctions { - public static DateTime timestamp(String timestampStr) { - return timestamp(timestampStr, "UTC"); - } - - @Strict - public static DateTime timestamp(String timestampStr, String timezone) { - return DateTimeUtils.findDateTimePattern(timestampStr) - .withZone(DateTimeZone.forTimeZone(TimeZone.getTimeZone(timezone))) - .parseDateTime(timestampStr); - } - - @Strict - public static DateTime timestamp(Integer numOfDays) { - return timestamp(numOfDays, "UTC"); - } - - @Strict - public static DateTime timestamp(Integer numOfDays, String timezone) { - return new DateTime((long) numOfDays * DateTimeUtils.MILLIS_PER_DAY, DateTimeZone.UTC) - .withZoneRetainFields(DateTimeZone.forTimeZone(TimeZone.getTimeZone(timezone))); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/package-info.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/package-info.java deleted file mode 100644 index 31ce9c11cdc4..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/package-info.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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. - */ - -/** - * Java implementation of ZetaSQL functions. - * - *

Used only by {@link org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel}. - */ -package org.apache.beam.sdk.extensions.sql.zetasql.translation.impl; diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/package-info.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/package-info.java deleted file mode 100644 index d01c6565c7e5..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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. - */ - -/** Conversion logic between ZetaSQL resolved query nodes and Calcite rel nodes. */ -package org.apache.beam.sdk.extensions.sql.zetasql.translation; diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRel.java deleted file mode 100644 index 4cce106c6279..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRel.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.unnest; - -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel; -import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelMetadataQuery; -import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats; -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.extensions.sql.impl.utils.CalciteUtils; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; - -/** - * {@link BeamRelNode} to implement an uncorrelated {@link ZetaSqlUnnest}, aka UNNEST. - * - *

This class is a copy of {@link org.apache.beam.sdk.extensions.sql.impl.rel.BeamUncollectRel} - * except that in UncollectDoFn it does not treat elements of struct type differently. - * - *

Details of why unwrapping structs breaks ZetaSQL UNNEST syntax is in - * https://issues.apache.org/jira/browse/BEAM-10896. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class BeamZetaSqlUncollectRel extends ZetaSqlUnnest implements BeamRelNode { - - public BeamZetaSqlUncollectRel( - RelOptCluster cluster, RelTraitSet traitSet, RelNode input, boolean withOrdinality) { - super(cluster, traitSet, input, withOrdinality); - } - - @Override - public RelNode copy(RelTraitSet traitSet, RelNode input) { - return new BeamZetaSqlUncollectRel(getCluster(), traitSet, input, withOrdinality); - } - - @Override - public PTransform, PCollection> buildPTransform() { - return new Transform(); - } - - private class Transform extends PTransform, PCollection> { - - @Override - public PCollection expand(PCollectionList pinput) { - checkArgument( - pinput.size() == 1, - "Wrong number of inputs for %s: %s", - BeamZetaSqlUncollectRel.class.getSimpleName(), - pinput); - PCollection upstream = pinput.get(0); - - // Each row of the input contains a single array of things to be emitted; Calcite knows - // what the row looks like - Schema outputSchema = CalciteUtils.toSchema(getRowType()); - - PCollection uncollected = - upstream.apply(ParDo.of(new UncollectDoFn(outputSchema))).setRowSchema(outputSchema); - - return uncollected; - } - } - - @Override - public NodeStats estimateNodeStats(BeamRelMetadataQuery mq) { - // We estimate the average length of each array by a constant. - // We might be able to get an estimate of the length by making a MetadataHandler for this - // purpose, and get the estimate by reading the first couple of the rows in the source. - return BeamSqlRelUtils.getNodeStats(this.input, mq).multiply(2); - } - - @Override - public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, BeamRelMetadataQuery mq) { - NodeStats estimates = BeamSqlRelUtils.getNodeStats(this, mq); - return BeamCostModel.FACTORY.makeCost(estimates.getRowCount(), estimates.getRate()); - } - - private static class UncollectDoFn extends DoFn { - - private final Schema schema; - - private UncollectDoFn(Schema schema) { - this.schema = schema; - } - - @ProcessElement - public void process(@Element Row inputRow, OutputReceiver output) { - for (Object element : inputRow.getArray(0)) { - output.output(Row.withSchema(schema).addValue(element).build()); - } - } - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRule.java deleted file mode 100644 index 61e2f07440b7..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRule.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.unnest; - -import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.convert.ConverterRule; - -/** - * A {@code ConverterRule} to replace {@link ZetaSqlUnnest} with {@link BeamZetaSqlUncollectRel}. - * - *

This class is a copy of {@link org.apache.beam.sdk.extensions.sql.impl.rel.BeamUncollectRel} - * except that it works on {@link ZetaSqlUnnest} instead of Calcite Uncollect. - * - *

Details of why unwrapping structs breaks ZetaSQL UNNEST syntax is in - * https://issues.apache.org/jira/browse/BEAM-10896. - */ -public class BeamZetaSqlUncollectRule extends ConverterRule { - public static final BeamZetaSqlUncollectRule INSTANCE = new BeamZetaSqlUncollectRule(); - - private BeamZetaSqlUncollectRule() { - super( - ZetaSqlUnnest.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamUncollectRule"); - } - - @Override - public RelNode convert(RelNode rel) { - ZetaSqlUnnest uncollect = (ZetaSqlUnnest) rel; - - return new BeamZetaSqlUncollectRel( - uncollect.getCluster(), - uncollect.getTraitSet().replace(BeamLogicalConvention.INSTANCE), - convert( - uncollect.getInput(), - uncollect.getInput().getTraitSet().replace(BeamLogicalConvention.INSTANCE)), - uncollect.withOrdinality); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRel.java deleted file mode 100644 index 54a0cc5ad919..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRel.java +++ /dev/null @@ -1,164 +0,0 @@ -/* - * 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.unnest; - -import java.util.Collection; -import java.util.List; -import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel; -import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelMetadataQuery; -import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats; -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.extensions.sql.impl.utils.CalciteUtils; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionList; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Correlate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.validate.SqlValidatorUtil; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * {@link BeamRelNode} to implement UNNEST, supporting specifically only {@link Correlate} with - * {@link ZetaSqlUnnest}. - * - *

This class is a copy of {@link org.apache.beam.sdk.extensions.sql.impl.rel.BeamUnnestRel} - * except that in UnnestFn it does not treat elements of struct type differently. - * - *

Details of why unwrapping structs breaks ZetaSQL UNNEST syntax is in - * https://issues.apache.org/jira/browse/BEAM-10896. - */ -@SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -public class BeamZetaSqlUnnestRel extends ZetaSqlUnnest implements BeamRelNode { - - private final RelDataType unnestType; - private final List unnestIndices; - - public BeamZetaSqlUnnestRel( - RelOptCluster cluster, - RelTraitSet traitSet, - RelNode input, - RelDataType unnestType, - List unnestIndices) { - super(cluster, traitSet, input, false); - this.unnestType = unnestType; - this.unnestIndices = unnestIndices; - } - - @Override - public ZetaSqlUnnest copy(RelTraitSet traitSet, RelNode input) { - return new BeamZetaSqlUnnestRel(getCluster(), traitSet, input, unnestType, unnestIndices); - } - - @Override - protected RelDataType deriveRowType() { - return SqlValidatorUtil.deriveJoinRowType( - input.getRowType(), - unnestType, - JoinRelType.INNER, - getCluster().getTypeFactory(), - null, - ImmutableList.of()); - } - - @Override - public NodeStats estimateNodeStats(BeamRelMetadataQuery mq) { - // We estimate the average length of each array by a constant. - // We might be able to get an estimate of the length by making a MetadataHandler for this - // purpose, and get the estimate by reading the first couple of the rows in the source. - return BeamSqlRelUtils.getNodeStats(this.input, mq).multiply(2); - } - - @Override - public BeamCostModel beamComputeSelfCost(RelOptPlanner planner, BeamRelMetadataQuery mq) { - NodeStats estimates = BeamSqlRelUtils.getNodeStats(this, mq); - return BeamCostModel.FACTORY.makeCost(estimates.getRowCount(), estimates.getRate()); - } - - @Override - public RelWriter explainTerms(RelWriter pw) { - return super.explainTerms(pw).item("unnestIndices", unnestIndices); - } - - @Override - public PTransform, PCollection> buildPTransform() { - return new Transform(); - } - - private class Transform extends PTransform, PCollection> { - @Override - public PCollection expand(PCollectionList pinput) { - // The set of rows where we run the correlated unnest for each row - PCollection outer = pinput.get(0); - - Schema joinedSchema = CalciteUtils.toSchema(getRowType()); - - return outer - .apply(ParDo.of(new UnnestFn(joinedSchema, unnestIndices))) - .setRowSchema(joinedSchema); - } - } - - private static class UnnestFn extends DoFn { - - private final Schema outputSchema; - private final List unnestIndices; - - private UnnestFn(Schema outputSchema, List unnestIndices) { - this.outputSchema = outputSchema; - this.unnestIndices = unnestIndices; - } - - @ProcessElement - public void process(@Element Row row, OutputReceiver out) { - Row rowWithArrayField = row; - Schema schemaWithArrayField = outputSchema; - for (int i = unnestIndices.size() - 1; i > 0; i--) { - rowWithArrayField = rowWithArrayField.getRow(unnestIndices.get(i)); - schemaWithArrayField = - schemaWithArrayField.getField(unnestIndices.get(i)).getType().getRowSchema(); - } - @Nullable Collection rawValues = rowWithArrayField.getArray(unnestIndices.get(0)); - - if (rawValues == null) { - return; - } - - for (Object uncollectedValue : rawValues) { - out.output( - Row.withSchema(outputSchema) - .addValues(row.getBaseValues()) - .addValue(uncollectedValue) - .build()); - } - } - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRule.java deleted file mode 100644 index d9fed7f6ae6b..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRule.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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.unnest; - -import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRule; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptRuleCall; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.volcano.RelSubset; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.SingleRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.Correlate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.core.JoinRelType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalCorrelate; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.logical.LogicalProject; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexFieldAccess; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; - -/** - * A {@code ConverterRule} to replace {@link Correlate} {@link ZetaSqlUnnest} with {@link - * BeamZetaSqlUnnestRel}. - * - *

This class is a copy of {@link org.apache.beam.sdk.extensions.sql.impl.rule.BeamUnnestRule} - * except that it works on {@link ZetaSqlUnnest} instead of Calcite Uncollect. - * - *

Details of why unwrapping structs breaks ZetaSQL UNNEST syntax is in - * https://issues.apache.org/jira/browse/BEAM-10896. - */ -public class BeamZetaSqlUnnestRule extends RelOptRule { - public static final BeamZetaSqlUnnestRule INSTANCE = new BeamZetaSqlUnnestRule(); - - // TODO: more general Correlate - private BeamZetaSqlUnnestRule() { - super( - operand( - LogicalCorrelate.class, operand(RelNode.class, any()), operand(SingleRel.class, any())), - "BeamZetaSqlUnnestRule"); - } - - @Override - public void onMatch(RelOptRuleCall call) { - LogicalCorrelate correlate = call.rel(0); - RelNode outer = call.rel(1); - RelNode uncollect = call.rel(2); - - if (correlate.getRequiredColumns().cardinality() != 1) { - // can only unnest a single column - return; - } - if (correlate.getJoinType() != JoinRelType.INNER) { - return; - } - - if (!(uncollect instanceof ZetaSqlUnnest)) { - // Drop projection - uncollect = ((SingleRel) uncollect).getInput(); - if (uncollect instanceof RelSubset) { - uncollect = ((RelSubset) uncollect).getOriginal(); - } - if (!(uncollect instanceof ZetaSqlUnnest)) { - return; - } - } - - RelNode project = ((ZetaSqlUnnest) uncollect).getInput(); - if (project instanceof RelSubset) { - project = ((RelSubset) project).getOriginal(); - } - if (!(project instanceof LogicalProject)) { - return; - } - - if (((LogicalProject) project).getProjects().size() != 1) { - // can only unnest a single column - return; - } - - RexNode exp = ((LogicalProject) project).getProjects().get(0); - if (!(exp instanceof RexFieldAccess)) { - return; - } - RexFieldAccess fieldAccess = (RexFieldAccess) exp; - // Innermost field index comes first (e.g. struct.field1.field2 => [2, 1]) - ImmutableList.Builder fieldAccessIndices = ImmutableList.builder(); - while (true) { - fieldAccessIndices.add(fieldAccess.getField().getIndex()); - if (!(fieldAccess.getReferenceExpr() instanceof RexFieldAccess)) { - break; - } - fieldAccess = (RexFieldAccess) fieldAccess.getReferenceExpr(); - } - - call.transformTo( - new BeamZetaSqlUnnestRel( - correlate.getCluster(), - correlate.getTraitSet().replace(BeamLogicalConvention.INSTANCE), - convert(outer, outer.getTraitSet().replace(BeamLogicalConvention.INSTANCE)), - call.rel(2).getRowType(), - fieldAccessIndices.build())); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/ZetaSqlUnnest.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/ZetaSqlUnnest.java deleted file mode 100644 index 1c6f47ab0372..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/ZetaSqlUnnest.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * 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.unnest; - -import java.util.List; -import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Convention; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCluster; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelTraitSet; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelInput; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelWriter; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.SingleRel; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.type.RelDataTypeField; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUnnestOperator; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlUtil; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.ArraySqlType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.MapSqlType; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; - -/** - * This class is a copy of Uncollect.java in Calcite: - * https://github.com/apache/calcite/blob/calcite-1.20.0/core/src/main/java/org/apache/calcite/rel/core/Uncollect.java - * except that in deriveUncollectRowType() it does not unwrap array elements of struct type. - * - *

Details of why unwrapping structs breaks ZetaSQL UNNEST syntax is in - * https://issues.apache.org/jira/browse/BEAM-10896. - */ -public class ZetaSqlUnnest extends SingleRel { - public final boolean withOrdinality; - - // ~ Constructors ----------------------------------------------------------- - - /** - * Creates an Uncollect. - * - *

Use {@link #create} unless you know what you're doing. - */ - public ZetaSqlUnnest( - RelOptCluster cluster, RelTraitSet traitSet, RelNode input, boolean withOrdinality) { - super(cluster, traitSet, input); - this.withOrdinality = withOrdinality; - } - - /** Creates an Uncollect by parsing serialized output. */ - public ZetaSqlUnnest(RelInput input) { - this( - input.getCluster(), - input.getTraitSet(), - input.getInput(), - input.getBoolean("withOrdinality", false)); - } - - /** - * Creates an Uncollect. - * - *

Each field of the input relational expression must be an array or multiset. - * - * @param traitSet Trait set - * @param input Input relational expression - * @param withOrdinality Whether output should contain an ORDINALITY column - */ - public static ZetaSqlUnnest create(RelTraitSet traitSet, RelNode input, boolean withOrdinality) { - final RelOptCluster cluster = input.getCluster(); - return new ZetaSqlUnnest(cluster, traitSet, input, withOrdinality); - } - - // ~ Methods ---------------------------------------------------------------- - - @Override - public RelWriter explainTerms(RelWriter pw) { - return super.explainTerms(pw).itemIf("withOrdinality", withOrdinality, withOrdinality); - } - - @Override - public final RelNode copy(RelTraitSet traitSet, List inputs) { - return copy(traitSet, sole(inputs)); - } - - public RelNode copy(RelTraitSet traitSet, RelNode input) { - assert traitSet.containsIfApplicable(Convention.NONE); - return new ZetaSqlUnnest(getCluster(), traitSet, input, withOrdinality); - } - - @Override - protected RelDataType deriveRowType() { - return deriveUncollectRowType(input, withOrdinality); - } - - /** - * Returns the row type returned by applying the 'UNNEST' operation to a relational expression. - * - *

Each column in the relational expression must be a multiset of structs or an array. The - * return type is the type of that column, plus an ORDINALITY column if {@code withOrdinality}. - */ - public static RelDataType deriveUncollectRowType(RelNode rel, boolean withOrdinality) { - RelDataType inputType = rel.getRowType(); - assert inputType.isStruct() : inputType + " is not a struct"; - final List fields = inputType.getFieldList(); - final RelDataTypeFactory typeFactory = rel.getCluster().getTypeFactory(); - final RelDataTypeFactory.Builder builder = typeFactory.builder(); - - if (fields.size() == 1 && fields.get(0).getType().getSqlTypeName() == SqlTypeName.ANY) { - // Component type is unknown to Uncollect, build a row type with input column name - // and Any type. - return builder.add(fields.get(0).getName(), SqlTypeName.ANY).nullable(true).build(); - } - - for (RelDataTypeField field : fields) { - if (field.getType() instanceof MapSqlType) { - builder.add( - SqlUnnestOperator.MAP_KEY_COLUMN_NAME, - Preconditions.checkArgumentNotNull( - field.getType().getKeyType(), - "Encountered MAP type with null key type in field %s", - field)); - builder.add( - SqlUnnestOperator.MAP_VALUE_COLUMN_NAME, - Preconditions.checkArgumentNotNull( - field.getType().getValueType(), - "Encountered MAP type with null value type in field %s", - field)); - } else { - assert field.getType() instanceof ArraySqlType; - RelDataType ret = - Preconditions.checkArgumentNotNull( - field.getType().getComponentType(), - "Encountered ARRAY type with null component type in field %s", - field); - // Only difference than Uncollect.java: treats record type and scalar type equally - builder.add(SqlUtil.deriveAliasFromOrdinal(field.getIndex()), ret); - } - } - if (withOrdinality) { - builder.add(SqlUnnestOperator.ORDINALITY_COLUMN_NAME, SqlTypeName.INTEGER); - } - return builder.build(); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/package-info.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/package-info.java deleted file mode 100644 index 8c741c29a36d..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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. - */ - -/** Temporary solution to support ZetaSQL UNNEST. To be removed after Calcite upgrade. */ -package org.apache.beam.sdk.extensions.sql.zetasql.unnest; diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRuleTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRuleTest.java deleted file mode 100644 index a6ca07b307ea..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRuleTest.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * 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 static org.hamcrest.Matchers.isA; - -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; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptPlanner; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Frameworks; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.joda.time.Duration; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for {@link BeamJavaUdfCalcRule}. */ -@RunWith(JUnit4.class) -public class BeamJavaUdfCalcRuleTest extends ZetaSqlTestBase { - @Rule public transient TestPipeline pipeline = TestPipeline.create(); - @Rule public ExpectedException thrown = ExpectedException.none(); - - @Before - public void setUp() { - initialize(); - - this.config = - Frameworks.newConfigBuilder(config) - .ruleSets( - ZetaSQLQueryPlanner.getZetaSqlRuleSets( - ImmutableList.of(BeamJavaUdfCalcRule.INSTANCE)) - .toArray(new RuleSet[0])) - .build(); - } - - @Test - public void testSelectLiteral() { - String sql = "SELECT 1;"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema singleField = Schema.builder().addInt64Field("field1").build(); - - PAssert.that(stream).containsInAnyOrder(Row.withSchema(singleField).addValues(1L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testBuiltinFunctionThrowsSqlConversionException() { - String sql = "SELECT abs(1);"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - - thrown.expect(SqlConversionException.class); - thrown.expectCause(isA(RelOptPlanner.CannotPlanException.class)); - - zetaSQLQueryPlanner.convertToBeamRel(sql); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRelTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRelTest.java deleted file mode 100644 index 2cb8501eb276..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRelTest.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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 org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner.QueryParameters; -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.runners.TransformHierarchy; -import org.apache.beam.sdk.schemas.FieldAccessDescriptor; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.DoFnSchemaInformation; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; - -/** Tests related to {@code BeamZetaSqlCalcRel}. */ -public class BeamZetaSqlCalcRelTest extends ZetaSqlTestBase { - - private PCollection compile(String sql) { - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, QueryParameters.ofNone()); - return BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - } - - @Rule public final TestPipeline pipeline = TestPipeline.create(); - - @Before - public void setUp() { - initialize(); - } - - private static class NodeGetter extends Pipeline.PipelineVisitor.Defaults { - - private final PValue target; - private TransformHierarchy.Node producer; - - private NodeGetter(PValue target) { - this.target = target; - } - - @Override - public void visitValue(PValue value, TransformHierarchy.Node producer) { - if (value == target) { - assert this.producer == null; - this.producer = producer; - } - } - } - - @Test - public void testSingleFieldAccess() throws IllegalAccessException { - String sql = "SELECT Key FROM KeyValue"; - - PCollection rows = compile(sql); - - final NodeGetter nodeGetter = new NodeGetter(rows); - pipeline.traverseTopologically(nodeGetter); - - ParDo.MultiOutput pardo = - (ParDo.MultiOutput) nodeGetter.producer.getTransform(); - PCollection input = - (PCollection) Iterables.getOnlyElement(nodeGetter.producer.getInputs().values()); - - DoFnSchemaInformation info = ParDo.getDoFnSchemaInformation(pardo.getFn(), input); - - FieldAccessDescriptor fieldAccess = info.getFieldAccessDescriptor(); - - Assert.assertTrue(fieldAccess.referencesSingleField()); - Assert.assertEquals("Key", Iterables.getOnlyElement(fieldAccess.fieldNamesAccessed())); - - pipeline.run().waitUntilFinish(); - } - - @Test - public void testNoFieldAccess() throws IllegalAccessException { - String sql = "SELECT 1 FROM KeyValue"; - - PCollection rows = compile(sql); - - final NodeGetter nodeGetter = new NodeGetter(rows); - pipeline.traverseTopologically(nodeGetter); - - ParDo.MultiOutput pardo = - (ParDo.MultiOutput) nodeGetter.producer.getTransform(); - PCollection input = - (PCollection) Iterables.getOnlyElement(nodeGetter.producer.getInputs().values()); - - DoFnSchemaInformation info = ParDo.getDoFnSchemaInformation(pardo.getFn(), input); - - FieldAccessDescriptor fieldAccess = info.getFieldAccessDescriptor(); - - Assert.assertFalse(fieldAccess.getAllFields()); - Assert.assertTrue(fieldAccess.getFieldsAccessed().isEmpty()); - Assert.assertTrue(fieldAccess.getNestedFieldsAccessed().isEmpty()); - - pipeline.run().waitUntilFinish(); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalogTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalogTest.java deleted file mode 100644 index 733de268c88b..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalogTest.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * 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 static org.apache.beam.sdk.extensions.sql.zetasql.BeamZetaSqlCatalog.USER_DEFINED_JAVA_SCALAR_FUNCTIONS; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -import com.google.zetasql.Analyzer; -import com.google.zetasql.AnalyzerOptions; -import com.google.zetasql.resolvedast.ResolvedNodes; -import java.lang.reflect.Method; -import java.sql.Time; -import java.util.List; -import org.apache.beam.sdk.extensions.sql.BeamSqlUdf; -import org.apache.beam.sdk.extensions.sql.impl.JdbcConnection; -import org.apache.beam.sdk.extensions.sql.impl.JdbcDriver; -import org.apache.beam.sdk.extensions.sql.impl.ScalarFunctionImpl; -import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider; -import org.apache.beam.sdk.extensions.sql.zetasql.translation.UserFunctionDefinitions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for {@link BeamZetaSqlCatalog}. */ -@RunWith(JUnit4.class) -public class BeamZetaSqlCatalogTest { - @Rule public ExpectedException thrown = ExpectedException.none(); - - public static class IncrementFn implements BeamSqlUdf { - public Long eval(Long i) { - return i + 1; - } - } - - public static class ReturnsArrayTimeFn implements BeamSqlUdf { - public List

System properties beam.sql.udf.test.jarpath and - * beam.sql.udf.test.empty_jar_path must be set. - */ -@RunWith(JUnit4.class) -public class ZetaSqlJavaUdfTest extends ZetaSqlTestBase { - @Rule public transient TestPipeline pipeline = TestPipeline.create(); - @Rule public ExpectedException thrown = ExpectedException.none(); - - private final String jarPathProperty = "beam.sql.udf.test.jar_path"; - private final String emptyJarPathProperty = "beam.sql.udf.test.empty_jar_path"; - - private final @Nullable String jarPath = System.getProperty(jarPathProperty); - private final @Nullable String emptyJarPath = System.getProperty(emptyJarPathProperty); - - @Before - public void setUp() { - if (jarPath == null) { - fail( - String.format( - "System property %s must be set to run %s.", - jarPathProperty, ZetaSqlJavaUdfTest.class.getSimpleName())); - } - if (emptyJarPath == null) { - fail( - String.format( - "System property %s must be set to run %s.", - emptyJarPathProperty, ZetaSqlJavaUdfTest.class.getSimpleName())); - } - initialize(); - } - - @Test - public void testNullaryJavaUdf() { - String sql = - String.format( - "CREATE FUNCTION helloWorld() RETURNS STRING LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT helloWorld();", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema singleField = Schema.builder().addStringField("field1").build(); - - PAssert.that(stream) - .containsInAnyOrder(Row.withSchema(singleField).addValues("Hello world!").build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testUnaryJavaUdf() { - String sql = - String.format( - "CREATE FUNCTION increment(i INT64) RETURNS INT64 LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT increment(1);", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema singleField = Schema.builder().addInt64Field("field1").build(); - - PAssert.that(stream).containsInAnyOrder(Row.withSchema(singleField).addValues(2L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testJavaUdfColumnReference() { - String sql = - String.format( - "CREATE FUNCTION increment(i INT64) RETURNS INT64 LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT increment(int64_col) FROM table_all_types;", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema singleField = Schema.builder().addInt64Field("field1").build(); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(singleField).addValues(0L).build(), - Row.withSchema(singleField).addValues(-1L).build(), - Row.withSchema(singleField).addValues(-2L).build(), - Row.withSchema(singleField).addValues(-3L).build(), - Row.withSchema(singleField).addValues(-4L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testNestedJavaUdf() { - String sql = - String.format( - "CREATE FUNCTION increment(i INT64) RETURNS INT64 LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT increment(increment(1));", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema singleField = Schema.builder().addInt64Field("field1").build(); - - PAssert.that(stream).containsInAnyOrder(Row.withSchema(singleField).addValues(3L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testUnexpectedNullArgumentThrowsRuntimeException() { - String sql = - String.format( - "CREATE FUNCTION increment(i INT64) RETURNS INT64 LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT increment(NULL);", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - thrown.expect(Pipeline.PipelineExecutionException.class); - thrown.expectMessage("CalcFn failed to evaluate"); - thrown.expectCause( - allOf(isA(RuntimeException.class), hasProperty("cause", isA(NullPointerException.class)))); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testExpectedNullArgument() { - String sql = - String.format( - "CREATE FUNCTION isNull(s STRING) RETURNS BOOLEAN LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT isNull(NULL);", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema singleField = Schema.builder().addBooleanField("field1").build(); - - PAssert.that(stream).containsInAnyOrder(Row.withSchema(singleField).addValues(true).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - public static class IncrementFn implements BeamSqlUdf { - public Long eval(Long i) { - return i + 1; - } - } - - @Test - public void testSqlTransformRegisterUdf() { - String sql = "SELECT increment(0);"; - PCollection stream = - pipeline.apply( - SqlTransform.query(sql) - .withQueryPlannerClass(ZetaSQLQueryPlanner.class) - .registerUdf("increment", IncrementFn.class)); - final Schema schema = Schema.builder().addInt64Field("field1").build(); - PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(1L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - /** This tests a subset of the code path used by {@link #testSqlTransformRegisterUdf()}. */ - @Test - public void testUdfFromCatalog() throws NoSuchMethodException { - // Add IncrementFn to Calcite schema. - JdbcConnection jdbcConnection = - JdbcDriver.connect( - new ReadOnlyTableProvider("empty_table_provider", ImmutableMap.of()), - PipelineOptionsFactory.create()); - Method method = IncrementFn.class.getMethod("eval", Long.class); - jdbcConnection.getCurrentSchemaPlus().add("increment", ScalarFunctionImpl.create(method)); - this.config = - Frameworks.newConfigBuilder(config) - .defaultSchema(jdbcConnection.getCurrentSchemaPlus()) - .build(); - - String sql = "SELECT increment(0);"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = Schema.builder().addInt64Field("field1").build(); - PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(1L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testNullArgumentIsTypeChecked() { - // The Java definition for isNull takes a String, but here we declare it in SQL with INT64. - String sql = - String.format( - "CREATE FUNCTION isNull(i INT64) RETURNS INT64 LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT isNull(NULL);", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - // TODO(https://github.com/apache/beam/issues/20614) This should fail earlier, before compiling - // the CalcFn. - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage("Could not compile CalcFn"); - thrown.expectCause( - allOf( - isA(CompileException.class), - hasProperty( - "message", - containsString( - "No applicable constructor/method found for actual parameters \"java.lang.Long\"")))); - BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - } - - @Test - public void testFunctionSignatureTypeMismatchFailsPipelineConstruction() { - // The Java definition for isNull takes a String, but here we pass it a Long. - String sql = - String.format( - "CREATE FUNCTION isNull(i INT64) RETURNS BOOLEAN LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT isNull(0);", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - // TODO(https://github.com/apache/beam/issues/20614) This should fail earlier, before compiling - // the CalcFn. - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage("Could not compile CalcFn"); - thrown.expectCause( - allOf( - isA(CompileException.class), - hasProperty( - "message", - containsString( - "No applicable constructor/method found for actual parameters \"long\"")))); - BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - } - - @Test - public void testJavaUdfWithNoReturnTypeIsRejected() { - String sql = - String.format( - "CREATE FUNCTION helloWorld() LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT helloWorld();", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(SqlException.class); - thrown.expectMessage("Non-SQL functions must specify a return type"); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testProjectUdfAndBuiltin() { - String sql = - String.format( - "CREATE FUNCTION matches(str STRING, regStr STRING) RETURNS BOOLEAN LANGUAGE java OPTIONS (path='%s'); " - + "SELECT matches(\"a\", \"a\"), 'apple'='beta'", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema schema = Schema.builder().addBooleanField("field1").addBooleanField("field2").build(); - - PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(true, false).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testProjectNestedUdfAndBuiltin() { - String sql = - String.format( - "CREATE FUNCTION increment(i INT64) RETURNS INT64 LANGUAGE java OPTIONS (path='%s'); " - + "SELECT increment(increment(0) + 1);", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema schema = Schema.builder().addInt64Field("field1").build(); - - PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(3L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testJavaUdfEmptyPath() { - String sql = - "CREATE FUNCTION foo() RETURNS STRING LANGUAGE java OPTIONS (path=''); SELECT foo();"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(RuntimeException.class); - thrown.expectMessage("Failed to define function 'foo'"); - thrown.expectCause( - allOf( - isA(IllegalArgumentException.class), - hasProperty("message", containsString("No jar was provided to define function foo.")))); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testJavaUdfNoJarProvided() { - String sql = "CREATE FUNCTION foo() RETURNS STRING LANGUAGE java; SELECT foo();"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(RuntimeException.class); - thrown.expectMessage("Failed to define function 'foo'"); - thrown.expectCause( - allOf( - isA(IllegalArgumentException.class), - hasProperty("message", containsString("No jar was provided to define function foo.")))); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testPathOptionNotString() { - String sql = - "CREATE FUNCTION foo() RETURNS STRING LANGUAGE java OPTIONS (path=23); SELECT foo();"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(RuntimeException.class); - thrown.expectMessage("Failed to define function 'foo'"); - thrown.expectCause( - allOf( - isA(IllegalArgumentException.class), - hasProperty( - "message", - containsString("Option 'path' has type TYPE_INT64 (expected TYPE_STRING).")))); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testUdaf() { - String sql = - String.format( - "CREATE AGGREGATE FUNCTION my_sum(f INT64) RETURNS INT64 LANGUAGE java OPTIONS (path='%s'); " - + "SELECT my_sum(f_int_1) from aggregate_test_table", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema singleField = Schema.builder().addInt64Field("field1").build(); - - PAssert.that(stream).containsInAnyOrder(Row.withSchema(singleField).addValues(28L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testUdafNotFoundFailsToParse() { - String sql = - String.format( - "CREATE AGGREGATE FUNCTION nonexistent(f INT64) RETURNS INT64 LANGUAGE java OPTIONS (path='%s'); " - + "SELECT nonexistent(f_int_1) from aggregate_test_table", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - - thrown.expect(RuntimeException.class); - thrown.expectMessage("Failed to define function 'nonexistent'"); - thrown.expectCause( - allOf( - isA(IllegalArgumentException.class), - hasProperty( - "message", - containsString("No implementation of aggregate function nonexistent found")))); - - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testRegisterUdaf() { - String sql = "SELECT my_sum(k) FROM UNNEST([1, 2, 3]) k;"; - PCollection stream = - pipeline.apply( - SqlTransform.query(sql) - .withQueryPlannerClass(ZetaSQLQueryPlanner.class) - .registerUdaf("my_sum", Sum.ofLongs())); - Schema singleField = Schema.builder().addInt64Field("field1").build(); - PAssert.that(stream).containsInAnyOrder(Row.withSchema(singleField).addValues(6L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateUdf() { - String sql = - String.format( - "CREATE FUNCTION dateIncrementAll(d DATE) RETURNS DATE LANGUAGE java " - + "OPTIONS (path='%s'); " - + "SELECT dateIncrementAll('2020-04-04');", - jarPath); - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - Schema singleField = Schema.builder().addLogicalTypeField("field1", SqlTypes.DATE).build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(singleField).addValues(LocalDate.of(2021, 5, 5)).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTypeTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTypeTest.java deleted file mode 100644 index 74569afd308b..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTypeTest.java +++ /dev/null @@ -1,586 +0,0 @@ -/* - * 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 java.math.BigDecimal; -import java.sql.Date; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.util.List; -import org.apache.beam.sdk.extensions.sql.BeamSqlUdf; -import org.apache.beam.sdk.extensions.sql.impl.JdbcConnection; -import org.apache.beam.sdk.extensions.sql.impl.JdbcDriver; -import org.apache.beam.sdk.extensions.sql.impl.ScalarFunctionImpl; -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.extensions.sql.meta.provider.ReadOnlyTableProvider; -import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.schema.SchemaPlus; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Frameworks; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Duration; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests verifying that various data types can be passed through Java UDFs without data loss. */ -@RunWith(JUnit4.class) -public class ZetaSqlJavaUdfTypeTest extends ZetaSqlTestBase { - @Rule public transient TestPipeline pipeline = TestPipeline.create(); - @Rule public ExpectedException thrown = ExpectedException.none(); - - private static final TestBoundedTable table = - TestBoundedTable.of( - Schema.builder() - .addBooleanField("boolean_true") - .addBooleanField("boolean_false") - .addInt64Field("int64_0") - .addInt64Field("int64_pos") - .addInt64Field("int64_neg") - .addInt64Field("int64_max") - .addInt64Field("int64_min") - .addStringField("string_empty") - .addStringField("string_ascii") - .addStringField("string_unicode") - .addByteArrayField("bytes_empty") - .addByteArrayField("bytes_ascii") - .addByteArrayField("bytes_unicode") - .addDoubleField("float64_0") - .addDoubleField("float64_noninteger") - .addDoubleField("float64_pos") - .addDoubleField("float64_neg") - .addDoubleField("float64_max") - .addDoubleField("float64_min_pos") - .addDoubleField("float64_inf") - .addDoubleField("float64_neg_inf") - .addDoubleField("float64_nan") - .addLogicalTypeField("f_date", SqlTypes.DATE) - .addDateTimeField("f_timestamp") - .addArrayField("array_int64", Schema.FieldType.INT64) - .addDecimalField("numeric_one") - .addDecimalField("numeric_max") - .addDecimalField("numeric_min") - .build()) - .addRows( - true /* boolean_true */, - false /* boolean_false */, - 0L /* int64_0 */, - 123L /* int64_pos */, - -123L /* int64_neg */, - 9223372036854775807L /* int64_max */, - -9223372036854775808L /* int64_min */, - "" /* string_empty */, - "abc" /* string_ascii */, - "スタリング" /* string_unicode */, - new byte[] {} /* bytes_empty */, - new byte[] {'a', 'b', 'c'} /* bytes_ascii */, - new byte[] {-29, -126, -71} /* bytes_unicode */, - 0.0 /* float64_0 */, - 0.123 /* float64_noninteger */, - 123.0 /* float64_pos */, - -123.0 /* float64_neg */, - 1.7976931348623157e+308 /* float64_max */, - 2.2250738585072014e-308 /* float64_min_pos */, - Double.POSITIVE_INFINITY /* float64_inf */, - Double.NEGATIVE_INFINITY /* float64_neg_inf */, - Double.NaN /* float64_nan */, - LocalDate.of(2021, 4, 26) /* f_date */, - new DateTime(2021, 5, 6, 3, 48, 32, DateTimeZone.UTC) /* f_timestamp */, - ImmutableList.of(1L, 2L, 3L) /* array_int64 */, - new BigDecimal("1.000000000" /* numeric_one */), - new BigDecimal("99999999999999999999999999999.999999999" /* numeric_max */), - new BigDecimal("-99999999999999999999999999999.999999999" /* numeric_min */)); - - @Before - public void setUp() throws NoSuchMethodException { - initialize(); - - // Register test table. - JdbcConnection jdbcConnection = - JdbcDriver.connect( - new ReadOnlyTableProvider("table_provider", ImmutableMap.of("table", table)), - PipelineOptionsFactory.create()); - - // Register UDFs. - SchemaPlus schema = jdbcConnection.getCurrentSchemaPlus(); - schema.add( - "test_boolean", - ScalarFunctionImpl.create(BooleanIdentityFn.class.getMethod("eval", Boolean.class))); - schema.add( - "test_int64", - ScalarFunctionImpl.create(Int64IdentityFn.class.getMethod("eval", Long.class))); - schema.add( - "test_string", - ScalarFunctionImpl.create(StringIdentityFn.class.getMethod("eval", String.class))); - schema.add( - "test_bytes", - ScalarFunctionImpl.create(BytesIdentityFn.class.getMethod("eval", byte[].class))); - schema.add( - "test_float64", - ScalarFunctionImpl.create(DoubleIdentityFn.class.getMethod("eval", Double.class))); - schema.add( - "test_date", ScalarFunctionImpl.create(DateIdentityFn.class.getMethod("eval", Date.class))); - schema.add( - "test_timestamp", - ScalarFunctionImpl.create(TimestampIdentityFn.class.getMethod("eval", Timestamp.class))); - schema.add( - "test_array", - ScalarFunctionImpl.create(ListIdentityFn.class.getMethod("eval", List.class))); - schema.add( - "test_numeric", - ScalarFunctionImpl.create(BigDecimalIdentityFn.class.getMethod("eval", BigDecimal.class))); - - this.config = Frameworks.newConfigBuilder(config).defaultSchema(schema).build(); - } - - public static class BooleanIdentityFn implements BeamSqlUdf { - public Boolean eval(Boolean input) { - return input; - } - } - - public static class Int64IdentityFn implements BeamSqlUdf { - public Long eval(Long input) { - return input; - } - } - - public static class StringIdentityFn implements BeamSqlUdf { - public String eval(String input) { - return input; - } - } - - public static class BytesIdentityFn implements BeamSqlUdf { - public byte[] eval(byte[] input) { - return input; - } - } - - public static class DoubleIdentityFn implements BeamSqlUdf { - public Double eval(Double input) { - return input; - } - } - - public static class DateIdentityFn implements BeamSqlUdf { - public Date eval(Date input) { - return input; - } - } - - public static class TimestampIdentityFn implements BeamSqlUdf { - public Timestamp eval(Timestamp input) { - return input; - } - } - - public static class ListIdentityFn implements BeamSqlUdf { - public List eval(List input) { - return input; - } - } - - public static class BigDecimalIdentityFn implements BeamSqlUdf { - public BigDecimal eval(BigDecimal input) { - return input; - } - } - - private void runUdfTypeTest(String query, Object result, Schema.TypeName typeName) { - runUdfTypeTest(query, result, Schema.FieldType.of(typeName)); - } - - private void runUdfTypeTest(String query, Object result, Schema.LogicalType logicalType) { - runUdfTypeTest(query, result, Schema.FieldType.logicalType(logicalType)); - } - - private void runUdfTypeTest(String query, Object result, Schema.FieldType fieldType) { - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(query); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema outputSchema = Schema.builder().addField("res", fieldType).build(); - PAssert.that(stream).containsInAnyOrder(Row.withSchema(outputSchema).addValues(result).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTrueLiteral() { - runUdfTypeTest("SELECT test_boolean(true);", true, Schema.TypeName.BOOLEAN); - } - - @Test - public void testTrueInput() { - runUdfTypeTest("SELECT test_boolean(boolean_true) FROM table;", true, Schema.TypeName.BOOLEAN); - } - - @Test - public void testFalseLiteral() { - runUdfTypeTest("SELECT test_boolean(false);", false, Schema.TypeName.BOOLEAN); - } - - @Test - public void testFalseInput() { - runUdfTypeTest( - "SELECT test_boolean(boolean_false) FROM table;", false, Schema.TypeName.BOOLEAN); - } - - @Test - public void testZeroInt64Literal() { - runUdfTypeTest("SELECT test_int64(0);", 0L, Schema.TypeName.INT64); - } - - @Test - public void testZeroInt64Input() { - runUdfTypeTest("SELECT test_int64(int64_0) FROM table;", 0L, Schema.TypeName.INT64); - } - - @Test - public void testPosInt64Literal() { - runUdfTypeTest("SELECT test_int64(123);", 123L, Schema.TypeName.INT64); - } - - @Test - public void testPosInt64Input() { - runUdfTypeTest("SELECT test_int64(int64_pos) FROM table;", 123L, Schema.TypeName.INT64); - } - - @Test - public void testNegInt64Literal() { - runUdfTypeTest("SELECT test_int64(-123);", -123L, Schema.TypeName.INT64); - } - - @Test - public void testNegInt64Input() { - runUdfTypeTest("SELECT test_int64(int64_neg) FROM table;", -123L, Schema.TypeName.INT64); - } - - @Test - public void testMaxInt64Literal() { - runUdfTypeTest( - "SELECT test_int64(9223372036854775807);", 9223372036854775807L, Schema.TypeName.INT64); - } - - @Test - public void testMaxInt64Input() { - runUdfTypeTest( - "SELECT test_int64(int64_max) FROM table;", 9223372036854775807L, Schema.TypeName.INT64); - } - - @Test - public void testMinInt64Literal() { - runUdfTypeTest( - "SELECT test_int64(-9223372036854775808);", -9223372036854775808L, Schema.TypeName.INT64); - } - - @Test - public void testMinInt64Input() { - runUdfTypeTest( - "SELECT test_int64(int64_min) FROM table;", -9223372036854775808L, Schema.TypeName.INT64); - } - - @Test - public void testEmptyStringLiteral() { - runUdfTypeTest("SELECT test_string('');", "", Schema.TypeName.STRING); - } - - @Test - public void testEmptyStringInput() { - runUdfTypeTest("SELECT test_string(string_empty) FROM table;", "", Schema.TypeName.STRING); - } - - @Test - public void testAsciiStringLiteral() { - runUdfTypeTest("SELECT test_string('abc');", "abc", Schema.TypeName.STRING); - } - - @Test - public void testAsciiStringInput() { - runUdfTypeTest("SELECT test_string(string_ascii) FROM table;", "abc", Schema.TypeName.STRING); - } - - @Test - public void testUnicodeStringLiteral() { - runUdfTypeTest("SELECT test_string('スタリング');", "スタリング", Schema.TypeName.STRING); - } - - @Test - public void testUnicodeStringInput() { - runUdfTypeTest( - "SELECT test_string(string_unicode) FROM table;", "スタリング", Schema.TypeName.STRING); - } - - @Test - public void testEmptyBytesLiteral() { - runUdfTypeTest("SELECT test_bytes(b'');", new byte[] {}, Schema.TypeName.BYTES); - } - - @Test - public void testEmptyBytesInput() { - runUdfTypeTest( - "SELECT test_bytes(bytes_empty) FROM table;", new byte[] {}, Schema.TypeName.BYTES); - } - - @Test - public void testAsciiBytesLiteral() { - runUdfTypeTest("SELECT test_bytes(b'abc');", new byte[] {'a', 'b', 'c'}, Schema.TypeName.BYTES); - } - - @Test - public void testAsciiBytesInput() { - runUdfTypeTest( - "SELECT test_bytes(bytes_ascii) FROM table;", - new byte[] {'a', 'b', 'c'}, - Schema.TypeName.BYTES); - } - - @Test - public void testUnicodeBytesLiteral() { - runUdfTypeTest("SELECT test_bytes(b'ス');", new byte[] {-29, -126, -71}, Schema.TypeName.BYTES); - } - - @Test - public void testUnicodeBytesInput() { - runUdfTypeTest( - "SELECT test_bytes(bytes_unicode) FROM table;", - new byte[] {-29, -126, -71}, - Schema.TypeName.BYTES); - } - - @Test - public void testZeroFloat64Literal() { - runUdfTypeTest("SELECT test_float64(0.0);", 0.0, Schema.TypeName.DOUBLE); - } - - @Test - public void testZeroFloat64Input() { - runUdfTypeTest("SELECT test_float64(float64_0) FROM table;", 0.0, Schema.TypeName.DOUBLE); - } - - @Test - public void testNonIntegerFloat64Literal() { - runUdfTypeTest("SELECT test_float64(0.123);", 0.123, Schema.TypeName.DOUBLE); - } - - @Test - public void testNonIntegerFloat64Input() { - runUdfTypeTest( - "SELECT test_float64(float64_noninteger) FROM table;", 0.123, Schema.TypeName.DOUBLE); - } - - @Test - public void testPosFloat64Literal() { - runUdfTypeTest("SELECT test_float64(123.0);", 123.0, Schema.TypeName.DOUBLE); - } - - @Test - public void testPosFloat64Input() { - runUdfTypeTest("SELECT test_float64(float64_pos) FROM table;", 123.0, Schema.TypeName.DOUBLE); - } - - @Test - public void testNegFloat64Literal() { - runUdfTypeTest("SELECT test_float64(-123.0);", -123.0, Schema.TypeName.DOUBLE); - } - - @Test - public void testNegFloat64Input() { - runUdfTypeTest("SELECT test_float64(float64_neg) FROM table;", -123.0, Schema.TypeName.DOUBLE); - } - - @Test - public void testMaxFloat64Literal() { - runUdfTypeTest( - "SELECT test_float64(1.7976931348623157e+308);", - 1.7976931348623157e+308, - Schema.TypeName.DOUBLE); - } - - @Test - public void testMaxFloat64Input() { - runUdfTypeTest( - "SELECT test_float64(float64_max) FROM table;", - 1.7976931348623157e+308, - Schema.TypeName.DOUBLE); - } - - @Test - public void testMinPosFloat64Literal() { - runUdfTypeTest( - "SELECT test_float64(2.2250738585072014e-308);", - 2.2250738585072014e-308, - Schema.TypeName.DOUBLE); - } - - @Test - public void testMinPosFloat64Input() { - runUdfTypeTest( - "SELECT test_float64(float64_min_pos) FROM table;", - 2.2250738585072014e-308, - Schema.TypeName.DOUBLE); - } - - @Test - public void testPosInfFloat64Literal() { - runUdfTypeTest( - "SELECT test_float64(CAST('+inf' AS FLOAT64));", - Double.POSITIVE_INFINITY, - Schema.TypeName.DOUBLE); - } - - @Test - public void testPosInfFloat64Input() { - runUdfTypeTest( - "SELECT test_float64(float64_inf) FROM table;", - Double.POSITIVE_INFINITY, - Schema.TypeName.DOUBLE); - } - - @Test - public void testNegInfFloat64Literal() { - runUdfTypeTest( - "SELECT test_float64(CAST('-inf' AS FLOAT64));", - Double.NEGATIVE_INFINITY, - Schema.TypeName.DOUBLE); - } - - @Test - public void testNegInfFloat64Input() { - runUdfTypeTest( - "SELECT test_float64(float64_neg_inf) FROM table;", - Double.NEGATIVE_INFINITY, - Schema.TypeName.DOUBLE); - } - - @Test - public void testNaNFloat64Literal() { - runUdfTypeTest( - "SELECT test_float64(CAST('NaN' AS FLOAT64));", Double.NaN, Schema.TypeName.DOUBLE); - } - - @Test - public void testNaNFloat64Input() { - runUdfTypeTest( - "SELECT test_float64(float64_nan) FROM table;", Double.NaN, Schema.TypeName.DOUBLE); - } - - @Test - public void testDateLiteral() { - runUdfTypeTest("SELECT test_date('2021-04-26');", LocalDate.of(2021, 4, 26), SqlTypes.DATE); - } - - @Test - public void testDateInput() { - runUdfTypeTest( - "SELECT test_date(f_date) FROM table;", LocalDate.of(2021, 4, 26), SqlTypes.DATE); - } - - @Test - public void testTimestampLiteral() { - runUdfTypeTest( - "SELECT test_timestamp('2021-05-06 03:48:32Z');", - new DateTime(2021, 5, 6, 3, 48, 32, DateTimeZone.UTC), - Schema.TypeName.DATETIME); - } - - @Test - public void testTimestampInput() { - runUdfTypeTest( - "SELECT test_timestamp(f_timestamp) FROM table;", - new DateTime(2021, 5, 6, 3, 48, 32, DateTimeZone.UTC), - Schema.TypeName.DATETIME); - } - - @Test - public void testArrayLiteral() { - runUdfTypeTest( - "SELECT test_array(ARRAY[1, 2, 3]);", - ImmutableList.of(1L, 2L, 3L), - Schema.FieldType.array(Schema.FieldType.INT64)); - } - - @Test - public void testArrayInput() { - runUdfTypeTest( - "SELECT test_array(array_int64) FROM table;", - ImmutableList.of(1L, 2L, 3L), - Schema.FieldType.array(Schema.FieldType.INT64)); - } - - @Test - public void testNumericOneLiteral() { - runUdfTypeTest( - "SELECT test_numeric(1.000000000);", - new BigDecimal("1.000000000"), - Schema.FieldType.DECIMAL); - } - - @Test - public void testNumericMaxLiteral() { - runUdfTypeTest( - "SELECT test_numeric(99999999999999999999999999999.999999999);", - new BigDecimal("99999999999999999999999999999.999999999"), - Schema.FieldType.DECIMAL); - } - - @Test - public void testNumericMinLiteral() { - runUdfTypeTest( - "SELECT test_numeric(-99999999999999999999999999999.999999999);", - new BigDecimal("-99999999999999999999999999999.999999999"), - Schema.FieldType.DECIMAL); - } - - @Test - public void testNumericOneInput() { - runUdfTypeTest( - "SELECT test_numeric(numeric_one) FROM table;", - new BigDecimal("1.000000000"), - Schema.FieldType.DECIMAL); - } - - @Test - public void testNumericMaxInput() { - runUdfTypeTest( - "SELECT test_numeric(numeric_max) FROM table;", - new BigDecimal("99999999999999999999999999999.999999999"), - Schema.FieldType.DECIMAL); - } - - @Test - public void testNumericMinInput() { - runUdfTypeTest( - "SELECT test_numeric(numeric_min) FROM table;", - new BigDecimal("-99999999999999999999999999999.999999999"), - Schema.FieldType.DECIMAL); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlMathFunctionsTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlMathFunctionsTest.java deleted file mode 100644 index 92e509c00d30..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlMathFunctionsTest.java +++ /dev/null @@ -1,1034 +0,0 @@ -/* - * 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 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; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.joda.time.Duration; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for ZetaSQL Math functions (on INT64, DOUBLE, NUMERIC types). */ -@RunWith(JUnit4.class) -public class ZetaSqlMathFunctionsTest extends ZetaSqlTestBase { - - @Rule public transient TestPipeline pipeline = TestPipeline.create(); - @Rule public ExpectedException thrown = ExpectedException.none(); - - @Before - public void setUp() { - initialize(); - } - - ///////////////////////////////////////////////////////////////////////////// - // INT64 type tests - ///////////////////////////////////////////////////////////////////////////// - - @Test - public void testArithmeticOperatorsInt64() { - String sql = "SELECT -1, 1 + 2, 1 - 2, 1 * 2, 1 / 2"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addInt64Field("f_int64_1") - .addInt64Field("f_int64_2") - .addInt64Field("f_int64_3") - .addInt64Field("f_int64_4") - .addDoubleField("f_double") - .build()) - .addValues(-1L, 3L, -1L, 2L, 0.5) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testAbsInt64() { - String sql = "SELECT ABS(1), ABS(-1)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addInt64Field("f_int64_1").addInt64Field("f_int64_2").build()) - .addValues(1L, 1L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testSignInt64() { - String sql = "SELECT SIGN(0), SIGN(5), SIGN(-5)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addInt64Field("f_int64_1") - .addInt64Field("f_int64_2") - .addInt64Field("f_int64_3") - .build()) - .addValues(0L, 1L, -1L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testModInt64() { - String sql = "SELECT MOD(4, 2)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_int64").build()) - .addValues(0L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDivInt64() { - String sql = "SELECT DIV(1, 2), DIV(2, 1)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addInt64Field("f_int64_1").addInt64Field("f_int64_2").build()) - .addValues(0L, 2L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testSafeArithmeticFunctionsInt64() { - String sql = - "SELECT SAFE_ADD(9223372036854775807, 1), " - + "SAFE_SUBTRACT(-9223372036854775808, 1), " - + "SAFE_MULTIPLY(9223372036854775807, 2), " - + "SAFE_DIVIDE(1, 0), " - + "SAFE_NEGATE(-9223372036854775808)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addNullableField("f_int64_1", Schema.FieldType.INT64) - .addNullableField("f_int64_2", Schema.FieldType.INT64) - .addNullableField("f_int64_3", Schema.FieldType.INT64) - .addNullableField("f_int64_4", Schema.FieldType.INT64) - .addNullableField("f_int64_5", Schema.FieldType.INT64) - .build()) - .addValues(null, null, null, null, null) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - ///////////////////////////////////////////////////////////////////////////// - // DOUBLE (FLOAT64) type tests - ///////////////////////////////////////////////////////////////////////////// - - @Test - public void testDoubleLiteral() { - String sql = - "SELECT 3.0, CAST('+inf' AS FLOAT64), CAST('-inf' AS FLOAT64), CAST('NaN' AS FLOAT64)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDoubleField("f_double1") - .addDoubleField("f_double2") - .addDoubleField("f_double3") - .addDoubleField("f_double4") - .build()) - .addValues(3.0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, Double.NaN) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testArithmeticOperatorsDouble() { - String sql = "SELECT -1.5, 1.5 + 2.5, 1.5 - 2.5, 1.5 * 2.5, 1.5 / 2.5"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDoubleField("f_double1") - .addDoubleField("f_double2") - .addDoubleField("f_double3") - .addDoubleField("f_double4") - .addDoubleField("f_double5") - .build()) - .addValues(-1.5, 4.0, -1.0, 3.75, 0.6) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testEqualsInf() { - String sql = - "SELECT CAST('+inf' AS FLOAT64) = CAST('+inf' AS FLOAT64), " - + "CAST('+inf' AS FLOAT64) = CAST('-inf' AS FLOAT64)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addBooleanField("f_boolean1") - .addBooleanField("f_boolean2") - .build()) - .addValues(true, false) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testEqualsNaN() { - String sql = "SELECT CAST('NaN' AS FLOAT64) = CAST('NaN' AS FLOAT64)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addBooleanField("f_boolean").build()) - .addValues(false) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testAbsDouble() { - String sql = "SELECT ABS(1.5), ABS(-1.0), ABS(CAST('NaN' AS FLOAT64))"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDoubleField("f_double1") - .addDoubleField("f_double2") - .addDoubleField("f_double3") - .build()) - .addValues(1.5, 1.0, Double.NaN) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testSignDouble() { - String sql = "SELECT SIGN(-0.0), SIGN(1.5), SIGN(-1.5), SIGN(CAST('NaN' AS FLOAT64))"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDoubleField("f_double1") - .addDoubleField("f_double2") - .addDoubleField("f_double3") - .addDoubleField("f_double4") - .build()) - .addValues(0.0, 1.0, -1.0, Double.NaN) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testRoundDouble() { - String sql = "SELECT ROUND(1.23), ROUND(-1.27, 1)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDoubleField("f_double1") - .addDoubleField("f_double2") - .build()) - .addValues(1.0, -1.3) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTruncDouble() { - String sql = "SELECT TRUNC(1.23), TRUNC(-1.27, 1)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDoubleField("f_double1") - .addDoubleField("f_double2") - .build()) - .addValues(1.0, -1.2) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testCeilDouble() { - String sql = "SELECT CEIL(1.2), CEIL(-1.2)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDoubleField("f_double1") - .addDoubleField("f_double2") - .build()) - .addValues(2.0, -1.0) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testFloorDouble() { - String sql = "SELECT FLOOR(1.2), FLOOR(-1.2)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDoubleField("f_double1") - .addDoubleField("f_double2") - .build()) - .addValues(1.0, -2.0) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testIsInf() { - String sql = - "SELECT IS_INF(CAST('+inf' AS FLOAT64)), IS_INF(CAST('-inf' AS FLOAT64)), IS_INF(3.0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addBooleanField("f_boolean1") - .addBooleanField("f_boolean2") - .addBooleanField("f_boolean3") - .build()) - .addValues(true, true, false) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testIsNaN() { - String sql = "SELECT IS_NAN(CAST('NaN' AS FLOAT64)), IS_NAN(3.0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addBooleanField("f_boolean1") - .addBooleanField("f_boolean2") - .build()) - .addValues(true, false) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testIeeeDivide() { - String sql = "SELECT IEEE_DIVIDE(1.0, 0.0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDoubleField("f_double").build()) - .addValues(Double.POSITIVE_INFINITY) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testSafeDivide() { - String sql = "SELECT SAFE_DIVIDE(1.0, 0.0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addNullableField("f_double", Schema.FieldType.DOUBLE).build()) - .addValue(null) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testSqrtDouble() { - String sql = "SELECT SQRT(4.0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDoubleField("f_double").build()) - .addValues(2.0) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testPowDouble() { - String sql = "SELECT POW(2.0, 3.0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDoubleField("f_double").build()) - .addValues(8.0) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testExpDouble() { - String sql = "SELECT EXP(2.0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDoubleField("f_double").build()) - .addValues(7.38905609893065) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testLnDouble() { - String sql = "SELECT LN(7.38905609893065)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDoubleField("f_double").build()) - .addValues(2.0) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testLog10Double() { - String sql = "SELECT LOG10(100.0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDoubleField("f_double").build()) - .addValues(2.0) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testLogDouble() { - String sql = "SELECT LOG(2.25, 1.5)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDoubleField("f_double").build()) - .addValues(2.0) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTrigonometricFunctions() { - String sql = - "SELECT COS(0.0), COSH(0.0), ACOS(1.0), ACOSH(1.0), " - + "SIN(0.0), SINH(0.0), ASIN(0.0), ASINH(0.0), " - + "TAN(0.0), TANH(0.0), ATAN(0.0), ATANH(0.0), ATAN2(0.0, 0.0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDoubleField("f_double1") - .addDoubleField("f_double2") - .addDoubleField("f_double3") - .addDoubleField("f_double4") - .addDoubleField("f_double5") - .addDoubleField("f_double6") - .addDoubleField("f_double7") - .addDoubleField("f_double8") - .addDoubleField("f_double9") - .addDoubleField("f_double10") - .addDoubleField("f_double11") - .addDoubleField("f_double12") - .addDoubleField("f_double13") - .build()) - .addValues(1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - ///////////////////////////////////////////////////////////////////////////// - // NUMERIC type tests - ///////////////////////////////////////////////////////////////////////////// - - @Test - public void testNumericLiteral() { - String sql = - "SELECT NUMERIC '0', " - + "NUMERIC '123456', " - + "NUMERIC '-3.14', " - + "NUMERIC '-0.54321', " - + "NUMERIC '1.23456e05', " - + "NUMERIC '-9.876e-3', " - // min value for ZetaSQL NUMERIC type - + "NUMERIC '-99999999999999999999999999999.999999999', " - // max value for ZetaSQL NUMERIC type - + "NUMERIC '99999999999999999999999999999.999999999'"; - ; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDecimalField("f_numeric1") - .addDecimalField("f_numeric2") - .addDecimalField("f_numeric3") - .addDecimalField("f_numeric4") - .addDecimalField("f_numeric5") - .addDecimalField("f_numeric6") - .addDecimalField("f_numeric7") - .addDecimalField("f_numeric8") - .build()) - .addValues( - ZetaSqlTypesUtils.bigDecimalAsNumeric("0"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("123456"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("-3.14"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("-0.54321"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("123456"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("-0.009876"), - ZetaSqlCalciteTranslationUtils.ZETASQL_NUMERIC_MIN_VALUE, - ZetaSqlCalciteTranslationUtils.ZETASQL_NUMERIC_MAX_VALUE) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testNumericColumn() { - String sql = "SELECT numeric_field FROM table_with_numeric"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = Schema.builder().addDecimalField("f_numeric").build(); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("123.4567")) - .build(), - Row.withSchema(schema) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("765.4321")) - .build(), - Row.withSchema(schema) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-555.5555")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testArithmeticOperatorsNumeric() { - String sql = - "SELECT - NUMERIC '1.23456e05', " - + "NUMERIC '1.23456e05' + NUMERIC '9.876e-3', " - + "NUMERIC '1.23456e05' - NUMERIC '-9.876e-3', " - + "NUMERIC '1.23e02' * NUMERIC '-1.001e-3', " - + "NUMERIC '-1.23123e-1' / NUMERIC '-1.001e-3', "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDecimalField("f_numeric1") - .addDecimalField("f_numeric2") - .addDecimalField("f_numeric3") - .addDecimalField("f_numeric4") - .addDecimalField("f_numeric5") - .build()) - .addValues( - ZetaSqlTypesUtils.bigDecimalAsNumeric("-123456"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("123456.009876"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("123456.009876"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("-0.123123"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("123")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testAbsNumeric() { - String sql = "SELECT ABS(NUMERIC '1.23456e04'), ABS(NUMERIC '-1.23456e04')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDecimalField("f_numeric1") - .addDecimalField("f_numeric2") - .build()) - .addValues( - ZetaSqlTypesUtils.bigDecimalAsNumeric("12345.6"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("12345.6")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testSignNumeric() { - String sql = "SELECT SIGN(NUMERIC '0'), SIGN(NUMERIC '1.23e01'), SIGN(NUMERIC '-1.23e01')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDecimalField("f_numeric1") - .addDecimalField("f_numeric2") - .addDecimalField("f_numeric3") - .build()) - .addValues( - ZetaSqlTypesUtils.bigDecimalAsNumeric("0"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("1"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("-1")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testRoundNumeric() { - String sql = "SELECT ROUND(NUMERIC '1.23456e04'), ROUND(NUMERIC '-1.234567e04', 1)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDecimalField("f_numeric1") - .addDecimalField("f_numeric2") - .build()) - .addValues( - ZetaSqlTypesUtils.bigDecimalAsNumeric("12346"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("-12345.7")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTruncNumeric() { - String sql = "SELECT TRUNC(NUMERIC '1.23456e04'), TRUNC(NUMERIC '-1.234567e04', 1)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDecimalField("f_numeric1") - .addDecimalField("f_numeric2") - .build()) - .addValues( - ZetaSqlTypesUtils.bigDecimalAsNumeric("12345"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("-12345.6")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testCeilNumeric() { - String sql = "SELECT CEIL(NUMERIC '1.23456e04'), CEIL(NUMERIC '-1.23456e04')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDecimalField("f_numeric1") - .addDecimalField("f_numeric2") - .build()) - .addValues( - ZetaSqlTypesUtils.bigDecimalAsNumeric("12346"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("-12345")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testFloorNumeric() { - String sql = "SELECT FLOOR(NUMERIC '1.23456e04'), FLOOR(NUMERIC '-1.23456e04')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDecimalField("f_numeric1") - .addDecimalField("f_numeric2") - .build()) - .addValues( - ZetaSqlTypesUtils.bigDecimalAsNumeric("12345"), - ZetaSqlTypesUtils.bigDecimalAsNumeric("-12346")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testModNumeric() { - String sql = "SELECT MOD(NUMERIC '1.23456e05', NUMERIC '5')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("1")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDivNumeric() { - String sql = "SELECT DIV(NUMERIC '1.23456e05', NUMERIC '5')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("24691")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testSafeArithmeticFunctionsNumeric() { - String sql = - "SELECT SAFE_ADD(NUMERIC '99999999999999999999999999999.999999999', NUMERIC '1'), " - + "SAFE_SUBTRACT(NUMERIC '-99999999999999999999999999999.999999999', NUMERIC '1'), " - + "SAFE_MULTIPLY(NUMERIC '99999999999999999999999999999.999999999', NUMERIC '2'), " - + "SAFE_DIVIDE(NUMERIC '1.23456e05', NUMERIC '0'), " - + "SAFE_NEGATE(NUMERIC '99999999999999999999999999999.999999999')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addNullableField("f_numeric1", Schema.FieldType.DECIMAL) - .addNullableField("f_numeric2", Schema.FieldType.DECIMAL) - .addNullableField("f_numeric3", Schema.FieldType.DECIMAL) - .addNullableField("f_numeric4", Schema.FieldType.DECIMAL) - .addNullableField("f_numeric5", Schema.FieldType.DECIMAL) - .build()) - .addValues( - null, - null, - null, - null, - ZetaSqlCalciteTranslationUtils.ZETASQL_NUMERIC_MIN_VALUE) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testSqrtNumeric() { - String sql = "SELECT SQRT(NUMERIC '4')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("2")) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testPowNumeric() { - String sql = "SELECT POW(NUMERIC '2', NUMERIC '3')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("8")) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testExpNumeric() { - String sql = "SELECT EXP(NUMERIC '2')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("7.389056099")) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testLnNumeric() { - String sql = "SELECT LN(NUMERIC '7.389056099')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("2")) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testLog10Numeric() { - String sql = "SELECT LOG10(NUMERIC '100')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("2")) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testLogNumeric() { - String sql = "SELECT LOG(NUMERIC '2.25', NUMERIC '1.5')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("2")) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testSumNumeric() { - String sql = "SELECT SUM(numeric_field) FROM table_with_numeric"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("333.3333")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testAvgNumeric() { - String sql = "SELECT AVG(numeric_field) FROM table_with_numeric"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDecimalField("f_numeric").build()) - .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("111.1111")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } -} 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 deleted file mode 100644 index 9f37472cfae1..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNativeUdfTest.java +++ /dev/null @@ -1,264 +0,0 @@ -/* - * 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 static org.hamcrest.Matchers.isA; - -import com.google.zetasql.SqlException; -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; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.joda.time.Duration; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for SQL-native user defined functions in the ZetaSQL dialect. */ -@RunWith(JUnit4.class) -public class ZetaSqlNativeUdfTest extends ZetaSqlTestBase { - @Rule public transient TestPipeline pipeline = TestPipeline.create(); - @Rule public ExpectedException thrown = ExpectedException.none(); - - @Before - public void setUp() { - initialize(); - } - - @Test - public void testAlreadyDefinedUDFThrowsException() { - String sql = "CREATE FUNCTION foo() AS (0); CREATE FUNCTION foo() AS (1); SELECT foo();"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(RuntimeException.class); - thrown.expectMessage("Failed to define function 'foo'"); - thrown.expectCause(isA(IllegalArgumentException.class)); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testCreateFunctionNoSelectThrowsException() { - String sql = "CREATE FUNCTION plusOne(x INT64) AS (x + 1);"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage("Statement list must end in a SELECT statement, not CreateFunctionStmt"); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testNullaryUdf() { - String sql = "CREATE FUNCTION zero() AS (0); SELECT zero();"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("x").build()).addValue(0L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testQualifiedNameUdfUnqualifiedCall() { - String sql = "CREATE FUNCTION foo.bar.baz() AS (\"uwu\"); SELECT baz();"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("x").build()).addValue("uwu").build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - @Ignore( - "Qualified paths can't be resolved due to a bug in ZetaSQL: " - + "https://github.com/google/zetasql/issues/42") - public void testQualifiedNameUdfQualifiedCallThrowsException() { - String sql = "CREATE FUNCTION foo.bar.baz() AS (\"uwu\"); SELECT foo.bar.baz();"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("x").build()).addValue("uwu").build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testUnaryUdf() { - String sql = "CREATE FUNCTION triple(x INT64) AS (3 * x); SELECT triple(triple(1));"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("x").build()).addValue(9L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testUdfWithinUdf() { - String sql = - "CREATE FUNCTION triple(x INT64) AS (3 * x);" - + " CREATE FUNCTION nonuple(x INT64) as (triple(triple(x)));" - + " SELECT nonuple(1);"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("x").build()).addValue(9L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testUndefinedUdfThrowsException() { - String sql = - "CREATE FUNCTION foo() AS (bar()); " - + "CREATE FUNCTION bar() AS (foo()); " - + "SELECT foo();"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(SqlException.class); - thrown.expectMessage("Function not found: bar"); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testRecursiveUdfThrowsException() { - String sql = "CREATE FUNCTION omega() AS (omega()); SELECT omega();"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(SqlException.class); - thrown.expectMessage("Function not found: omega"); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testUDTVF() { - String sql = - "CREATE TABLE FUNCTION CustomerRange(MinID INT64, MaxID INT64)\n" - + " AS\n" - + " SELECT *\n" - + " FROM KeyValue\n" - + " WHERE key >= MinId AND key <= MaxId; \n" - + " SELECT key FROM CustomerRange(10, 14)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema singleField = Schema.builder().addInt64Field("field1").build(); - PAssert.that(stream).containsInAnyOrder(Row.withSchema(singleField).addValues(14L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testNullaryUdtvf() { - String sql = - "CREATE TABLE FUNCTION CustomerRange()\n" - + " AS\n" - + " SELECT *\n" - + " FROM KeyValue;\n" - + " SELECT key FROM CustomerRange()"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - Schema singleField = Schema.builder().addInt64Field("field1").build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(singleField).addValues(14L).build(), - Row.withSchema(singleField).addValues(15L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testUDTVFTableNotFound() { - String sql = - "CREATE TABLE FUNCTION CustomerRange(MinID INT64, MaxID INT64)\n" - + " AS\n" - + " SELECT *\n" - + " FROM TableNotExist\n" - + " WHERE key >= MinId AND key <= MaxId; \n" - + " SELECT key FROM CustomerRange(10, 14)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(ZetaSqlException.class); - thrown.expectMessage("Wasn't able to resolve the path [TableNotExist] in schema: beam"); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testUDTVFFunctionNotFound() { - String sql = - "CREATE TABLE FUNCTION CustomerRange(MinID INT64, MaxID INT64)\n" - + " AS\n" - + " SELECT *\n" - + " FROM KeyValue\n" - + " WHERE key >= MinId AND key <= MaxId; \n" - + " SELECT key FROM FunctionNotFound(10, 14)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(SqlException.class); - thrown.expectMessage("Table-valued function not found: FunctionNotFound"); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testJavascriptUdfUnsupported() { - String sql = "CREATE FUNCTION foo() RETURNS STRING LANGUAGE js; SELECT foo();"; - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage("Function foo uses unsupported language js."); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testSqlNativeAggregateFunctionNotSupported() { - String sql = - "CREATE AGGREGATE FUNCTION double_sum(col FLOAT64)\n" - + "AS (2 * SUM(col));\n" - + "SELECT double_sum(col1) AS doubled_sum\n" - + "FROM (SELECT 1 AS col1 UNION ALL\n" - + " SELECT 3 AS col1 UNION ALL\n" - + " SELECT 5 AS col1\n" - + ");"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage( - "Native SQL aggregate functions are not supported (https://github.com/apache/beam/issues/20193)."); - 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 deleted file mode 100644 index a3178c3031c2..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlNumberTypesTest.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.Value; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for ZetaSQL number type handling (on INT64, DOUBLE, NUMERIC types). */ -@RunWith(JUnit4.class) -public class ZetaSqlNumberTypesTest extends ZetaSqlTestBase { - @Rule public transient TestPipeline pipeline = TestPipeline.create(); - @Rule public ExpectedException thrown = ExpectedException.none(); - - @Before - public void setUp() { - initialize(); - } - - @Test - public void testCastDoubleAsNumericOverflow() { - double val = 1.7976931348623157e+308; - String sql = "SELECT CAST(@p0 AS NUMERIC) AS ColA"; - - thrown.expect(ZetaSqlException.class); - thrown.expectMessage("Casting TYPE_DOUBLE as TYPE_NUMERIC would cause overflow of literal"); - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - zetaSQLQueryPlanner.convertToBeamRel(sql, ImmutableMap.of("p0", Value.createDoubleValue(val))); - } - - @Test - public void testCastDoubleAsNumericUnderflow() { - double val = -1.7976931348623157e+308; - String sql = "SELECT CAST(@p0 AS NUMERIC) AS ColA"; - - thrown.expect(ZetaSqlException.class); - thrown.expectMessage("Casting TYPE_DOUBLE as TYPE_NUMERIC would cause underflow of literal"); - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - zetaSQLQueryPlanner.convertToBeamRel(sql, ImmutableMap.of("p0", Value.createDoubleValue(val))); - } - - @Test - public void testCastDoubleAsNumericScaleTooLarge() { - double val = 2.2250738585072014e-308; - String sql = "SELECT CAST(@p0 AS NUMERIC) AS ColA"; - - thrown.expect(ZetaSqlException.class); - thrown.expectMessage("Cannot cast TYPE_DOUBLE as TYPE_NUMERIC: scale 1022 exceeds 9"); - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - zetaSQLQueryPlanner.convertToBeamRel(sql, ImmutableMap.of("p0", Value.createDoubleValue(val))); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java deleted file mode 100644 index 5b67ad76fd9d..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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 java.util.HashMap; -import java.util.Map; -import org.apache.beam.sdk.extensions.sql.impl.JdbcConnection; -import org.apache.beam.sdk.extensions.sql.impl.JdbcDriver; -import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel; -import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; -import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider; -import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.Contexts; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.ConventionTraitDef; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.FrameworkConfig; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.Frameworks; -import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.tools.RuleSet; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; - -/** Common setup for ZetaSQL tests. */ -public abstract class ZetaSqlTestBase { - protected static final long PIPELINE_EXECUTION_WAITTIME_MINUTES = 2L; - - protected FrameworkConfig config; - - private TableProvider createBeamTableProvider() { - Map testBoundedTableMap = new HashMap<>(); - testBoundedTableMap.put("KeyValue", TestInput.BASIC_TABLE_ONE); - testBoundedTableMap.put("BigTable", TestInput.BASIC_TABLE_TWO); - testBoundedTableMap.put("Spanner", TestInput.BASIC_TABLE_THREE); - testBoundedTableMap.put("aggregate_test_table", TestInput.AGGREGATE_TABLE_ONE); - testBoundedTableMap.put("window_test_table", TestInput.TIMESTAMP_TABLE_ONE); - testBoundedTableMap.put("window_test_table_two", TestInput.TIMESTAMP_TABLE_TWO); - testBoundedTableMap.put("all_null_table", TestInput.TABLE_ALL_NULL); - testBoundedTableMap.put("table_with_struct", TestInput.TABLE_WITH_STRUCT); - testBoundedTableMap.put("table_with_struct_two", TestInput.TABLE_WITH_STRUCT_TWO); - testBoundedTableMap.put("table_with_array", TestInput.TABLE_WITH_ARRAY); - testBoundedTableMap.put("table_with_array_for_unnest", TestInput.TABLE_WITH_ARRAY_FOR_UNNEST); - testBoundedTableMap.put("table_with_array_of_struct", TestInput.TABLE_WITH_ARRAY_OF_STRUCT); - testBoundedTableMap.put("table_with_struct_of_struct", TestInput.TABLE_WITH_STRUCT_OF_STRUCT); - testBoundedTableMap.put( - "table_with_struct_of_struct_of_array", TestInput.TABLE_WITH_STRUCT_OF_STRUCT_OF_ARRAY); - testBoundedTableMap.put( - "table_with_array_of_struct_of_struct", TestInput.TABLE_WITH_ARRAY_OF_STRUCT_OF_STRUCT); - testBoundedTableMap.put( - "table_with_struct_of_array_of_struct", TestInput.TABLE_WITH_STRUCT_OF_ARRAY_OF_STRUCT); - testBoundedTableMap.put( - "table_with_array_of_struct_of_array", TestInput.TABLE_WITH_ARRAY_OF_STRUCT_OF_ARRAY); - testBoundedTableMap.put("table_for_case_when", TestInput.TABLE_FOR_CASE_WHEN); - testBoundedTableMap.put("aggregate_test_table_two", TestInput.AGGREGATE_TABLE_TWO); - testBoundedTableMap.put("table_empty", TestInput.TABLE_EMPTY); - testBoundedTableMap.put("table_all_types", TestInput.TABLE_ALL_TYPES); - testBoundedTableMap.put("table_all_types_2", TestInput.TABLE_ALL_TYPES_2); - testBoundedTableMap.put("table_with_map", TestInput.TABLE_WITH_MAP); - testBoundedTableMap.put("table_with_date", TestInput.TABLE_WITH_DATE); - testBoundedTableMap.put("table_with_time", TestInput.TABLE_WITH_TIME); - testBoundedTableMap.put("table_with_numeric", TestInput.TABLE_WITH_NUMERIC); - testBoundedTableMap.put("table_with_datetime", TestInput.TABLE_WITH_DATETIME); - testBoundedTableMap.put( - "table_with_struct_ts_string", TestInput.TABLE_WITH_STRUCT_TIMESTAMP_STRING); - testBoundedTableMap.put("streaming_sql_test_table_a", TestInput.STREAMING_SQL_TABLE_A); - testBoundedTableMap.put("streaming_sql_test_table_b", TestInput.STREAMING_SQL_TABLE_B); - - return new ReadOnlyTableProvider("test_table_provider", testBoundedTableMap); - } - - protected void initialize() { - JdbcConnection jdbcConnection = - JdbcDriver.connect(createBeamTableProvider(), PipelineOptionsFactory.create()); - - this.config = - Frameworks.newConfigBuilder() - .defaultSchema(jdbcConnection.getCurrentSchemaPlus()) - .traitDefs(ImmutableList.of(ConventionTraitDef.INSTANCE)) - .context(Contexts.of(jdbcConnection.config())) - .ruleSets(ZetaSQLQueryPlanner.getZetaSqlRuleSets().toArray(new RuleSet[0])) - .costFactory(BeamCostModel.FACTORY) - .typeSystem(jdbcConnection.getTypeFactory().getTypeSystem()) - .build(); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java deleted file mode 100644 index cfc0fce737bc..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java +++ /dev/null @@ -1,1779 +0,0 @@ -/* - * 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 static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.parseDateToValue; -import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.parseTimeToValue; -import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.parseTimestampWithTZToValue; -import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.parseTimestampWithTimeZone; -import static org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils.parseTimestampWithUTCTimeZone; - -import com.google.zetasql.Value; -import com.google.zetasql.ZetaSQLType.TypeKind; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import org.apache.beam.sdk.extensions.sql.impl.BeamSqlPipelineOptions; -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; -import org.apache.beam.sdk.schemas.Schema.FieldType; -import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.joda.time.Duration; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Tests for ZetaSQL time functions (DATE, TIME, DATETIME, and TIMESTAMP functions). */ -@RunWith(JUnit4.class) -public class ZetaSqlTimeFunctionsTest extends ZetaSqlTestBase { - - @Rule public transient TestPipeline pipeline = TestPipeline.create(); - @Rule public ExpectedException thrown = ExpectedException.none(); - - @Before - public void setUp() { - initialize(); - } - - ///////////////////////////////////////////////////////////////////////////// - // DATE type tests - ///////////////////////////////////////////////////////////////////////////// - - @Test - public void testDateLiteral() { - String sql = "SELECT DATE '2020-3-30'"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_date", SqlTypes.DATE).build()) - .addValues(LocalDate.of(2020, 3, 30)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateColumn() { - // NOTE: Do not use textual format parameters (%b or %h: The abbreviated month name) as these - // are locale dependent. - String sql = "SELECT FORMAT_DATE('%m-%d-%Y', date_field) FROM table_with_date"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("f_date_str").build()) - .addValues("12-25-2008") - .build(), - Row.withSchema(Schema.builder().addStringField("f_date_str").build()) - .addValues("04-07-2020") - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testGroupByDate() { - String sql = "SELECT date_field, COUNT(*) FROM table_with_date GROUP BY date_field"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = - Schema.builder() - .addLogicalTypeField("date_field", SqlTypes.DATE) - .addInt64Field("count") - .build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema).addValues(LocalDate.of(2008, 12, 25), 1L).build(), - Row.withSchema(schema).addValues(LocalDate.of(2020, 4, 7), 1L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testAggregateOnDate() { - String sql = "SELECT MAX(date_field) FROM table_with_date GROUP BY str_field"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("date_field", SqlTypes.DATE).build()) - .addValues(LocalDate.of(2020, 4, 7)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - // TODO[https://github.com/apache/beam/issues/19980]: Add a test for CURRENT_DATE function - // ("SELECT CURRENT_DATE()") - - @Test - public void testExtractFromDate() { - String sql = - "WITH Dates AS (\n" - + " SELECT DATE '2015-12-31' AS date UNION ALL\n" - + " SELECT DATE '2016-01-01'\n" - + ")\n" - + "SELECT\n" - + " EXTRACT(ISOYEAR FROM date) AS isoyear,\n" - + " EXTRACT(YEAR FROM date) AS year,\n" - + " EXTRACT(ISOWEEK FROM date) AS isoweek,\n" - // TODO[https://github.com/apache/beam/issues/20338]: Add tests for DATE_TRUNC and - // EXTRACT with "week with weekday" date - // parts once they are supported - // + " EXTRACT(WEEK FROM date) AS week,\n" - + " EXTRACT(MONTH FROM date) AS month,\n" - + " EXTRACT(QUARTER FROM date) AS quarter,\n" - + " EXTRACT(DAY FROM date) AS day,\n" - + " EXTRACT(DAYOFYEAR FROM date) AS dayofyear,\n" - + " EXTRACT(DAYOFWEEK FROM date) AS dayofweek\n" - + "FROM Dates"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = - Schema.builder() - .addInt64Field("isoyear") - .addInt64Field("year") - .addInt64Field("isoweek") - // .addInt64Field("week") - .addInt64Field("month") - .addInt64Field("quarter") - .addInt64Field("day") - .addInt64Field("dayofyear") - .addInt64Field("dayofweek") - .build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema) - .addValues(2015L, 2015L, 53L /* , 52L */, 12L, 4L, 31L, 365L, 5L) - .build(), - Row.withSchema(schema) - .addValues(2015L, 2016L, 53L /* , 0L */, 1L, 1L, 1L, 1L, 6L) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateFromYearMonthDay() { - String sql = "SELECT DATE(2008, 12, 25)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_date", SqlTypes.DATE).build()) - .addValues(LocalDate.of(2008, 12, 25)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateFromTimestamp() { - String sql = "SELECT DATE(TIMESTAMP '2016-12-25 05:30:00+07', 'America/Los_Angeles')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_date", SqlTypes.DATE).build()) - .addValues(LocalDate.of(2016, 12, 24)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateFromDateTime() { - String sql = "SELECT DATE(DATETIME '2008-12-25 15:30:00.123456')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_date", SqlTypes.DATE).build()) - .addValues(LocalDate.of(2008, 12, 25)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateAdd() { - String sql = - "SELECT " - + "DATE_ADD(DATE '2008-12-25', INTERVAL 5 DAY), " - + "DATE_ADD(DATE '2008-12-25', INTERVAL 1 MONTH), " - + "DATE_ADD(DATE '2008-12-25', INTERVAL 1 YEAR), "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addLogicalTypeField("f_date1", SqlTypes.DATE) - .addLogicalTypeField("f_date2", SqlTypes.DATE) - .addLogicalTypeField("f_date3", SqlTypes.DATE) - .build()) - .addValues( - LocalDate.of(2008, 12, 30), - LocalDate.of(2009, 1, 25), - LocalDate.of(2009, 12, 25)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateAddWithParameter() { - String sql = - "SELECT " - + "DATE_ADD(@p0, INTERVAL @p1 DAY), " - + "DATE_ADD(@p2, INTERVAL @p3 DAY), " - + "DATE_ADD(@p4, INTERVAL @p5 YEAR), " - + "DATE_ADD(@p6, INTERVAL @p7 DAY), " - + "DATE_ADD(@p8, INTERVAL @p9 MONTH)"; - - ImmutableMap params = - ImmutableMap.builder() - .put("p0", Value.createDateValue(0)) // 1970-01-01 - .put("p1", Value.createInt64Value(2L)) - .put("p2", parseDateToValue("2019-01-01")) - .put("p3", Value.createInt64Value(2L)) - .put("p4", Value.createSimpleNullValue(TypeKind.TYPE_DATE)) - .put("p5", Value.createInt64Value(1L)) - .put("p6", parseDateToValue("2000-02-29")) - .put("p7", Value.createInt64Value(-365L)) - .put("p8", parseDateToValue("1999-03-31")) - .put("p9", Value.createInt64Value(-1L)) - .build(); - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, params); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = - Schema.builder() - .addLogicalTypeField("f_date1", SqlTypes.DATE) - .addLogicalTypeField("f_date2", SqlTypes.DATE) - .addNullableField("f_date3", FieldType.logicalType(SqlTypes.DATE)) - .addLogicalTypeField("f_date4", SqlTypes.DATE) - .addLogicalTypeField("f_date5", SqlTypes.DATE) - .build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema) - .addValues( - LocalDate.of(1970, 1, 3), - LocalDate.of(2019, 1, 3), - null, - LocalDate.of(1999, 3, 1), - LocalDate.of(1999, 2, 28)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateSub() { - String sql = - "SELECT " - + "DATE_SUB(DATE '2008-12-25', INTERVAL 5 DAY), " - + "DATE_SUB(DATE '2008-12-25', INTERVAL 1 MONTH), " - + "DATE_SUB(DATE '2008-12-25', INTERVAL 1 YEAR), "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addLogicalTypeField("f_date1", SqlTypes.DATE) - .addLogicalTypeField("f_date2", SqlTypes.DATE) - .addLogicalTypeField("f_date3", SqlTypes.DATE) - .build()) - .addValues( - LocalDate.of(2008, 12, 20), - LocalDate.of(2008, 11, 25), - LocalDate.of(2007, 12, 25)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateDiff() { - String sql = "SELECT DATE_DIFF(DATE '2010-07-07', DATE '2008-12-25', DAY)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_date_diff").build()) - .addValues(559L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateDiffNegativeResult() { - String sql = "SELECT DATE_DIFF(DATE '2017-12-17', DATE '2017-12-18', ISOWEEK)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_date_diff").build()) - .addValues(-1L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTrunc() { - String sql = "SELECT DATE_TRUNC(DATE '2015-06-15', ISOYEAR)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("f_date_trunc", SqlTypes.DATE).build()) - .addValues(LocalDate.of(2014, 12, 29)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testFormatDate() { - // NOTE: Do not use textual format parameters (%b or %h: The abbreviated month name) as these - // are locale dependent. - String sql = "SELECT FORMAT_DATE('%m-%d-%Y', DATE '2008-12-25')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("f_date_str").build()) - .addValues("12-25-2008") - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testParseDate() { - String sql = "SELECT PARSE_DATE('%m %d %y', '10 14 18')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_date", SqlTypes.DATE).build()) - .addValues(LocalDate.of(2018, 10, 14)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateToUnixInt64() { - String sql = "SELECT UNIX_DATE(DATE '2008-12-25')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_unix_date").build()) - .addValues(14238L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateFromUnixInt64() { - String sql = "SELECT DATE_FROM_UNIX_DATE(14238)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_date", SqlTypes.DATE).build()) - .addValues(LocalDate.of(2008, 12, 25)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - ///////////////////////////////////////////////////////////////////////////// - // TIME type tests - ///////////////////////////////////////////////////////////////////////////// - - @Test - public void testTimeLiteral() { - String sql = "SELECT TIME '15:30:00', TIME '15:30:00.135246' "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addLogicalTypeField("f_time1", SqlTypes.TIME) - .addLogicalTypeField("f_time2", SqlTypes.TIME) - .build()) - .addValues(LocalTime.of(15, 30, 0)) - .addValues(LocalTime.of(15, 30, 0, 135246000)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeColumn() { - String sql = "SELECT FORMAT_TIME('%T', time_field) FROM table_with_time"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("f_time_str").build()) - .addValues("15:30:00") - .build(), - Row.withSchema(Schema.builder().addStringField("f_time_str").build()) - .addValues("23:35:59") - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testGroupByTime() { - String sql = "SELECT time_field, COUNT(*) FROM table_with_time GROUP BY time_field"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = - Schema.builder() - .addLogicalTypeField("time_field", SqlTypes.TIME) - .addInt64Field("count") - .build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema).addValues(LocalTime.of(15, 30, 0), 1L).build(), - Row.withSchema(schema).addValues(LocalTime.of(23, 35, 59), 1L).build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testAggregateOnTime() { - String sql = "SELECT MAX(time_field) FROM table_with_time GROUP BY str_field"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("time_field", SqlTypes.TIME).build()) - .addValues(LocalTime.of(23, 35, 59)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - // TODO[https://github.com/apache/beam/issues/19980]: Add a test for CURRENT_TIME function - // ("SELECT CURRENT_TIME()") - - @Test - public void testExtractFromTime() { - String sql = - "SELECT " - + "EXTRACT(HOUR FROM TIME '15:30:35.123456') as hour, " - + "EXTRACT(MINUTE FROM TIME '15:30:35.123456') as minute, " - + "EXTRACT(SECOND FROM TIME '15:30:35.123456') as second, " - + "EXTRACT(MILLISECOND FROM TIME '15:30:35.123456') as millisecond, " - + "EXTRACT(MICROSECOND FROM TIME '15:30:35.123456') as microsecond "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = - Schema.builder() - .addInt64Field("hour") - .addInt64Field("minute") - .addInt64Field("second") - .addInt64Field("millisecond") - .addInt64Field("microsecond") - .build(); - PAssert.that(stream) - .containsInAnyOrder(Row.withSchema(schema).addValues(15L, 30L, 35L, 123L, 123456L).build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeFromHourMinuteSecond() { - String sql = "SELECT TIME(15, 30, 0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_time", SqlTypes.TIME).build()) - .addValues(LocalTime.of(15, 30, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeFromTimestamp() { - String sql = "SELECT TIME(TIMESTAMP '2008-12-25 15:30:00+08', 'America/Los_Angeles')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_time", SqlTypes.TIME).build()) - .addValues(LocalTime.of(23, 30, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeFromDateTime() { - String sql = "SELECT TIME(DATETIME '2008-12-25 15:30:00.123456')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_time", SqlTypes.TIME).build()) - .addValues(LocalTime.of(15, 30, 0, 123456000)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeAdd() { - String sql = - "SELECT " - + "TIME_ADD(TIME '15:30:00', INTERVAL 10 MICROSECOND), " - + "TIME_ADD(TIME '15:30:00', INTERVAL 10 MILLISECOND), " - + "TIME_ADD(TIME '15:30:00', INTERVAL 10 SECOND), " - + "TIME_ADD(TIME '15:30:00', INTERVAL 10 MINUTE), " - + "TIME_ADD(TIME '15:30:00', INTERVAL 10 HOUR) "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addLogicalTypeField("f_time1", SqlTypes.TIME) - .addLogicalTypeField("f_time2", SqlTypes.TIME) - .addLogicalTypeField("f_time3", SqlTypes.TIME) - .addLogicalTypeField("f_time4", SqlTypes.TIME) - .addLogicalTypeField("f_time5", SqlTypes.TIME) - .build()) - .addValues( - LocalTime.of(15, 30, 0, 10000), - LocalTime.of(15, 30, 0, 10000000), - LocalTime.of(15, 30, 10, 0), - LocalTime.of(15, 40, 0, 0), - LocalTime.of(1, 30, 0, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeAddWithParameter() { - String sql = "SELECT TIME_ADD(@p0, INTERVAL @p1 SECOND)"; - ImmutableMap params = - ImmutableMap.of( - "p0", parseTimeToValue("12:13:14.123"), - "p1", Value.createInt64Value(1L)); - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, params); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_time", SqlTypes.TIME).build()) - .addValues(LocalTime.of(12, 13, 15, 123000000)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeSub() { - String sql = - "SELECT " - + "TIME_SUB(TIME '15:30:00', INTERVAL 10 MICROSECOND), " - + "TIME_SUB(TIME '15:30:00', INTERVAL 10 MILLISECOND), " - + "TIME_SUB(TIME '15:30:00', INTERVAL 10 SECOND), " - + "TIME_SUB(TIME '15:30:00', INTERVAL 10 MINUTE), " - + "TIME_SUB(TIME '15:30:00', INTERVAL 10 HOUR) "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addLogicalTypeField("f_time1", SqlTypes.TIME) - .addLogicalTypeField("f_time2", SqlTypes.TIME) - .addLogicalTypeField("f_time3", SqlTypes.TIME) - .addLogicalTypeField("f_time4", SqlTypes.TIME) - .addLogicalTypeField("f_time5", SqlTypes.TIME) - .build()) - .addValues( - LocalTime.of(15, 29, 59, 999990000), - LocalTime.of(15, 29, 59, 990000000), - LocalTime.of(15, 29, 50, 0), - LocalTime.of(15, 20, 0, 0), - LocalTime.of(5, 30, 0, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeDiff() { - String sql = "SELECT TIME_DIFF(TIME '15:30:00', TIME '14:35:00', MINUTE)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_time_diff").build()) - .addValues(55L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeDiffNegativeResult() { - String sql = "SELECT TIME_DIFF(TIME '14:35:00', TIME '15:30:00', MINUTE)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_time_diff").build()) - .addValues(-55L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimeTrunc() { - String sql = "SELECT TIME_TRUNC(TIME '15:30:35', HOUR)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("f_time_trunc", SqlTypes.TIME).build()) - .addValues(LocalTime.of(15, 0, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testFormatTime() { - String sql = "SELECT FORMAT_TIME('%R', TIME '15:30:00')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("f_time_str").build()) - .addValues("15:30") - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testParseTime() { - String sql = "SELECT PARSE_TIME('%H', '15')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("f_time", SqlTypes.TIME).build()) - .addValues(LocalTime.of(15, 0, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - ///////////////////////////////////////////////////////////////////////////// - // DATETIME type tests - ///////////////////////////////////////////////////////////////////////////// - - @Test - public void testDateTimeLiteral() { - String sql = "SELECT DATETIME '2008-12-25 15:30:00.123456'"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build()) - .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeColumn() { - String sql = "SELECT FORMAT_DATETIME('%D %T %E6S', datetime_field) FROM table_with_datetime"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("f_datetime_str").build()) - .addValues("12/25/08 15:30:00 00.123456") - .build(), - Row.withSchema(Schema.builder().addStringField("f_datetime_str").build()) - .addValues("10/06/12 11:45:00 00.987654") - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testGroupByDateTime() { - String sql = "SELECT datetime_field, COUNT(*) FROM table_with_datetime GROUP BY datetime_field"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = - Schema.builder() - .addLogicalTypeField("datetime_field", SqlTypes.DATETIME) - .addInt64Field("count") - .build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema) - .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000), 1L) - .build(), - Row.withSchema(schema) - .addValues(LocalDateTime.of(2012, 10, 6, 11, 45, 0).withNano(987654000), 1L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testAggregateOnDateTime() { - String sql = "SELECT MAX(datetime_field) FROM table_with_datetime GROUP BY str_field"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addLogicalTypeField("datetime_field", SqlTypes.DATETIME) - .build()) - .addValues(LocalDateTime.of(2012, 10, 6, 11, 45, 0).withNano(987654000)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - // TODO[https://github.com/apache/beam/issues/19980]: Add a test for CURRENT_DATETIME function - // ("SELECT CURRENT_DATETIME()") - - @Test - public void testExtractFromDateTime() { - String sql = - "SELECT " - + "EXTRACT(YEAR FROM DATETIME '2008-12-25 15:30:00') as year, " - + "EXTRACT(QUARTER FROM DATETIME '2008-12-25 15:30:00') as quarter, " - + "EXTRACT(MONTH FROM DATETIME '2008-12-25 15:30:00') as month, " - // TODO[https://github.com/apache/beam/issues/20338]: Add tests for DATETIME_TRUNC and - // EXTRACT with "week with weekday" - // date parts once they are supported - // + "EXTRACT(WEEK FROM DATETIME '2008-12-25 15:30:00') as week, " - + "EXTRACT(DAY FROM DATETIME '2008-12-25 15:30:00') as day, " - + "EXTRACT(DAYOFWEEK FROM DATETIME '2008-12-25 15:30:00') as dayofweek, " - + "EXTRACT(DAYOFYEAR FROM DATETIME '2008-12-25 15:30:00') as dayofyear, " - + "EXTRACT(HOUR FROM DATETIME '2008-12-25 15:30:00.123456') as hour, " - + "EXTRACT(MINUTE FROM DATETIME '2008-12-25 15:30:00.123456') as minute, " - + "EXTRACT(SECOND FROM DATETIME '2008-12-25 15:30:00.123456') as second, " - + "EXTRACT(MILLISECOND FROM DATETIME '2008-12-25 15:30:00.123456') as millisecond, " - + "EXTRACT(MICROSECOND FROM DATETIME '2008-12-25 15:30:00.123456') as microsecond, " - + "EXTRACT(DATE FROM DATETIME '2008-12-25 15:30:00.123456') as date, " - + "EXTRACT(TIME FROM DATETIME '2008-12-25 15:30:00.123456') as time "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = - Schema.builder() - .addInt64Field("year") - .addInt64Field("quarter") - .addInt64Field("month") - // .addInt64Field("week") - .addInt64Field("day") - .addInt64Field("dayofweek") - .addInt64Field("dayofyear") - .addInt64Field("hour") - .addInt64Field("minute") - .addInt64Field("second") - .addInt64Field("millisecond") - .addInt64Field("microsecond") - .addLogicalTypeField("date", SqlTypes.DATE) - .addLogicalTypeField("time", SqlTypes.TIME) - .build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema) - .addValues( - 2008L, - 4L, - 12L, - // 52L, - 25L, - 5L, - 360L, - 15L, - 30L, - 0L, - 123L, - 123456L, - LocalDate.of(2008, 12, 25), - LocalTime.of(15, 30, 0, 123456000)) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeFromDateAndTime() { - String sql = "SELECT DATETIME(DATE '2008-12-25', TIME '15:30:00.123456')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build()) - .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeFromDate() { - String sql = "SELECT DATETIME(DATE '2008-12-25')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build()) - .addValues(LocalDateTime.of(2008, 12, 25, 0, 0, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeFromYearMonthDayHourMinuteSecond() { - String sql = "SELECT DATETIME(2008, 12, 25, 15, 30, 0)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build()) - .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeFromTimestamp() { - String sql = "SELECT DATETIME(TIMESTAMP '2008-12-25 15:30:00+08', 'America/Los_Angeles')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build()) - .addValues(LocalDateTime.of(2008, 12, 24, 23, 30, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeAdd() { - String sql = - "SELECT " - + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MICROSECOND), " - + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MILLISECOND), " - + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 SECOND), " - + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MINUTE), " - + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 HOUR), " - + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 DAY), " - + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MONTH), " - + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 QUARTER), " - + "DATETIME_ADD(DATETIME '2008-12-25 15:30:00', INTERVAL 10 YEAR) "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addLogicalTypeField("f_time1", SqlTypes.DATETIME) - .addLogicalTypeField("f_time2", SqlTypes.DATETIME) - .addLogicalTypeField("f_time3", SqlTypes.DATETIME) - .addLogicalTypeField("f_time4", SqlTypes.DATETIME) - .addLogicalTypeField("f_time5", SqlTypes.DATETIME) - .addLogicalTypeField("f_time6", SqlTypes.DATETIME) - .addLogicalTypeField("f_time7", SqlTypes.DATETIME) - .addLogicalTypeField("f_time8", SqlTypes.DATETIME) - .addLogicalTypeField("f_time9", SqlTypes.DATETIME) - .build()) - .addValues( - LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(10000), - LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(10000000), - LocalDateTime.of(2008, 12, 25, 15, 30, 10), - LocalDateTime.of(2008, 12, 25, 15, 40, 0), - LocalDateTime.of(2008, 12, 26, 1, 30, 0), - LocalDateTime.of(2009, 1, 4, 15, 30, 0), - LocalDateTime.of(2009, 10, 25, 15, 30, 0), - LocalDateTime.of(2011, 6, 25, 15, 30, 0), - LocalDateTime.of(2018, 12, 25, 15, 30, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeAddWithParameter() { - String sql = "SELECT DATETIME_ADD(@p0, INTERVAL @p1 HOUR)"; - - LocalDateTime datetime = LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000); - ImmutableMap params = - ImmutableMap.of( - "p0", Value.createDatetimeValue(datetime), - "p1", Value.createInt64Value(3L)); - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, params); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build()) - .addValues(LocalDateTime.of(2008, 12, 25, 18, 30, 0).withNano(123456000)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeSub() { - String sql = - "SELECT " - + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MICROSECOND), " - + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MILLISECOND), " - + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 SECOND), " - + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MINUTE), " - + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 HOUR), " - + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 DAY), " - + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 MONTH), " - + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 QUARTER), " - + "DATETIME_SUB(DATETIME '2008-12-25 15:30:00', INTERVAL 10 YEAR) "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addLogicalTypeField("f_time1", SqlTypes.DATETIME) - .addLogicalTypeField("f_time2", SqlTypes.DATETIME) - .addLogicalTypeField("f_time3", SqlTypes.DATETIME) - .addLogicalTypeField("f_time4", SqlTypes.DATETIME) - .addLogicalTypeField("f_time5", SqlTypes.DATETIME) - .addLogicalTypeField("f_time6", SqlTypes.DATETIME) - .addLogicalTypeField("f_time7", SqlTypes.DATETIME) - .addLogicalTypeField("f_time8", SqlTypes.DATETIME) - .addLogicalTypeField("f_time9", SqlTypes.DATETIME) - .build()) - .addValues( - LocalDateTime.of(2008, 12, 25, 15, 29, 59).withNano(999990000), - LocalDateTime.of(2008, 12, 25, 15, 29, 59).withNano(990000000), - LocalDateTime.of(2008, 12, 25, 15, 29, 50), - LocalDateTime.of(2008, 12, 25, 15, 20, 0), - LocalDateTime.of(2008, 12, 25, 5, 30, 0), - LocalDateTime.of(2008, 12, 15, 15, 30, 0), - LocalDateTime.of(2008, 2, 25, 15, 30, 0), - LocalDateTime.of(2006, 6, 25, 15, 30, 0), - LocalDateTime.of(1998, 12, 25, 15, 30, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeDiff() { - String sql = - "SELECT DATETIME_DIFF(DATETIME '2008-12-25 15:30:00', DATETIME '2008-10-25 15:30:00', DAY)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_datetime_diff").build()) - .addValues(61L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeDiffNegativeResult() { - String sql = - "SELECT DATETIME_DIFF(DATETIME '2008-10-25 15:30:00', DATETIME '2008-12-25 15:30:00', DAY)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_datetime_diff").build()) - .addValues(-61L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testDateTimeTrunc() { - String sql = "SELECT DATETIME_TRUNC(DATETIME '2008-12-25 15:30:00', HOUR)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addLogicalTypeField("f_datetime_trunc", SqlTypes.DATETIME) - .build()) - .addValues(LocalDateTime.of(2008, 12, 25, 15, 0, 0)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testFormatDateTime() { - String sql = "SELECT FORMAT_DATETIME('%D %T %E6S', DATETIME '2008-12-25 15:30:00.123456')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("f_datetime_str").build()) - .addValues("12/25/08 15:30:00 00.123456") - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testParseDateTime() { - String sql = "SELECT PARSE_DATETIME('%Y-%m-%d %H:%M:%E6S', '2008-12-25 15:30:00.123456')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("f_datetime", SqlTypes.DATETIME).build()) - .addValues(LocalDateTime.of(2008, 12, 25, 15, 30, 0).withNano(123456000)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - ///////////////////////////////////////////////////////////////////////////// - // TIMESTAMP type tests - ///////////////////////////////////////////////////////////////////////////// - - @Test - public void testTimestampMicrosecondUnsupported() { - String sql = - "WITH Timestamps AS (\n" - + " SELECT TIMESTAMP '2000-01-01 00:11:22.345678+00' as timestamp\n" - + ")\n" - + "SELECT\n" - + " timestamp,\n" - + " EXTRACT(ISOYEAR FROM timestamp) AS isoyear,\n" - + " EXTRACT(YEAR FROM timestamp) AS year,\n" - + " EXTRACT(ISOWEEK FROM timestamp) AS week,\n" - + " EXTRACT(MINUTE FROM timestamp) AS minute\n" - + "FROM Timestamps\n"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - thrown.expect(UnsupportedOperationException.class); - zetaSQLQueryPlanner.convertToBeamRel(sql); - } - - @Test - public void testTimestampLiteralWithoutTimeZone() { - String sql = "SELECT TIMESTAMP '2016-12-25 05:30:00'"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("field1").build()) - .addValues(parseTimestampWithUTCTimeZone("2016-12-25 05:30:00")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampLiteralWithUTCTimeZone() { - String sql = "SELECT TIMESTAMP '2016-12-25 05:30:00+00'"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("field1").build()) - .addValues(parseTimestampWithUTCTimeZone("2016-12-25 05:30:00")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampLiteralWithNonUTCTimeZone() { - String sql = "SELECT TIMESTAMP '2018-12-10 10:38:59-10:00'"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("f_timestamp_with_time_zone").build()) - .addValues(parseTimestampWithTimeZone("2018-12-10 10:38:59-1000")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - // TODO[https://github.com/apache/beam/issues/19980]: Add a test for CURRENT_TIMESTAMP function - // ("SELECT CURRENT_TIMESTAMP()") - - @Test - public void testExtractFromTimestamp() { - String sql = - "WITH Timestamps AS (\n" - + " SELECT TIMESTAMP '2007-12-31 12:34:56.789' AS timestamp UNION ALL\n" - + " SELECT TIMESTAMP '2009-12-31'\n" - + ")\n" - + "SELECT\n" - + " EXTRACT(ISOYEAR FROM timestamp) AS isoyear,\n" - + " EXTRACT(YEAR FROM timestamp) AS year,\n" - + " EXTRACT(ISOWEEK FROM timestamp) AS isoweek,\n" - // TODO[https://github.com/apache/beam/issues/20338]: Add tests for TIMESTAMP_TRUNC and - // EXTRACT with "week with weekday" - // date parts once they are supported - // + " EXTRACT(WEEK FROM timestamp) AS week,\n" - + " EXTRACT(MONTH FROM timestamp) AS month,\n" - + " EXTRACT(QUARTER FROM timestamp) AS quarter,\n" - + " EXTRACT(DAY FROM timestamp) AS day,\n" - + " EXTRACT(DAYOFYEAR FROM timestamp) AS dayofyear,\n" - + " EXTRACT(DAYOFWEEK FROM timestamp) AS dayofweek,\n" - + " EXTRACT(HOUR FROM timestamp) AS hour,\n" - + " EXTRACT(MINUTE FROM timestamp) AS minute,\n" - + " EXTRACT(SECOND FROM timestamp) AS second,\n" - + " EXTRACT(MILLISECOND FROM timestamp) AS millisecond\n" - + "FROM Timestamps"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = - Schema.builder() - .addInt64Field("isoyear") - .addInt64Field("year") - .addInt64Field("isoweek") - // .addInt64Field("week") - .addInt64Field("month") - .addInt64Field("quarter") - .addInt64Field("day") - .addInt64Field("dayofyear") - .addInt64Field("dayofweek") - .addInt64Field("hour") - .addInt64Field("minute") - .addInt64Field("second") - .addInt64Field("millisecond") - .build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema) - .addValues( - 2008L, 2007L, 1L /* , 53L */, 12L, 4L, 31L, 365L, 2L, 12L, 34L, 56L, 789L) - .build(), - Row.withSchema(schema) - .addValues(2009L, 2009L, 53L /* , 52L */, 12L, 4L, 31L, 365L, 5L, 0L, 0L, 0L, 0L) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testExtractDateFromTimestamp() { - String sql = "SELECT EXTRACT(DATE FROM TIMESTAMP '2017-05-26 12:34:56')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("date", SqlTypes.DATE).build()) - .addValues(LocalDate.of(2017, 5, 26)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testExtractTimeFromTimestamp() { - String sql = "SELECT EXTRACT(TIME FROM TIMESTAMP '2017-05-26 12:34:56')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addLogicalTypeField("time", SqlTypes.TIME).build()) - .addValues(LocalTime.of(12, 34, 56)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testExtractDateTimeFromTimestamp() { - String sql = "SELECT EXTRACT(DATETIME FROM TIMESTAMP '2017-05-26 12:34:56')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder().addLogicalTypeField("datetime", SqlTypes.DATETIME).build()) - .addValues(LocalDateTime.of(2017, 5, 26, 12, 34, 56)) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testExtractFromTimestampAtTimeZone() { - String sql = - "WITH Timestamps AS (\n" - + " SELECT TIMESTAMP '2007-12-31 12:34:56.789' AS timestamp\n" - + ")\n" - + "SELECT\n" - + " EXTRACT(DAY FROM timestamp AT TIME ZONE 'America/Vancouver') AS day,\n" - + " EXTRACT(DATE FROM timestamp AT TIME ZONE 'UTC') AS date,\n" - + " EXTRACT(TIME FROM timestamp AT TIME ZONE 'Asia/Shanghai') AS time\n" - + "FROM Timestamps"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = - Schema.builder() - .addInt64Field("day") - .addLogicalTypeField("date", SqlTypes.DATE) - .addLogicalTypeField("time", SqlTypes.TIME) - .build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema) - .addValues(31L, LocalDate.of(2007, 12, 31), LocalTime.of(20, 34, 56, 789000000)) - .build()); - - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testStringFromTimestamp() { - String sql = "SELECT STRING(TIMESTAMP '2008-12-25 15:30:00', 'America/Los_Angeles')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("f_timestamp_string").build()) - .addValues("2008-12-25 07:30:00-08") - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampFromString() { - String sql = "SELECT TIMESTAMP('2008-12-25 15:30:00', 'America/Los_Angeles')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("f_timestamp").build()) - .addValues(parseTimestampWithTimeZone("2008-12-25 15:30:00-08")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampFromDate() { - String sql = "SELECT TIMESTAMP(DATE '2014-01-31')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("f_timestamp").build()) - .addValues(parseTimestampWithTimeZone("2014-01-31 00:00:00+00")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - // test default timezone works properly in query execution stage - public void testTimestampFromDateWithDefaultTimezoneSet() { - String sql = "SELECT TIMESTAMP(DATE '2014-01-31')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - zetaSQLQueryPlanner.setDefaultTimezone("Asia/Shanghai"); - pipeline - .getOptions() - .as(BeamSqlPipelineOptions.class) - .setZetaSqlDefaultTimezone("Asia/Shanghai"); - - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("f_timestamp").build()) - .addValues(parseTimestampWithTimeZone("2014-01-31 00:00:00+08")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampFromDateTime() { - String sql = "SELECT TIMESTAMP(DATETIME '2008-12-25 15:30:00')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("f_timestamp").build()) - .addValues(parseTimestampWithTimeZone("2008-12-25 15:30:00+00")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - // test default timezone works properly in query execution stage - public void testTimestampFromDateTimeWithDefaultTimezoneSet() { - String sql = "SELECT TIMESTAMP(DATETIME '2008-12-25 15:30:00')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - zetaSQLQueryPlanner.setDefaultTimezone("Asia/Shanghai"); - pipeline - .getOptions() - .as(BeamSqlPipelineOptions.class) - .setZetaSqlDefaultTimezone("Asia/Shanghai"); - - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("f_timestamp").build()) - .addValues(parseTimestampWithTimeZone("2008-12-25 15:30:00+08")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampAdd() { - String sql = - "SELECT " - + "TIMESTAMP_ADD(TIMESTAMP '2008-12-25 15:30:00 UTC', INTERVAL 5+5 MINUTE), " - + "TIMESTAMP_ADD(TIMESTAMP '2008-12-25 15:30:00+07:30', INTERVAL 10 MINUTE)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDateTimeField("f_timestamp_add") - .addDateTimeField("f_timestamp_with_time_zone_add") - .build()) - .addValues( - DateTimeUtils.parseTimestampWithUTCTimeZone("2008-12-25 15:40:00"), - parseTimestampWithTimeZone("2008-12-25 15:40:00+0730")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampAddWithParameter1() { - String sql = "SELECT TIMESTAMP_ADD(@p0, INTERVAL @p1 MILLISECOND)"; - ImmutableMap params = - ImmutableMap.of( - "p0", parseTimestampWithTZToValue("2001-01-01 00:00:00+00"), - "p1", Value.createInt64Value(1L)); - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, params); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = Schema.builder().addDateTimeField("field1").build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema) - .addValues(parseTimestampWithTimeZone("2001-01-01 00:00:00.001+00")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampAddWithParameter2() { - String sql = "SELECT TIMESTAMP_ADD(@p0, INTERVAL @p1 MINUTE)"; - ImmutableMap params = - ImmutableMap.of( - "p0", parseTimestampWithTZToValue("2008-12-25 15:30:00+07:30"), - "p1", Value.createInt64Value(10L)); - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql, params); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - final Schema schema = Schema.builder().addDateTimeField("field1").build(); - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(schema) - .addValues(parseTimestampWithTimeZone("2008-12-25 15:40:00+07:30")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampSub() { - String sql = - "SELECT " - + "TIMESTAMP_SUB(TIMESTAMP '2008-12-25 15:30:00 UTC', INTERVAL 5+5 MINUTE), " - + "TIMESTAMP_SUB(TIMESTAMP '2008-12-25 15:30:00+07:30', INTERVAL 10 MINUTE)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDateTimeField("f_timestamp_sub") - .addDateTimeField("f_timestamp_with_time_zone_sub") - .build()) - .addValues( - DateTimeUtils.parseTimestampWithUTCTimeZone("2008-12-25 15:20:00"), - parseTimestampWithTimeZone("2008-12-25 15:20:00+0730")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampDiff() { - String sql = - "SELECT TIMESTAMP_DIFF(" - + "TIMESTAMP '2018-10-14 15:30:00.000 UTC', " - + "TIMESTAMP '2018-08-14 15:05:00.001 UTC', " - + "MILLISECOND)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_timestamp_diff").build()) - .addValues((61L * 24 * 60 + 25) * 60 * 1000 - 1) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampDiffNegativeResult() { - String sql = "SELECT TIMESTAMP_DIFF(TIMESTAMP '2018-08-14', TIMESTAMP '2018-10-14', DAY)"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addInt64Field("f_timestamp_diff").build()) - .addValues(-61L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampTrunc() { - String sql = "SELECT TIMESTAMP_TRUNC(TIMESTAMP '2017-11-06 00:00:00+12', ISOWEEK, 'UTC')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("f_timestamp_trunc").build()) - .addValues(DateTimeUtils.parseTimestampWithUTCTimeZone("2017-10-30 00:00:00")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testFormatTimestamp() { - String sql = "SELECT FORMAT_TIMESTAMP('%D %T', TIMESTAMP '2018-10-14 15:30:00.123+00', 'UTC')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addStringField("f_timestamp_str").build()) - .addValues("10/14/18 15:30:00") - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testParseTimestamp() { - String sql = "SELECT PARSE_TIMESTAMP('%m-%d-%y %T', '10-14-18 15:30:00', 'UTC')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema(Schema.builder().addDateTimeField("f_timestamp").build()) - .addValues(DateTimeUtils.parseTimestampWithUTCTimeZone("2018-10-14 15:30:00")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampFromInt64() { - String sql = "SELECT TIMESTAMP_SECONDS(1230219000), TIMESTAMP_MILLIS(1230219000123) "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDateTimeField("f_timestamp_seconds") - .addDateTimeField("f_timestamp_millis") - .build()) - .addValues( - DateTimeUtils.parseTimestampWithUTCTimeZone("2008-12-25 15:30:00"), - DateTimeUtils.parseTimestampWithUTCTimeZone("2008-12-25 15:30:00.123")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampToUnixInt64() { - String sql = - "SELECT " - + "UNIX_SECONDS(TIMESTAMP '2008-12-25 15:30:00 UTC'), " - + "UNIX_MILLIS(TIMESTAMP '2008-12-25 15:30:00.123 UTC')"; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addInt64Field("f_unix_seconds") - .addInt64Field("f_unix_millis") - .build()) - .addValues(1230219000L, 1230219000123L) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } - - @Test - public void testTimestampFromUnixInt64() { - String sql = - "SELECT " - + "TIMESTAMP_FROM_UNIX_SECONDS(1230219000), " - + "TIMESTAMP_FROM_UNIX_MILLIS(1230219000123) "; - - ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config); - BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql); - PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode); - - PAssert.that(stream) - .containsInAnyOrder( - Row.withSchema( - Schema.builder() - .addDateTimeField("f_timestamp_seconds") - .addDateTimeField("f_timestamp_millis") - .build()) - .addValues( - DateTimeUtils.parseTimestampWithUTCTimeZone("2008-12-25 15:30:00"), - DateTimeUtils.parseTimestampWithUTCTimeZone("2008-12-25 15:30:00.123")) - .build()); - pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES)); - } -} diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTypesUtils.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTypesUtils.java deleted file mode 100644 index 3d8d47839586..000000000000 --- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTypesUtils.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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 java.math.BigDecimal; -import org.apache.beam.sdk.annotations.Internal; - -/** Utils to deal with ZetaSQL type generation. */ -@Internal -public class ZetaSqlTypesUtils { - - private ZetaSqlTypesUtils() {} - - /** - * Create a ZetaSQL NUMERIC value represented as BigDecimal. - * - *

ZetaSQL NUMERIC type is an exact numeric value with 38 digits of precision and 9 decimal - * digits of scale. - * - *

Precision is the number of digits that the number contains. - * - *

Scale is how many of these digits appear after the decimal point. - */ - public static BigDecimal bigDecimalAsNumeric(String s) { - return new BigDecimal(s).setScale(9); - } -} diff --git a/sdks/python/apache_beam/transforms/external_test.py b/sdks/python/apache_beam/transforms/external_test.py index 84a7025c0a5e..2ed7d622ecd6 100644 --- a/sdks/python/apache_beam/transforms/external_test.py +++ b/sdks/python/apache_beam/transforms/external_test.py @@ -377,7 +377,7 @@ def test_sanitize_java_traceback(self): \tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) \tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) \tat java.lang.Thread.run(Thread.java:748) -Caused by: java.lang.IllegalArgumentException: Received unknown SQL Dialect 'X'. Known dialects: [zetasql, calcite] +Caused by: java.lang.IllegalArgumentException: Received unknown SQL Dialect 'X'. Known dialects: [calcite] \tat org.apache.beam.sdk.extensions.sql.expansion.ExternalSqlTransformRegistrar$Builder.buildExternal(ExternalSqlTransformRegistrar.java:73) \tat org.apache.beam.sdk.extensions.sql.expansion.ExternalSqlTransformRegistrar$Builder.buildExternal(ExternalSqlTransformRegistrar.java:63) \tat org.apache.beam.sdk.expansion.service.ExpansionService$TransformProviderForBuilder.getTransform(ExpansionService.java:303) diff --git a/sdks/python/apache_beam/transforms/sql.py b/sdks/python/apache_beam/transforms/sql.py index 21cae3f6c757..ce46c652ddd0 100644 --- a/sdks/python/apache_beam/transforms/sql.py +++ b/sdks/python/apache_beam/transforms/sql.py @@ -82,7 +82,7 @@ def __init__(self, query, dialect=None, ddl=None, expansion_service=None): Creates a SqlTransform which will be expanded to Java's SqlTransform. (See class docs). :param query: The SQL query. - :param dialect: (optional) The dialect, e.g. use 'zetasql' for ZetaSQL. + :param dialect: (optional, deprecated) The dialect. :param ddl: (optional) The DDL statement. :param expansion_service: (optional) The URL of the expansion service to use """ diff --git a/sdks/python/apache_beam/transforms/sql_test.py b/sdks/python/apache_beam/transforms/sql_test.py index cf4136436027..fc55320ba699 100644 --- a/sdks/python/apache_beam/transforms/sql_test.py +++ b/sdks/python/apache_beam/transforms/sql_test.py @@ -20,7 +20,6 @@ # pytype: skip-file import logging -import subprocess import typing import unittest @@ -33,7 +32,6 @@ from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.transforms.sql import SqlTransform -from apache_beam.utils import subprocess_server SimpleRow = typing.NamedTuple( "SimpleRow", [("id", int), ("str", str), ("flt", float)]) @@ -71,23 +69,6 @@ class SqlTransformTest(unittest.TestCase): """ _multiprocess_can_split_ = True - @staticmethod - def _disable_zetasql_test(): - # disable if run on Java8 which is no longer supported by ZetaSQL - try: - java = subprocess_server.JavaHelper.get_java() - result = subprocess.run([java, '-version'], - check=True, - capture_output=True, - text=True) - version_line = result.stderr.splitlines()[0] - version = version_line.split()[2].strip('\"') - if version.startswith("1."): - return True - return False - except: # pylint: disable=bare-except - return False - def test_generate_data(self): with TestPipeline() as p: out = p | SqlTransform( @@ -168,19 +149,6 @@ def test_row(self): | SqlTransform("SELECT a*a as s, LENGTH(b) AS c FROM PCOLLECTION")) assert_that(out, equal_to([(1, 1), (4, 1), (100, 2)])) - def test_zetasql_generate_data(self): - if self._disable_zetasql_test(): - raise unittest.SkipTest("ZetaSQL tests need Java11+") - - with TestPipeline() as p: - out = p | SqlTransform( - """SELECT - CAST(1 AS INT64) AS `int`, - CAST('foo' AS STRING) AS `str`, - CAST(3.14 AS FLOAT64) AS `flt`""", - dialect="zetasql") - assert_that(out, equal_to([(1, "foo", 3.14)])) - def test_windowing_before_sql(self): with TestPipeline() as p: out = ( diff --git a/sdks/python/apache_beam/yaml/tests/sql.yaml b/sdks/python/apache_beam/yaml/tests/sql.yaml index 160871c16f7a..afa9e834fe93 100644 --- a/sdks/python/apache_beam/yaml/tests/sql.yaml +++ b/sdks/python/apache_beam/yaml/tests/sql.yaml @@ -65,9 +65,8 @@ pipelines: - type: Sql config: - dialect: zetasql query: - "SELECT a, cast(b AS STRING) as s FROM PCOLLECTION" + "SELECT a, cast(b AS VARCHAR) as s FROM PCOLLECTION" - type: AssertEqual config: