Skip to content

Commit

Permalink
[FLINK-27891][Table] Add ARRAY_APPEND and ARRAY_PREPEND functions
Browse files Browse the repository at this point in the history
  • Loading branch information
snuyanzin committed Feb 12, 2024
1 parent 6f74889 commit a7cb97e
Show file tree
Hide file tree
Showing 11 changed files with 392 additions and 0 deletions.
6 changes: 6 additions & 0 deletions docs/data/sql_functions.yml
Expand Up @@ -628,6 +628,9 @@ collection:
- sql: map ‘[’ value ‘]’
table: MAP.at(ANY)
description: Returns the value specified by key value in map.
- sql: ARRAY_APPEND(array, element)
table: array.arrayAppend(element)
description: Appends an element to the end of the array and returns the result. If the array itself is null, the function will return null. If an element to add is null, the null element will be added to the end of the array.
- sql: ARRAY_CONTAINS(haystack, needle)
table: haystack.arrayContains(needle)
description: Returns whether the given element exists in an array. Checking for null elements in the array is supported. If the array itself is null, the function will return null. The given element is cast implicitly to the array's element type if necessary.
Expand All @@ -637,6 +640,9 @@ collection:
- sql: ARRAY_POSITION(haystack, needle)
table: haystack.arrayPosition(needle)
description: Returns the position of the first occurrence of element in the given array as int. Returns 0 if the given value could not be found in the array. Returns null if either of the arguments are null. And this is not zero based, but 1-based index. The first element in the array has index 1.
- sql: ARRAY_PREPEND(array, element)
table: array.arrayPrepend(element)
description: Appends an element to the beginning of the array and returns the result. If the array itself is null, the function will return null. If an element to add is null, the null element will be added to the beginning of the array.
- sql: ARRAY_REMOVE(haystack, needle)
table: haystack.arrayRemove(needle)
description: Removes all elements that equal to element from array. If the array itself is null, the function will return null. Keeps ordering of elements.
Expand Down
6 changes: 6 additions & 0 deletions docs/data/sql_functions_zh.yml
Expand Up @@ -763,6 +763,9 @@ collection:
- sql: map ‘[’ value ‘]’
table: MAP.at(ANY)
description: 返回 map 中指定 key 对应的值。
- sql: ARRAY_APPEND(array, element)
table: array.arrayAppend(element)
description: Appends an element to the end of the array and returns the result. If the array itself is null, the function will return null. If an element to add is null, the null element will be added to the end of the array.
- sql: ARRAY_CONTAINS(haystack, needle)
table: haystack.arrayContains(needle)
description: 返回是否数组 haystack 中包含指定元素 needle。支持检查数组中是否存在 null。 如果数组本身是null,函数会返回 null。如果需要,指定元素会隐式转换为数组的元素类型。
Expand All @@ -772,6 +775,9 @@ collection:
- sql: ARRAY_POSITION(haystack, needle)
table: haystack.arrayPosition(needle)
description: 返回数组中第一次出现 needle 元素的位置,返回类型为 int。如果数组中不存在该元素则返回 0。如果两个参数中任何一个参数为 null,则返回 null。序号不是从 0 开始,而是从 1 开始,第一个元素的序号为 1。
- sql: ARRAY_PREPEND(array, element)
table: array.arrayPrepend(element)
description: Appends an element to the beginning of the array and returns the result. If the array itself is null, the function will return null. If an element to add is null, the null element will be added to the beginning of the array.
- sql: ARRAY_REMOVE(haystack, needle)
table: haystack.arrayRemove(needle)
description: 删除数组中所有和元素 needle 相等的元素。如果数组是 null,则返回 null。函数会保留数组中元素的顺序。
Expand Down
18 changes: 18 additions & 0 deletions flink-python/pyflink/table/expression.py
Expand Up @@ -1474,6 +1474,15 @@ def element(self) -> 'Expression':
"""
return _unary_op("element")(self)

def array_append(self, addition) -> 'Expression':
"""
Appends an element to the end of the array and returns the result.
If the array itself is null, the function will return null. If an element to add is null,
the null element will be added to the end of the array.
"""
return _binary_op("arrayAppend")(self, addition)

def array_contains(self, needle) -> 'Expression':
"""
Returns whether the given element exists in an array.
Expand Down Expand Up @@ -1502,6 +1511,15 @@ def array_position(self, needle) -> 'Expression':
"""
return _binary_op("arrayPosition")(self, needle)

def array_prepend(self, addition) -> 'Expression':
"""
Appends an element to the beginning of the array and returns the result.
If the array itself is null, the function will return null. If an element to add is null,
the null element will be added to the beginning of the array.
"""
return _binary_op("arrayPrepend")(self, addition)

def array_remove(self, needle) -> 'Expression':
"""
Removes all elements that equal to element from array.
Expand Down
Expand Up @@ -54,13 +54,15 @@
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ACOS;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AND;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_AGG;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_APPEND;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_CONCAT;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_CONTAINS;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_DISTINCT;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_ELEMENT;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_MAX;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_MIN;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_POSITION;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_PREPEND;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_REMOVE;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_REVERSE;
import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_SLICE;
Expand Down Expand Up @@ -1355,6 +1357,18 @@ public OutType element() {
return toApiSpecificExpression(unresolvedCall(ARRAY_ELEMENT, toExpr()));
}

/**
* Appends an element to the end of the array and returns the result.
*
* <p>If the array itself is null, the function will return null. If an element to add is null,
* the null element will be added to the end of the array. The given element is cast implicitly
* to the array's element type if necessary.
*/
public OutType arrayAppend(InType element) {
return toApiSpecificExpression(
unresolvedCall(ARRAY_APPEND, toExpr(), objectToExpression(element)));
}

/**
* Returns whether the given element exists in an array.
*
Expand Down Expand Up @@ -1389,6 +1403,18 @@ public OutType arrayPosition(InType needle) {
unresolvedCall(ARRAY_POSITION, toExpr(), objectToExpression(needle)));
}

/**
* Appends an element to the beginning of the array and returns the result.
*
* <p>If the array itself is null, the function will return null. If an element to add is null,
* the null element will be added to the beginning of the array. The given element is cast
* implicitly to the array's element type if necessary.
*/
public OutType arrayPrepend(InType element) {
return toApiSpecificExpression(
unresolvedCall(ARRAY_PREPEND, toExpr(), objectToExpression(element)));
}

/**
* Removes all elements that equal to element from array.
*
Expand Down
Expand Up @@ -102,6 +102,7 @@
import static org.apache.flink.table.types.inference.strategies.SpecificInputTypeStrategies.JSON_ARGUMENT;
import static org.apache.flink.table.types.inference.strategies.SpecificInputTypeStrategies.TWO_EQUALS_COMPARABLE;
import static org.apache.flink.table.types.inference.strategies.SpecificInputTypeStrategies.TWO_FULLY_COMPARABLE;
import static org.apache.flink.table.types.inference.strategies.SpecificTypeStrategies.ARRAY_APPEND_PREPEND;

/** Dictionary of function definitions for all built-in functions. */
@PublicEvolving
Expand Down Expand Up @@ -215,6 +216,20 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
"org.apache.flink.table.runtime.functions.scalar.CoalesceFunction")
.build();

public static final BuiltInFunctionDefinition ARRAY_APPEND =
BuiltInFunctionDefinition.newBuilder()
.name("ARRAY_APPEND")
.kind(SCALAR)
.inputTypeStrategy(
sequence(
Arrays.asList("array", "element"),
Arrays.asList(
logical(LogicalTypeRoot.ARRAY), ARRAY_ELEMENT_ARG)))
.outputTypeStrategy(nullableIfArgs(nullableIfArgs(ARRAY_APPEND_PREPEND)))
.runtimeClass(
"org.apache.flink.table.runtime.functions.scalar.ArrayAppendFunction")
.build();

public static final BuiltInFunctionDefinition ARRAY_CONTAINS =
BuiltInFunctionDefinition.newBuilder()
.name("ARRAY_CONTAINS")
Expand Down Expand Up @@ -258,6 +273,20 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL)
"org.apache.flink.table.runtime.functions.scalar.ArrayPositionFunction")
.build();

public static final BuiltInFunctionDefinition ARRAY_PREPEND =
BuiltInFunctionDefinition.newBuilder()
.name("ARRAY_PREPEND")
.kind(SCALAR)
.inputTypeStrategy(
sequence(
Arrays.asList("array", "element"),
Arrays.asList(
logical(LogicalTypeRoot.ARRAY), ARRAY_ELEMENT_ARG)))
.outputTypeStrategy(nullableIfArgs(ARRAY_APPEND_PREPEND))
.runtimeClass(
"org.apache.flink.table.runtime.functions.scalar.ArrayPrependFunction")
.build();

public static final BuiltInFunctionDefinition ARRAY_REMOVE =
BuiltInFunctionDefinition.newBuilder()
.name("ARRAY_REMOVE")
Expand Down
@@ -0,0 +1,55 @@
/*
* 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.types.inference.strategies;

import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.inference.CallContext;
import org.apache.flink.table.types.inference.TypeStrategy;
import org.apache.flink.table.types.logical.LogicalType;

import java.util.List;
import java.util.Optional;

import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;

/**
* Type strategy that returns a {@link DataTypes#ARRAY(DataType)} with element type equal to the
* type of the first argument if it's not nullable or element to add is not nullable, otherwise it
* returns {@link DataTypes#ARRAY(DataType)} with type equal to the type of the element to add to
* array.
*/
@Internal
public class ArrayAppendPrependTypeStrategy implements TypeStrategy {
@Override
public Optional<DataType> inferType(CallContext callContext) {
final List<DataType> argumentDataTypes = callContext.getArgumentDataTypes();
final DataType arrayDataType = argumentDataTypes.get(0);
final DataType elementToAddDataType = argumentDataTypes.get(1);
final LogicalType arrayElementLogicalType =
arrayDataType.getLogicalType().getChildren().get(0);
if (elementToAddDataType.getLogicalType().isNullable()
&& !arrayElementLogicalType.isNullable()) {
return Optional.of(
DataTypes.ARRAY(fromLogicalToDataType(arrayElementLogicalType).nullable()));
}
return Optional.of(arrayDataType);
}
}
Expand Up @@ -65,6 +65,10 @@ public final class SpecificTypeStrategies {
public static final TypeStrategy ARRAY_ELEMENT = new ArrayElementTypeStrategy();

public static final TypeStrategy ITEM_AT = new ItemAtTypeStrategy();

/** See {@link ArrayAppendPrependTypeStrategy}. */
public static final TypeStrategy ARRAY_APPEND_PREPEND = new ArrayAppendPrependTypeStrategy();

/** See {@link GetTypeStrategy}. */
public static final TypeStrategy GET = new GetTypeStrategy();

Expand Down
@@ -0,0 +1,56 @@
/*
* 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.types.inference.strategies;

import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.types.inference.TypeStrategiesTestBase;

import java.util.stream.Stream;

/** Tests for {@link ArrayAppendPrependTypeStrategy}. */
public class ArrayAppendPrependTypeStrategyTest extends TypeStrategiesTestBase {
@Override
protected Stream<TestSpec> testData() {
return Stream.of(
TestSpec.forStrategy(
"Array element is nullable, element to add is nullable",
SpecificTypeStrategies.ARRAY_APPEND_PREPEND)
.inputTypes(DataTypes.ARRAY(DataTypes.BIGINT()), DataTypes.BIGINT())
.expectDataType(DataTypes.ARRAY(DataTypes.BIGINT())),
TestSpec.forStrategy(
"Array element type is non null, element to add is nullable",
SpecificTypeStrategies.ARRAY_APPEND_PREPEND)
.inputTypes(
DataTypes.ARRAY(DataTypes.BIGINT().notNull()), DataTypes.BIGINT())
.expectDataType(DataTypes.ARRAY(DataTypes.BIGINT())),
TestSpec.forStrategy(
"Array element type is nullable, element to add is non null",
SpecificTypeStrategies.ARRAY_APPEND_PREPEND)
.inputTypes(
DataTypes.ARRAY(DataTypes.BIGINT()), DataTypes.BIGINT().notNull())
.expectDataType(DataTypes.ARRAY(DataTypes.BIGINT())),
TestSpec.forStrategy(
"Array element type is non null, element to add is non null",
SpecificTypeStrategies.ARRAY_APPEND_PREPEND)
.inputTypes(
DataTypes.ARRAY(DataTypes.BIGINT().notNull()),
DataTypes.BIGINT().notNull())
.expectDataType(DataTypes.ARRAY(DataTypes.BIGINT().notNull())));
}
}

0 comments on commit a7cb97e

Please sign in to comment.