dawidwys commented on code in PR #23173: URL: https://github.com/apache/flink/pull/23173#discussion_r1491037396
########## flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java: ########## @@ -391,6 +391,16 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL) .internal() .build(); + public static final BuiltInFunctionDefinition ARRAY_EXCEPT = + BuiltInFunctionDefinition.newBuilder() + .name("ARRAY_EXCEPT") + .kind(SCALAR) + .inputTypeStrategy(commonArrayType(2)) + .outputTypeStrategy(nullableIfArgs(COMMON)) Review Comment: I want to double check with you if this is the behaviour we expect. Is this correct that for: `ARRAY_EXCEPT(ARRAY(1,2,3), null)` we get `null`? That what this output type suggests. Is that correct we take into account the nullability of the second argument? ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/ArrayExceptFunction.java: ########## @@ -0,0 +1,161 @@ +/* + * 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.runtime.functions.scalar; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.SpecializedFunction; +import org.apache.flink.table.types.CollectionDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nullable; + +import java.lang.invoke.MethodHandle; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.apache.flink.table.api.Expressions.$; + +/** Implementation of {@link BuiltInFunctionDefinitions#ARRAY_EXCEPT}. */ +@Internal +public class ArrayExceptFunction extends BuiltInScalarFunction { + private final ArrayData.ElementGetter elementGetter; + private final SpecializedFunction.ExpressionEvaluator hashcodeEvaluator; + private final SpecializedFunction.ExpressionEvaluator equalityEvaluator; + private transient MethodHandle hashcodeHandle; + + private transient MethodHandle equalityHandle; + + public ArrayExceptFunction(SpecializedFunction.SpecializedContext context) { + super(BuiltInFunctionDefinitions.ARRAY_EXCEPT, context); + final DataType dataType = + ((CollectionDataType) context.getCallContext().getArgumentDataTypes().get(0)) + .getElementDataType() + .toInternal(); + elementGetter = ArrayData.createElementGetter(dataType.toInternal().getLogicalType()); + hashcodeEvaluator = + context.createEvaluator( + Expressions.call("$HASHCODE$1", $("element1")), Review Comment: Can we use `BuiltinFunctionDefinitions.INTERNAL_HASHCODE` instead of the `$HASHCODE$1` string? ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/ArrayExceptFunction.java: ########## @@ -0,0 +1,161 @@ +/* + * 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.runtime.functions.scalar; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.SpecializedFunction; +import org.apache.flink.table.types.CollectionDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nullable; + +import java.lang.invoke.MethodHandle; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.apache.flink.table.api.Expressions.$; + +/** Implementation of {@link BuiltInFunctionDefinitions#ARRAY_EXCEPT}. */ +@Internal +public class ArrayExceptFunction extends BuiltInScalarFunction { + private final ArrayData.ElementGetter elementGetter; + private final SpecializedFunction.ExpressionEvaluator hashcodeEvaluator; + private final SpecializedFunction.ExpressionEvaluator equalityEvaluator; + private transient MethodHandle hashcodeHandle; + + private transient MethodHandle equalityHandle; + + public ArrayExceptFunction(SpecializedFunction.SpecializedContext context) { + super(BuiltInFunctionDefinitions.ARRAY_EXCEPT, context); + final DataType dataType = + ((CollectionDataType) context.getCallContext().getArgumentDataTypes().get(0)) + .getElementDataType() + .toInternal(); + elementGetter = ArrayData.createElementGetter(dataType.toInternal().getLogicalType()); + hashcodeEvaluator = Review Comment: We could extract that to a utility class e.g. `EqualityAndHashcodeProvider`: ``` class EqualityAndHashcodeProvider implements Closeable { open(FunctionContext); equals(Object o1, Object o2); hashcode(Object o); } ``` that way we could reuse it in all functions that need it. ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/ArrayExceptFunction.java: ########## @@ -0,0 +1,161 @@ +/* + * 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.runtime.functions.scalar; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.SpecializedFunction; +import org.apache.flink.table.types.CollectionDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nullable; + +import java.lang.invoke.MethodHandle; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.apache.flink.table.api.Expressions.$; + +/** Implementation of {@link BuiltInFunctionDefinitions#ARRAY_EXCEPT}. */ +@Internal +public class ArrayExceptFunction extends BuiltInScalarFunction { + private final ArrayData.ElementGetter elementGetter; + private final SpecializedFunction.ExpressionEvaluator hashcodeEvaluator; + private final SpecializedFunction.ExpressionEvaluator equalityEvaluator; + private transient MethodHandle hashcodeHandle; + + private transient MethodHandle equalityHandle; + + public ArrayExceptFunction(SpecializedFunction.SpecializedContext context) { + super(BuiltInFunctionDefinitions.ARRAY_EXCEPT, context); + final DataType dataType = + ((CollectionDataType) context.getCallContext().getArgumentDataTypes().get(0)) + .getElementDataType() + .toInternal(); + elementGetter = ArrayData.createElementGetter(dataType.toInternal().getLogicalType()); + hashcodeEvaluator = + context.createEvaluator( + Expressions.call("$HASHCODE$1", $("element1")), + DataTypes.INT(), + DataTypes.FIELD("element1", dataType.notNull().toInternal())); + equalityEvaluator = + context.createEvaluator( + $("element1").isEqual($("element2")), + DataTypes.BOOLEAN(), + DataTypes.FIELD("element1", dataType.notNull().toInternal()), + DataTypes.FIELD("element2", dataType.notNull().toInternal())); + } + + @Override + public void open(FunctionContext context) throws Exception { + hashcodeHandle = hashcodeEvaluator.open(context); + equalityHandle = equalityEvaluator.open(context); + } + + public @Nullable ArrayData eval(ArrayData arrayOne, ArrayData arrayTwo) { + try { + if (arrayOne == null) { + return null; + } + + List<Object> list = new ArrayList<>(); + Set<ObjectContainer> seen = new HashSet<>(); + + boolean isNullPresentInArrayTwo = false; + if (arrayTwo != null) { + for (int pos = 0; pos < arrayTwo.size(); pos++) { + final Object element = elementGetter.getElementOrNull(arrayTwo, pos); + if (element == null) { + isNullPresentInArrayTwo = true; + } else { + ObjectContainer objectContainer = new ObjectContainer(element); + seen.add(objectContainer); + } + } + } + boolean isNullPresentInArrayOne = false; + for (int pos = 0; pos < arrayOne.size(); pos++) { + final Object element = elementGetter.getElementOrNull(arrayOne, pos); + if (element == null) { + isNullPresentInArrayOne = true; + } else { + ObjectContainer objectContainer = new ObjectContainer(element); + if (!seen.contains(objectContainer)) { + seen.add(objectContainer); + list.add(element); + } + } + } + if (!isNullPresentInArrayTwo && isNullPresentInArrayOne) { + list.add(null); + } Review Comment: This will not preserve the order of the null within the array. Is that the expected behaviour? ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/ArrayExceptFunction.java: ########## @@ -0,0 +1,161 @@ +/* + * 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.runtime.functions.scalar; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.SpecializedFunction; +import org.apache.flink.table.types.CollectionDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nullable; + +import java.lang.invoke.MethodHandle; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.apache.flink.table.api.Expressions.$; + +/** Implementation of {@link BuiltInFunctionDefinitions#ARRAY_EXCEPT}. */ +@Internal +public class ArrayExceptFunction extends BuiltInScalarFunction { + private final ArrayData.ElementGetter elementGetter; + private final SpecializedFunction.ExpressionEvaluator hashcodeEvaluator; + private final SpecializedFunction.ExpressionEvaluator equalityEvaluator; + private transient MethodHandle hashcodeHandle; + + private transient MethodHandle equalityHandle; + + public ArrayExceptFunction(SpecializedFunction.SpecializedContext context) { + super(BuiltInFunctionDefinitions.ARRAY_EXCEPT, context); + final DataType dataType = + ((CollectionDataType) context.getCallContext().getArgumentDataTypes().get(0)) + .getElementDataType() + .toInternal(); + elementGetter = ArrayData.createElementGetter(dataType.toInternal().getLogicalType()); + hashcodeEvaluator = + context.createEvaluator( + Expressions.call("$HASHCODE$1", $("element1")), + DataTypes.INT(), + DataTypes.FIELD("element1", dataType.notNull().toInternal())); + equalityEvaluator = + context.createEvaluator( + $("element1").isEqual($("element2")), + DataTypes.BOOLEAN(), + DataTypes.FIELD("element1", dataType.notNull().toInternal()), + DataTypes.FIELD("element2", dataType.notNull().toInternal())); + } + + @Override + public void open(FunctionContext context) throws Exception { + hashcodeHandle = hashcodeEvaluator.open(context); + equalityHandle = equalityEvaluator.open(context); + } + + public @Nullable ArrayData eval(ArrayData arrayOne, ArrayData arrayTwo) { + try { + if (arrayOne == null) { + return null; + } + + List<Object> list = new ArrayList<>(); + Set<ObjectContainer> seen = new HashSet<>(); + + boolean isNullPresentInArrayTwo = false; + if (arrayTwo != null) { + for (int pos = 0; pos < arrayTwo.size(); pos++) { + final Object element = elementGetter.getElementOrNull(arrayTwo, pos); + if (element == null) { + isNullPresentInArrayTwo = true; + } else { + ObjectContainer objectContainer = new ObjectContainer(element); + seen.add(objectContainer); + } + } + } + boolean isNullPresentInArrayOne = false; + for (int pos = 0; pos < arrayOne.size(); pos++) { + final Object element = elementGetter.getElementOrNull(arrayOne, pos); + if (element == null) { + isNullPresentInArrayOne = true; + } else { + ObjectContainer objectContainer = new ObjectContainer(element); + if (!seen.contains(objectContainer)) { + seen.add(objectContainer); + list.add(element); + } + } + } + if (!isNullPresentInArrayTwo && isNullPresentInArrayOne) { + list.add(null); + } + return new GenericArrayData(list.toArray()); + } catch (Throwable t) { + throw new FlinkRuntimeException(t); + } + } + + private class ObjectContainer { Review Comment: I believe you need it in another functions as well. Have you thought of making this a top-level internal class that you can use also e.g. in `ArrayIntersectFunction`? ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/ArrayExceptFunction.java: ########## @@ -0,0 +1,161 @@ +/* + * 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.runtime.functions.scalar; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.SpecializedFunction; +import org.apache.flink.table.types.CollectionDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nullable; + +import java.lang.invoke.MethodHandle; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.apache.flink.table.api.Expressions.$; + +/** Implementation of {@link BuiltInFunctionDefinitions#ARRAY_EXCEPT}. */ +@Internal +public class ArrayExceptFunction extends BuiltInScalarFunction { + private final ArrayData.ElementGetter elementGetter; + private final SpecializedFunction.ExpressionEvaluator hashcodeEvaluator; + private final SpecializedFunction.ExpressionEvaluator equalityEvaluator; + private transient MethodHandle hashcodeHandle; + + private transient MethodHandle equalityHandle; + + public ArrayExceptFunction(SpecializedFunction.SpecializedContext context) { + super(BuiltInFunctionDefinitions.ARRAY_EXCEPT, context); + final DataType dataType = + ((CollectionDataType) context.getCallContext().getArgumentDataTypes().get(0)) + .getElementDataType() + .toInternal(); + elementGetter = ArrayData.createElementGetter(dataType.toInternal().getLogicalType()); + hashcodeEvaluator = + context.createEvaluator( + Expressions.call("$HASHCODE$1", $("element1")), + DataTypes.INT(), + DataTypes.FIELD("element1", dataType.notNull().toInternal())); + equalityEvaluator = + context.createEvaluator( + $("element1").isEqual($("element2")), + DataTypes.BOOLEAN(), + DataTypes.FIELD("element1", dataType.notNull().toInternal()), + DataTypes.FIELD("element2", dataType.notNull().toInternal())); + } + + @Override + public void open(FunctionContext context) throws Exception { + hashcodeHandle = hashcodeEvaluator.open(context); + equalityHandle = equalityEvaluator.open(context); + } + + public @Nullable ArrayData eval(ArrayData arrayOne, ArrayData arrayTwo) { + try { + if (arrayOne == null) { + return null; + } + + List<Object> list = new ArrayList<>(); + Set<ObjectContainer> seen = new HashSet<>(); + + boolean isNullPresentInArrayTwo = false; + if (arrayTwo != null) { + for (int pos = 0; pos < arrayTwo.size(); pos++) { + final Object element = elementGetter.getElementOrNull(arrayTwo, pos); + if (element == null) { + isNullPresentInArrayTwo = true; + } else { + ObjectContainer objectContainer = new ObjectContainer(element); + seen.add(objectContainer); + } + } + } + boolean isNullPresentInArrayOne = false; + for (int pos = 0; pos < arrayOne.size(); pos++) { + final Object element = elementGetter.getElementOrNull(arrayOne, pos); + if (element == null) { + isNullPresentInArrayOne = true; + } else { + ObjectContainer objectContainer = new ObjectContainer(element); + if (!seen.contains(objectContainer)) { + seen.add(objectContainer); + list.add(element); + } + } + } + if (!isNullPresentInArrayTwo && isNullPresentInArrayOne) { + list.add(null); + } + return new GenericArrayData(list.toArray()); + } catch (Throwable t) { + throw new FlinkRuntimeException(t); + } + } + + private class ObjectContainer { + + Object o; Review Comment: ```suggestion private final Object o; ``` -- 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