From 22efb4c0ff1d3c2a25a74d937d96b09dd355bde4 Mon Sep 17 00:00:00 2001 From: wenmo <32723967+aiwenmo@users.noreply.github.com> Date: Sat, 28 Dec 2024 01:23:28 +0800 Subject: [PATCH] Remove BuiltInSqlFunction --- .../runtime/functions/BuiltInSqlFunction.java | 239 ------------------ .../metadata/TransformSqlOperatorTable.java | 31 +-- 2 files changed, 10 insertions(+), 260 deletions(-) delete mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/functions/BuiltInSqlFunction.java diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/functions/BuiltInSqlFunction.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/functions/BuiltInSqlFunction.java deleted file mode 100644 index 2bea19514f2..00000000000 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/functions/BuiltInSqlFunction.java +++ /dev/null @@ -1,239 +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.flink.cdc.runtime.functions; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.functions.BuiltInFunctionDefinition; - -import org.apache.calcite.sql.SqlFunction; -import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlOperatorBinding; -import org.apache.calcite.sql.type.SqlOperandTypeChecker; -import org.apache.calcite.sql.type.SqlOperandTypeInference; -import org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.calcite.sql.validate.SqlMonotonicity; - -import javax.annotation.Nullable; - -import java.util.Optional; -import java.util.function.Function; - -import static org.apache.flink.table.functions.BuiltInFunctionDefinition.DEFAULT_VERSION; -import static org.apache.flink.table.functions.BuiltInFunctionDefinition.qualifyFunctionName; -import static org.apache.flink.table.functions.BuiltInFunctionDefinition.validateFunction; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * SQL version of {@link BuiltInFunctionDefinition}. This is the case when the operator has a - * special parsing syntax or uses other Calcite-specific features that are not exposed via {@link - * BuiltInFunctionDefinition} yet. - * - *

Note: Try to keep usages of this class to a minimum and use Flink's {@link - * BuiltInFunctionDefinition} stack instead. - * - *

For simple functions, use the provided builder. Otherwise, this class can also be extended. - */ -@Internal -public class BuiltInSqlFunction extends SqlFunction { - - private final @Nullable Integer version; - - private final boolean isDeterministic; - - private final boolean isInternal; - - private final Function monotonicity; - - protected BuiltInSqlFunction( - String name, - int version, - SqlKind kind, - @Nullable SqlReturnTypeInference returnTypeInference, - @Nullable SqlOperandTypeInference operandTypeInference, - @Nullable SqlOperandTypeChecker operandTypeChecker, - SqlFunctionCategory category, - boolean isDeterministic, - boolean isInternal, - Function monotonicity) { - super( - checkNotNull(name), - checkNotNull(kind), - returnTypeInference, - operandTypeInference, - operandTypeChecker, - checkNotNull(category)); - this.version = isInternal ? null : version; - this.isDeterministic = isDeterministic; - this.isInternal = isInternal; - this.monotonicity = monotonicity; - validateFunction(name, version, isInternal); - } - - protected BuiltInSqlFunction( - String name, - SqlKind kind, - SqlReturnTypeInference returnTypeInference, - SqlOperandTypeInference operandTypeInference, - @Nullable SqlOperandTypeChecker operandTypeChecker, - SqlFunctionCategory category) { - this( - name, - DEFAULT_VERSION, - kind, - returnTypeInference, - operandTypeInference, - operandTypeChecker, - category, - true, - false, - call -> SqlMonotonicity.NOT_MONOTONIC); - } - - /** Builder for configuring and creating instances of {@link BuiltInSqlFunction}. */ - public static Builder newBuilder() { - return new Builder(); - } - - public final Optional getVersion() { - return Optional.ofNullable(version); - } - - public String getQualifiedName() { - if (isInternal) { - return getName(); - } - assert version != null; - return qualifyFunctionName(getName(), version); - } - - @Override - public boolean isDeterministic() { - return isDeterministic; - } - - public final boolean isInternal() { - return isInternal; - } - - @Override - public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) { - return monotonicity.apply(call); - } - - // -------------------------------------------------------------------------------------------- - // Builder - // -------------------------------------------------------------------------------------------- - - /** Builder for fluent definition of built-in functions. */ - public static class Builder { - - private String name; - - private int version = DEFAULT_VERSION; - - private SqlKind kind = SqlKind.OTHER_FUNCTION; - - private SqlReturnTypeInference returnTypeInference; - - private SqlOperandTypeInference operandTypeInference; - - private SqlOperandTypeChecker operandTypeChecker; - - private SqlFunctionCategory category = SqlFunctionCategory.SYSTEM; - - private boolean isInternal = false; - - private boolean isDeterministic = true; - - private Function monotonicity = - call -> SqlMonotonicity.NOT_MONOTONIC; - - /** @see BuiltInFunctionDefinition.Builder#name(String) */ - public Builder name(String name) { - this.name = name; - return this; - } - - /** @see BuiltInFunctionDefinition.Builder#version(int) */ - public Builder version(int version) { - this.version = version; - return this; - } - - public Builder kind(SqlKind kind) { - this.kind = kind; - return this; - } - - public Builder returnType(SqlReturnTypeInference returnTypeInference) { - this.returnTypeInference = returnTypeInference; - return this; - } - - public Builder operandTypeInference(SqlOperandTypeInference operandTypeInference) { - this.operandTypeInference = operandTypeInference; - return this; - } - - public Builder operandTypeChecker(SqlOperandTypeChecker operandTypeChecker) { - this.operandTypeChecker = operandTypeChecker; - return this; - } - - public Builder category(SqlFunctionCategory category) { - this.category = category; - return this; - } - - public Builder notDeterministic() { - this.isDeterministic = false; - return this; - } - - /** @see BuiltInFunctionDefinition.Builder#internal() */ - public Builder internal() { - this.isInternal = true; - return this; - } - - public Builder monotonicity(SqlMonotonicity staticMonotonicity) { - this.monotonicity = call -> staticMonotonicity; - return this; - } - - public Builder monotonicity(Function monotonicity) { - this.monotonicity = monotonicity; - return this; - } - - public BuiltInSqlFunction build() { - return new BuiltInSqlFunction( - name, - version, - kind, - returnTypeInference, - operandTypeInference, - operandTypeChecker, - category, - isDeterministic, - isInternal, - monotonicity); - } - } -} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/metadata/TransformSqlOperatorTable.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/metadata/TransformSqlOperatorTable.java index b6f83d95c16..d47db49f8e7 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/metadata/TransformSqlOperatorTable.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/parser/metadata/TransformSqlOperatorTable.java @@ -18,7 +18,6 @@ package org.apache.flink.cdc.runtime.parser.metadata; import org.apache.flink.cdc.runtime.functions.BuiltInScalarFunction; -import org.apache.flink.cdc.runtime.functions.BuiltInSqlFunction; import org.apache.flink.cdc.runtime.functions.BuiltInTimestampFunction; import org.apache.calcite.sql.SqlBinaryOperator; @@ -42,7 +41,6 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeTransforms; import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable; -import org.apache.calcite.sql.validate.SqlMonotonicity; import org.apache.calcite.sql.validate.SqlNameMatcher; import org.apache.calcite.sql.validate.SqlNameMatchers; @@ -215,25 +213,16 @@ public SqlSyntax getSyntax() { } }; public static final SqlFunction UNIX_TIMESTAMP = - BuiltInSqlFunction.newBuilder() - .name("UNIX_TIMESTAMP") - .returnType(ReturnTypes.BIGINT_NULLABLE) - .operandTypeChecker( - OperandTypes.or( - OperandTypes.NILADIC, - OperandTypes.family(SqlTypeFamily.CHARACTER), - OperandTypes.family( - SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER))) - .notDeterministic() - .monotonicity( - call -> { - if (call.getOperandCount() == 0) { - return SqlMonotonicity.INCREASING; - } else { - return SqlMonotonicity.NOT_MONOTONIC; - } - }) - .build(); + new SqlFunction( + "UNIX_TIMESTAMP", + SqlKind.OTHER_FUNCTION, + ReturnTypes.BIGINT_NULLABLE, + null, + OperandTypes.or( + OperandTypes.NILADIC, + OperandTypes.family(SqlTypeFamily.CHARACTER), + OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER)), + SqlFunctionCategory.TIMEDATE); public static final SqlFunction FROM_UNIXTIME = new SqlFunction( "FROM_UNIXTIME",