Skip to content

Commit

Permalink
[FLINK-24586][table] SQL functions should return STRING instead of VA…
Browse files Browse the repository at this point in the history
…RCHAR(2000)

This closes apache#17691.
  • Loading branch information
SteNicholas authored and twalthr committed Nov 15, 2021
1 parent 1266df4 commit 6f5c649
Show file tree
Hide file tree
Showing 11 changed files with 204 additions and 69 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@

import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.ARG0_VARCHAR_FORCE_NULLABLE;
import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.STR_MAP_NULLABLE;
import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_2000_NULLABLE;
import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_FORCE_NULLABLE;

/** Operator table that contains only Flink-specific functions and operators. */
public class FlinkSqlOperatorTable extends ReflectiveSqlOperatorTable {
Expand Down Expand Up @@ -307,7 +307,7 @@ public void lookupOperatorOverloads(
new SqlFunction(
"CHR",
SqlKind.OTHER_FUNCTION,
VARCHAR_2000_NULLABLE,
VARCHAR_FORCE_NULLABLE,
null,
OperandTypes.family(SqlTypeFamily.INTEGER),
SqlFunctionCategory.STRING);
Expand Down Expand Up @@ -351,7 +351,7 @@ public void lookupOperatorOverloads(
new SqlFunction(
"REVERSE",
SqlKind.OTHER_FUNCTION,
VARCHAR_2000_NULLABLE,
VARCHAR_FORCE_NULLABLE,
null,
OperandTypes.family(SqlTypeFamily.STRING),
SqlFunctionCategory.STRING);
Expand All @@ -372,7 +372,7 @@ public void lookupOperatorOverloads(
new SqlFunction(
"SPLIT_INDEX",
SqlKind.OTHER_FUNCTION,
VARCHAR_2000_NULLABLE,
VARCHAR_FORCE_NULLABLE,
null,
OperandTypes.or(
OperandTypes.family(
Expand Down Expand Up @@ -401,9 +401,7 @@ public void lookupOperatorOverloads(
new SqlFunction(
"REGEXP_EXTRACT",
SqlKind.OTHER_FUNCTION,
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.VARCHAR),
SqlTypeTransforms.FORCE_NULLABLE),
VARCHAR_FORCE_NULLABLE,
null,
OperandTypes.or(OperandTypes.STRING_STRING_INTEGER, OperandTypes.STRING_STRING),
SqlFunctionCategory.STRING);
Expand Down Expand Up @@ -507,9 +505,7 @@ public void lookupOperatorOverloads(
new SqlFunction(
"DATE_FORMAT",
SqlKind.OTHER_FUNCTION,
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.VARCHAR),
SqlTypeTransforms.FORCE_NULLABLE),
VARCHAR_FORCE_NULLABLE,
InferTypes.RETURN_TYPE,
OperandTypes.or(
OperandTypes.family(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.STRING),
Expand All @@ -529,7 +525,7 @@ public void lookupOperatorOverloads(
new SqlFunction(
"PARSE_URL",
SqlKind.OTHER_FUNCTION,
VARCHAR_2000_NULLABLE,
VARCHAR_FORCE_NULLABLE,
null,
OperandTypes.or(
OperandTypes.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING),
Expand Down Expand Up @@ -604,7 +600,7 @@ public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
new SqlFunction(
"FROM_UNIXTIME",
SqlKind.OTHER_FUNCTION,
VARCHAR_2000_NULLABLE,
VARCHAR_FORCE_NULLABLE,
null,
OperandTypes.or(
OperandTypes.family(SqlTypeFamily.INTEGER),
Expand Down Expand Up @@ -781,9 +777,7 @@ public boolean isDeterministic() {
new SqlFunction(
"CONVERT_TZ",
SqlKind.OTHER_FUNCTION,
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.VARCHAR),
SqlTypeTransforms.FORCE_NULLABLE),
VARCHAR_FORCE_NULLABLE,
null,
OperandTypes.family(
SqlTypeFamily.STRING, SqlTypeFamily.STRING, SqlTypeFamily.STRING),
Expand Down Expand Up @@ -827,7 +821,7 @@ public boolean isDeterministic() {
new SqlFunction(
"DECODE",
SqlKind.OTHER_FUNCTION,
VARCHAR_2000_NULLABLE,
VARCHAR_FORCE_NULLABLE,
null,
OperandTypes.family(SqlTypeFamily.BINARY, SqlTypeFamily.STRING),
SqlFunctionCategory.STRING);
Expand Down Expand Up @@ -1145,13 +1139,13 @@ public List<SqlGroupedWindowFunction> getAuxiliaryFunctions() {
// JSON FUNCTIONS
public static final SqlFunction JSON_EXISTS = SqlStdOperatorTable.JSON_EXISTS;
public static final SqlFunction JSON_VALUE = SqlStdOperatorTable.JSON_VALUE;
public static final SqlFunction JSON_QUERY = SqlStdOperatorTable.JSON_QUERY;
public static final SqlFunction JSON_OBJECT = new SqlJsonObjectFunction();
public static final SqlFunction JSON_QUERY = new SqlJsonQueryFunctionWrapper();
public static final SqlFunction JSON_OBJECT = new SqlJsonObjectFunctionWrapper();
public static final SqlAggFunction JSON_OBJECTAGG_NULL_ON_NULL =
SqlStdOperatorTable.JSON_OBJECTAGG;
public static final SqlAggFunction JSON_OBJECTAGG_ABSENT_ON_NULL =
SqlStdOperatorTable.JSON_OBJECTAGG.with(SqlJsonConstructorNullClause.ABSENT_ON_NULL);
public static final SqlFunction JSON_ARRAY = SqlStdOperatorTable.JSON_ARRAY;
public static final SqlFunction JSON_ARRAY = new SqlJsonArrayFunctionWrapper();
public static final SqlAggFunction JSON_ARRAYAGG_NULL_ON_NULL =
SqlStdOperatorTable.JSON_ARRAYAGG.with(SqlJsonConstructorNullClause.NULL_ON_NULL);
public static final SqlAggFunction JSON_ARRAYAGG_ABSENT_ON_NULL =
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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:https://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.table.planner.functions.sql;

import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlOperatorBinding;
import org.apache.calcite.sql.fun.SqlJsonArrayFunction;
import org.apache.calcite.sql.type.SqlReturnTypeInference;

import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_NOT_NULL;

/**
* This class is a wrapper class for the {@link SqlJsonArrayFunction} but using the {@code
* VARCHAR_NOT_NULL} return type inference.
*/
class SqlJsonArrayFunctionWrapper extends SqlJsonArrayFunction {

@Override
public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
RelDataType returnType = VARCHAR_NOT_NULL.inferReturnType(opBinding);
if (returnType == null) {
throw new IllegalArgumentException(
"Cannot infer return type for "
+ opBinding.getOperator()
+ "; operand types: "
+ opBinding.collectOperandTypes());
} else {
return returnType;
}
}

@Override
public SqlReturnTypeInference getReturnTypeInference() {
return VARCHAR_NOT_NULL;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlOperandCountRange;
import org.apache.calcite.sql.SqlWriter;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlOperandCountRanges;
Expand All @@ -46,8 +45,8 @@
/**
* This class has been copied from Calcite to backport the fix made during CALCITE-4394.
*
* <p>TODO Remove this class with Calcite 1.27 and replace it with {@link
* SqlStdOperatorTable#JSON_OBJECT}.
* <p>TODO Remove this class with Calcite 1.27 and replace the {@link SqlJsonObjectFunction} in the
* {@link SqlJsonObjectFunctionWrapper} using the {@code VARCHAR_NOT_NULL} return type inference.
*/
public class SqlJsonObjectFunction extends SqlFunction {
public SqlJsonObjectFunction() {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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:https://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.table.planner.functions.sql;

import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlOperatorBinding;
import org.apache.calcite.sql.type.SqlReturnTypeInference;

import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_NOT_NULL;

/**
* This class is a wrapper class for the {@link SqlJsonObjectFunction} but using the {@code
* VARCHAR_NOT_NULL} return type inference.
*/
class SqlJsonObjectFunctionWrapper extends SqlJsonObjectFunction {

@Override
public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
RelDataType returnType = VARCHAR_NOT_NULL.inferReturnType(opBinding);
if (returnType == null) {
throw new IllegalArgumentException(
"Cannot infer return type for "
+ opBinding.getOperator()
+ "; operand types: "
+ opBinding.collectOperandTypes());
} else {
return returnType;
}
}

@Override
public SqlReturnTypeInference getReturnTypeInference() {
return VARCHAR_NOT_NULL;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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:https://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.table.planner.functions.sql;

import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlOperatorBinding;
import org.apache.calcite.sql.fun.SqlJsonQueryFunction;
import org.apache.calcite.sql.type.SqlReturnTypeInference;

import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_FORCE_NULLABLE;

/**
* This class is a wrapper class for the {@link SqlJsonQueryFunction} but using the {@code
* VARCHAR_FORCE_NULLABLE} return type inference.
*/
class SqlJsonQueryFunctionWrapper extends SqlJsonQueryFunction {

@Override
public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
RelDataType returnType = VARCHAR_FORCE_NULLABLE.inferReturnType(opBinding);
if (returnType == null) {
throw new IllegalArgumentException(
"Cannot infer return type for "
+ opBinding.getOperator()
+ "; operand types: "
+ opBinding.collectOperandTypes());
} else {
return returnType;
}
}

@Override
public SqlReturnTypeInference getReturnTypeInference() {
return VARCHAR_FORCE_NULLABLE;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -104,9 +104,13 @@ public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
}
};

/** Type-inference strategy that always returns "VARCHAR(2000)" with nulls always allowed. */
public static final SqlReturnTypeInference VARCHAR_2000_NULLABLE =
ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE);
public static final SqlReturnTypeInference VARCHAR_FORCE_NULLABLE =
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.VARCHAR), SqlTypeTransforms.FORCE_NULLABLE);

public static final SqlReturnTypeInference VARCHAR_NOT_NULL =
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.VARCHAR), SqlTypeTransforms.TO_NOT_NULLABLE);

public static final SqlReturnTypeInference ROUND_FUNCTION_NULLABLE =
ReturnTypes.cascade(ROUND_FUNCTION, SqlTypeTransforms.TO_NULLABLE);
Expand Down
Loading

0 comments on commit 6f5c649

Please sign in to comment.