Airblader commented on a change in pull request #17256: URL: https://github.com/apache/flink/pull/17256#discussion_r708923508
########## File path: docs/data/sql_functions.yml ########## @@ -516,7 +516,14 @@ conditional: - sql: NULLIF(value1, value2) description: Returns NULL if value1 is equal to value2; returns value1 otherwise. E.g., NULLIF(5, 5) returns NULL; NULLIF(5, 0) returns 5. - sql: COALESCE(value1, value2 [, value3]*) Review comment: The syntax here seems to be incorrect as it suggests a required minimum of two arguments, but it's only one argument. ########## File path: flink-python/pyflink/table/expressions.py ########## @@ -544,6 +544,25 @@ def if_then_else(condition: Union[bool, Expression[bool]], if_true, if_false) -> return _ternary_op("ifThenElse", condition, if_true, if_false) +def coalesce(*args) -> Expression: + """ + Coalesce specifies a series of expressions, and returns the first expression whose value is + not null. If all the expressions evaluate as null, coalesce returns a null value. The return + type of the coalesce function is the least common type among its arguments, and it's nullable + if all its arguments are nullable. + + e.g. coalesce(null, "-") returns "-". Review comment: Here, too, please refer to e.g. the `json_*` functions on how we want to format the example sections. Same for Java & Scala. ########## File path: docs/data/sql_functions.yml ########## @@ -516,7 +516,14 @@ conditional: - sql: NULLIF(value1, value2) description: Returns NULL if value1 is equal to value2; returns value1 otherwise. E.g., NULLIF(5, 5) returns NULL; NULLIF(5, 0) returns 5. - sql: COALESCE(value1, value2 [, value3]*) - description: Returns the first value that is not NULL from value1, value2, .... E.g., COALESCE(NULL, 5) returns 5. + description: | + Coalesce specifies a series of expressions, and returns the first expression whose value is not null. + + If all the expressions evaluate as null, coalesce returns a null value. + The return type of the coalesce function is the least common type among its arguments, and it's nullable if all its arguments are nullable. Review comment: The function itself doesn't specify any expressions, but rather the user passes them in, so I would rephrase this slightly. Since this is the SQL docs, we should use NULL for consistency. Also, "least common type" sounds a bit odd to me, it kind of sounds like "we'll count the types of the operands and whichever one appears least often will be used". ```suggestion Returns the first argument that is not NULL. If all arguments are NULL, it returns NULL as well. The return type is the least restrictive, common type of all of its arguments. The return type is nullable if all arguments are nullable as well. ``` ```suggestion Coalesce specifies a series of expressions, and returns the first expression whose value is not null. If all the expressions evaluate as null, coalesce returns a null value. The return type of the coalesce function is the least common type among its arguments, and it's nullable if all its arguments are nullable. ``` ########## File path: docs/data/sql_functions.yml ########## @@ -516,7 +516,14 @@ conditional: - sql: NULLIF(value1, value2) description: Returns NULL if value1 is equal to value2; returns value1 otherwise. E.g., NULLIF(5, 5) returns NULL; NULLIF(5, 0) returns 5. - sql: COALESCE(value1, value2 [, value3]*) Review comment: Please also add the entry for the Table API syntax. ########## File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/strategies/NullableIfArgsTypeStrategy.java ########## @@ -70,12 +75,24 @@ public NullableIfArgsTypeStrategy( .orElse(argumentDataTypes.size()), argumentDataTypes.size()); - final boolean isNullableArgument = - IntStream.range(fromArg, toArg) - .mapToObj(argumentDataTypes::get) - .anyMatch( - dataType -> - dataType.getLogicalType().isNullable()); + final boolean isNullableArgument; + if (nullableIfAllArgsNullable) { + isNullableArgument = + IntStream.range(fromArg, toArg) + .mapToObj(argumentDataTypes::get) + .allMatch( Review comment: nit: here and below, we can have Spotify make this a bit prettier by using another `map`: ``` IntStream.range(fromArg, toArg) .mapToObj(argumentDataTypes::get) .map(DataType::getLogicalType) .allMatch(LogicalType::isNullable); ``` ########## File path: docs/data/sql_functions.yml ########## @@ -516,7 +516,14 @@ conditional: - sql: NULLIF(value1, value2) description: Returns NULL if value1 is equal to value2; returns value1 otherwise. E.g., NULLIF(5, 5) returns NULL; NULLIF(5, 0) returns 5. - sql: COALESCE(value1, value2 [, value3]*) - description: Returns the first value that is not NULL from value1, value2, .... E.g., COALESCE(NULL, 5) returns 5. + description: | + Coalesce specifies a series of expressions, and returns the first expression whose value is not null. + + If all the expressions evaluate as null, coalesce returns a null value. + The return type of the coalesce function is the least common type among its arguments, and it's nullable if all its arguments are nullable. Review comment: Of course this goes for all places of the docs. ########## File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeStrategies.java ########## @@ -105,6 +105,23 @@ public static TypeStrategy nullableIfArgs(TypeStrategy initialStrategy) { return nullableIfArgs(ConstantArgumentCount.any(), initialStrategy); } + /** + * A type strategy that can be used to make a result type nullable if all the selected input + * arguments is nullable. Otherwise the type will be not null. Review comment: ```suggestion * arguments are nullable. Otherwise the type will be non-nullable. ``` ########## File path: flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CoalesceFunctionITCase.java ########## @@ -0,0 +1,62 @@ +/* + * 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.table.planner.functions; + +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; + +import org.junit.runners.Parameterized; + +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.table.api.DataTypes.BIGINT; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.Expressions.$; +import static org.apache.flink.table.api.Expressions.coalesce; + +/** Test COALESCE and its return type. * */ Review comment: There's a trailing `*` at the end here. We could also link to `BuiltInFunctionDefinitions#COALESCE` from here (nit). ########## File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/inference/TypeStrategies.java ########## @@ -105,6 +105,23 @@ public static TypeStrategy nullableIfArgs(TypeStrategy initialStrategy) { return nullableIfArgs(ConstantArgumentCount.any(), initialStrategy); } + /** + * A type strategy that can be used to make a result type nullable if all the selected input + * arguments is nullable. Otherwise the type will be not null. + */ + public static TypeStrategy nullableIfAllArgs( Review comment: Yeah, I don't think it would work either. The common argument strategy will make the type nullable if any type is nullable, but we want nullability when _all_ are nullable. ########## File path: docs/data/sql_functions.yml ########## @@ -516,7 +516,14 @@ conditional: - sql: NULLIF(value1, value2) description: Returns NULL if value1 is equal to value2; returns value1 otherwise. E.g., NULLIF(5, 5) returns NULL; NULLIF(5, 0) returns 5. - sql: COALESCE(value1, value2 [, value3]*) - description: Returns the first value that is not NULL from value1, value2, .... E.g., COALESCE(NULL, 5) returns 5. + description: | + Coalesce specifies a series of expressions, and returns the first expression whose value is not null. + + If all the expressions evaluate as null, coalesce returns a null value. + The return type of the coalesce function is the least common type among its arguments, and it's nullable if all its arguments are nullable. + + E.g. COALESCE(NULL, "-") returns "-". + E.g. COALESCE(f0, f1, "-") returns the first non-null value among f0 and f1, if both are null returns "-". Review comment: Let's please turn this into a proper code block of examples. For example, look at the `JSON_*` functions we recently introduced. The examples should also be valid SQL, i.e. make use of SQL comments and proper syntax for string literals, something like ``` -- Returns 'default' COALESCE(NULL, 'default') -- Returns 'default' if f0 and f1 are both NULL COALESCE(f0, f1, 'default') ``` ########## File path: flink-python/pyflink/table/expressions.py ########## @@ -544,6 +544,25 @@ def if_then_else(condition: Union[bool, Expression[bool]], if_true, if_false) -> return _ternary_op("ifThenElse", condition, if_true, if_false) +def coalesce(*args) -> Expression: + """ + Coalesce specifies a series of expressions, and returns the first expression whose value is + not null. If all the expressions evaluate as null, coalesce returns a null value. The return + type of the coalesce function is the least common type among its arguments, and it's nullable + if all its arguments are nullable. + + e.g. coalesce(null, "-") returns "-". + + e.g. coalesce(col("f0"), col("f1"), "-") returns the first non-null value among f0 and f1, if both + are null returns "-". + + :param args: the input expressions. + """ + gateway = get_gateway() + args = to_jarray(gateway.jvm.Object, [_get_java_expression(arg) for arg in args]) Review comment: This is fine, I guess, but after we merge https://github.com/apache/flink/pull/17186/files#diff-5930126827bbe9069974e6094238fca02034cab5435aafd5a6fc012b34f9df83R70 we'd also be able to just call `_varargs_op("coalesce", *args)`, I think. ########## File path: docs/data/sql_functions.yml ########## @@ -516,7 +516,14 @@ conditional: - sql: NULLIF(value1, value2) description: Returns NULL if value1 is equal to value2; returns value1 otherwise. E.g., NULLIF(5, 5) returns NULL; NULLIF(5, 0) returns 5. - sql: COALESCE(value1, value2 [, value3]*) - description: Returns the first value that is not NULL from value1, value2, .... E.g., COALESCE(NULL, 5) returns 5. + description: | + Coalesce specifies a series of expressions, and returns the first expression whose value is not null. + + If all the expressions evaluate as null, coalesce returns a null value. + The return type of the coalesce function is the least common type among its arguments, and it's nullable if all its arguments are nullable. + + E.g. COALESCE(NULL, "-") returns "-". + E.g. COALESCE(f0, f1, "-") returns the first non-null value among f0 and f1, if both are null returns "-". Review comment: Also make sure to run the docs locally to verify everything's looking good. Unfortunately we currently need to manually make sure the code block does not scroll horizontally (which results in really poor UX), and line-break the examples if necessary. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org