> 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