This is an automated email from the ASF dual-hosted git repository.
dwysakowicz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 2429c296a60 [FLINK-31663] Implement ARRAY_EXCEPT function
2429c296a60 is described below
commit 2429c296a60bf0a0e8a4acebc04a059008708d1f
Author: Hanyu Zheng <[email protected]>
AuthorDate: Wed Jul 26 23:31:09 2023 -0700
[FLINK-31663] Implement ARRAY_EXCEPT function
---
docs/data/sql_functions.yml | 5 +-
.../docs/reference/pyflink.table/expressions.rst | 1 +
flink-python/pyflink/table/expression.py | 9 ++
.../flink/table/api/internal/BaseExpressions.java | 14 +++
.../functions/BuiltInFunctionDefinitions.java | 10 ++
.../functions/CollectionFunctionsITCase.java | 127 ++++++++++++++++++++-
.../functions/scalar/ArrayExceptFunction.java | 103 +++++++++++++++++
.../runtime/util/EqualityAndHashcodeProvider.java | 92 +++++++++++++++
.../flink/table/runtime/util/ObjectContainer.java | 65 +++++++++++
9 files changed, 424 insertions(+), 2 deletions(-)
diff --git a/docs/data/sql_functions.yml b/docs/data/sql_functions.yml
index 7f06199ecb0..b7150391606 100644
--- a/docs/data/sql_functions.yml
+++ b/docs/data/sql_functions.yml
@@ -682,7 +682,10 @@ collection:
- sql: MAP_FROM_ARRAYS(array_of_keys, array_of_values)
table: mapFromArrays(array_of_keys, array_of_values)
description: Returns a map created from an arrays of keys and values. Note
that the lengths of two arrays should be the same.
-
+ - sql: ARRAY_EXCEPT(array1, array2)
+ table: arrayOne.arrayExcept(arrayTwo)
+ description: Returns an ARRAY that contains the elements from array1 that
are not in array2. If no elements remain after excluding the elements in array2
from array1, the function returns an empty ARRAY. If one or both arguments are
NULL, the function returns NULL. The order of the elements from array1 is kept.
+
json:
- sql: IS JSON [ { VALUE | SCALAR | ARRAY | OBJECT } ]
table: STRING.isJson([JsonType type])
diff --git a/flink-python/docs/reference/pyflink.table/expressions.rst
b/flink-python/docs/reference/pyflink.table/expressions.rst
index 3d3cc1ad326..dbc69682a3b 100644
--- a/flink-python/docs/reference/pyflink.table/expressions.rst
+++ b/flink-python/docs/reference/pyflink.table/expressions.rst
@@ -241,6 +241,7 @@ advanced type helper functions
Expression.map_entries
Expression.map_keys
Expression.map_values
+ Expression.array_except
time definition functions
diff --git a/flink-python/pyflink/table/expression.py
b/flink-python/pyflink/table/expression.py
index c3b8f1b6591..8c892b52e35 100644
--- a/flink-python/pyflink/table/expression.py
+++ b/flink-python/pyflink/table/expression.py
@@ -1609,6 +1609,15 @@ class Expression(Generic[T]):
"""
return _unary_op("arrayMin")(self)
+ def array_except(self, array) -> 'Expression':
+ """
+ Returns an ARRAY that contains the elements from array1 that are not
in array2.
+ If no elements remain after excluding the elements in array2 from
array1,
+ the function returns an empty ARRAY. If one or both arguments are NULL,
+ the function returns NULL. The order of the elements from array1 is
kept.
+ """
+ return _binary_op("arrayExcept")(self, array)
+
@property
def map_keys(self) -> 'Expression':
"""
diff --git
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
index 8e022179277..090274d89a5 100644
---
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
+++
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/BaseExpressions.java
@@ -59,6 +59,7 @@ import static
org.apache.flink.table.functions.BuiltInFunctionDefinitions.ARRAY_
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_EXCEPT;
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;
@@ -230,6 +231,19 @@ public abstract class BaseExpressions<InType, OutType> {
.toArray(Expression[]::new)));
}
+ /**
+ * Returns an ARRAY that contains the elements from array1 that are not in
array2. If no
+ * elements remain after excluding the elements in array2 from array1, the
function returns an
+ * empty ARRAY.
+ *
+ * <p>If one or both arguments are NULL, the function returns NULL. The
order of the elements
+ * from array1 is kept.
+ */
+ public OutType arrayExcept(InType array) {
+ return toApiSpecificExpression(
+ unresolvedCall(ARRAY_EXCEPT, toExpr(),
objectToExpression(array)));
+ }
+
/**
* Boolean AND in three-valued logic. This is an infix notation. See also
{@link
* Expressions#and(Object, Object, Object...)} for prefix notation with
multiple arguments.
diff --git
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
index c1aad6fe1ad..07de1eb2c9c 100644
---
a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
+++
b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java
@@ -439,6 +439,16 @@ public final class BuiltInFunctionDefinitions {
.internal()
.build();
+ public static final BuiltInFunctionDefinition ARRAY_EXCEPT =
+ BuiltInFunctionDefinition.newBuilder()
+ .name("ARRAY_EXCEPT")
+ .kind(SCALAR)
+ .inputTypeStrategy(commonArrayType(2))
+ .outputTypeStrategy(nullableIfArgs(COMMON))
+ .runtimeClass(
+
"org.apache.flink.table.runtime.functions.scalar.ArrayExceptFunction")
+ .build();
+
//
--------------------------------------------------------------------------------------------
// Logic functions
//
--------------------------------------------------------------------------------------------
diff --git
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java
index 7d0821661bb..65b65d68c46 100644
---
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java
+++
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CollectionFunctionsITCase.java
@@ -54,7 +54,8 @@ class CollectionFunctionsITCase extends
BuiltInFunctionTestBase {
arrayJoinTestCases(),
arraySliceTestCases(),
arrayMinTestCases(),
- arraySortTestCases())
+ arraySortTestCases(),
+ arrayExceptTestCases())
.flatMap(s -> s);
}
@@ -1596,4 +1597,128 @@ class CollectionFunctionsITCase extends
BuiltInFunctionTestBase {
},
DataTypes.ARRAY(DataTypes.DATE())));
}
+
+ private Stream<TestSetSpec> arrayExceptTestCases() {
+ return Stream.of(
+
TestSetSpec.forFunction(BuiltInFunctionDefinitions.ARRAY_EXCEPT)
+ .onFieldsWithData(
+ new Integer[] {1, 2, 2},
+ null,
+ new Row[] {
+ Row.of(true, LocalDate.of(2022, 4, 20)),
+ Row.of(true, LocalDate.of(1990, 10, 14)),
+ null
+ },
+ new Integer[] {null, null, 1},
+ new Integer[][] {
+ new Integer[] {1, null, 3}, new Integer[]
{0}, new Integer[] {1}
+ },
+ new Map[] {
+ CollectionUtil.map(entry(1, "a"), entry(2,
"b")),
+ CollectionUtil.map(entry(3, "c"), entry(4,
"d")),
+ null
+ },
+ new Integer[] {1, 2, 3, 2, 4, 2})
+ .andDataTypes(
+ DataTypes.ARRAY(DataTypes.INT()),
+ DataTypes.ARRAY(DataTypes.INT()),
+ DataTypes.ARRAY(
+ DataTypes.ROW(DataTypes.BOOLEAN(),
DataTypes.DATE())),
+ DataTypes.ARRAY(DataTypes.INT()),
+
DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.INT())),
+ DataTypes.ARRAY(DataTypes.MAP(DataTypes.INT(),
DataTypes.STRING())),
+ DataTypes.ARRAY(DataTypes.INT()))
+ // ARRAY<INT>
+ .testResult(
+ $("f0").arrayExcept(new Integer[] {1, null,
4}),
+ "ARRAY_EXCEPT(f0, ARRAY[1, NULL, 4])",
+ new Integer[] {2, 2},
+ DataTypes.ARRAY(DataTypes.INT()).nullable())
+ .testResult(
+ $("f0").arrayExcept(new Integer[] {1}),
+ "ARRAY_EXCEPT(f0, ARRAY[1])",
+ new Integer[] {2, 2},
+ DataTypes.ARRAY(DataTypes.INT()).nullable())
+ .testResult(
+ $("f0").arrayExcept(new Integer[] {42}),
+ "ARRAY_EXCEPT(f0, ARRAY[42])",
+ new Integer[] {1, 2, 2},
+ DataTypes.ARRAY(DataTypes.INT()).nullable())
+ .testResult(
+ $("f6").arrayExcept(new Integer[] {2, 2}),
+ "ARRAY_EXCEPT(f6, ARRAY[2, 2])",
+ new Integer[] {1, 3, 4, 2},
+ DataTypes.ARRAY(DataTypes.INT()).nullable())
+ // arrayTwo is NULL
+ .testResult(
+ $("f0").arrayExcept(
+ lit(null,
DataTypes.ARRAY(DataTypes.INT()))
+
.cast(DataTypes.ARRAY(DataTypes.INT()))),
+ "ARRAY_EXCEPT(f0, CAST(NULL AS ARRAY<INT>))",
+ null,
+ DataTypes.ARRAY(DataTypes.INT()).nullable())
+ // arrayTwo contains null elements
+ .testResult(
+ $("f0").arrayExcept(new Integer[] {null, 2}),
+ "ARRAY_EXCEPT(f0, ARRAY[null, 2])",
+ new Integer[] {1, 2},
+ DataTypes.ARRAY(DataTypes.INT()).nullable())
+ // arrayOne is NULL
+ .testResult(
+ $("f1").arrayExcept(new Integer[] {1, 2, 3}),
+ "ARRAY_EXCEPT(f1, ARRAY[1,2,3])",
+ null,
+ DataTypes.ARRAY(DataTypes.INT()).nullable())
+ // arrayOne contains null elements
+ .testResult(
+ $("f3").arrayExcept(new Integer[] {null, 42}),
+ "ARRAY_EXCEPT(f3, ARRAY[null, 42])",
+ new Integer[] {null, 1},
+ DataTypes.ARRAY(DataTypes.INT()).nullable())
+ // ARRAY<ROW<BOOLEAN, DATE>>
+ .testResult(
+ $("f2").arrayExcept(
+ new Row[] {
+ Row.of(true,
LocalDate.of(1990, 10, 14))
+ }),
+ "ARRAY_EXCEPT(f2, ARRAY[(TRUE, DATE
'1990-10-14')])",
+ new Row[] {Row.of(true, LocalDate.of(2022, 4,
20)), null},
+ DataTypes.ARRAY(
+ DataTypes.ROW(
+ DataTypes.BOOLEAN(),
DataTypes.DATE()))
+ .nullable())
+ // ARRAY<ARRAY<INT>>
+ .testResult(
+ $("f4").arrayExcept(new Integer[][] {new
Integer[] {0}}),
+ "ARRAY_EXCEPT(f4, ARRAY[ARRAY[0]])",
+ new Integer[][] {new Integer[] {1, null, 3},
new Integer[] {1}},
+
DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.INT()).nullable()))
+ // ARRAY<MAP<INT, STRING>> with NULL elements
+ .testResult(
+ $("f5").arrayExcept(
+ new Map[] {
+
CollectionUtil.map(entry(3, "c"), entry(4, "d"))
+ }),
+ "ARRAY_EXCEPT(f5, ARRAY[MAP[3, 'c', 4, 'd']])",
+ new Map[] {CollectionUtil.map(entry(1, "a"),
entry(2, "b")), null},
+ DataTypes.ARRAY(DataTypes.MAP(DataTypes.INT(),
DataTypes.STRING()))
+ .nullable())
+ // Invalid signatures
+ .testSqlValidationError(
+ "ARRAY_EXCEPT(f0, TRUE)",
+ "Invalid input arguments. Expected signatures
are:\n"
+ + "ARRAY_EXCEPT(<COMMON>, <COMMON>)")
+ .testTableApiValidationError(
+ $("f0").arrayExcept(true),
+ "Invalid input arguments. Expected signatures
are:\n"
+ + "ARRAY_EXCEPT(<COMMON>, <COMMON>)")
+ .testSqlValidationError(
+ "ARRAY_EXCEPT(f0, ARRAY['hi', 'there'])",
+ "Invalid input arguments. Expected signatures
are:\n"
+ + "ARRAY_EXCEPT(<COMMON>, <COMMON>)")
+ .testTableApiValidationError(
+ $("f0").arrayExcept(new String[] {"hi",
"there"}),
+ "Invalid input arguments. Expected signatures
are:\n"
+ + "ARRAY_EXCEPT(<COMMON>, <COMMON>)"));
+ }
}
diff --git
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/ArrayExceptFunction.java
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/ArrayExceptFunction.java
new file mode 100644
index 00000000000..98125730214
--- /dev/null
+++
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/scalar/ArrayExceptFunction.java
@@ -0,0 +1,103 @@
+/*
+ * 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.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.runtime.util.EqualityAndHashcodeProvider;
+import org.apache.flink.table.runtime.util.ObjectContainer;
+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.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Implementation of {@link BuiltInFunctionDefinitions#ARRAY_EXCEPT}. */
+@Internal
+public class ArrayExceptFunction extends BuiltInScalarFunction {
+ private final ArrayData.ElementGetter elementGetter;
+ private final EqualityAndHashcodeProvider equalityAndHashcodeProvider;
+
+ 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());
+ this.equalityAndHashcodeProvider = new
EqualityAndHashcodeProvider(context, dataType);
+ }
+
+ @Override
+ public void open(FunctionContext context) throws Exception {
+ equalityAndHashcodeProvider.open(context);
+ }
+
+ public @Nullable ArrayData eval(ArrayData arrayOne, ArrayData arrayTwo) {
+ try {
+ if (arrayOne == null || arrayTwo == null) {
+ return null;
+ }
+
+ List<Object> list = new ArrayList<>();
+ Map<ObjectContainer, Integer> map = new HashMap<>();
+ for (int pos = 0; pos < arrayTwo.size(); pos++) {
+ final Object element =
elementGetter.getElementOrNull(arrayTwo, pos);
+ final ObjectContainer objectContainer =
createObjectContainer(element);
+ map.merge(objectContainer, 1, (k, v) -> v + 1);
+ }
+ for (int pos = 0; pos < arrayOne.size(); pos++) {
+ final Object element =
elementGetter.getElementOrNull(arrayOne, pos);
+ final ObjectContainer objectContainer =
createObjectContainer(element);
+ if (map.containsKey(objectContainer)) {
+ map.compute(objectContainer, (k, v) -> v == null || v == 1
? null : v - 1);
+ } else {
+ list.add(element);
+ }
+ }
+ return new GenericArrayData(list.toArray());
+ } catch (Throwable t) {
+ throw new FlinkRuntimeException(t);
+ }
+ }
+
+ private ObjectContainer createObjectContainer(Object element) {
+ if (element == null) {
+ return null;
+ }
+ return new ObjectContainer(
+ element,
+ equalityAndHashcodeProvider::equals,
+ equalityAndHashcodeProvider::hashCode);
+ }
+
+ @Override
+ public void close() throws Exception {
+ equalityAndHashcodeProvider.close();
+ }
+}
diff --git
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/EqualityAndHashcodeProvider.java
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/EqualityAndHashcodeProvider.java
new file mode 100644
index 00000000000..be8ea5b8f97
--- /dev/null
+++
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/EqualityAndHashcodeProvider.java
@@ -0,0 +1,92 @@
+/*
+ * 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.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+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.DataType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.invoke.MethodHandle;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedCall;
+
+/**
+ * This class is used for scalar function, in that it is used for sharing the
initialization context
+ * between scalar functions that need code generated hashcode and equals
method.
+ */
+@Internal
+public class EqualityAndHashcodeProvider implements Closeable, Serializable {
+ private final SpecializedFunction.ExpressionEvaluator hashcodeEvaluator;
+ private final SpecializedFunction.ExpressionEvaluator equalityEvaluator;
+ private transient MethodHandle hashcodeHandle;
+
+ private transient MethodHandle equalityHandle;
+
+ public EqualityAndHashcodeProvider(
+ SpecializedFunction.SpecializedContext context, DataType dataType)
{
+ hashcodeEvaluator =
+ context.createEvaluator(
+
unresolvedCall(BuiltInFunctionDefinitions.INTERNAL_HASHCODE, $("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()));
+ }
+
+ public void open(FunctionContext context) throws Exception {
+ hashcodeHandle = hashcodeEvaluator.open(context);
+ equalityHandle = equalityEvaluator.open(context);
+ }
+
+ public boolean equals(Object o1, Object o2) {
+ try {
+ return (Boolean) equalityHandle.invoke(o1, o2);
+ } catch (Throwable e) {
+ throw new FlinkRuntimeException(e);
+ }
+ }
+
+ public int hashCode(Object o) {
+ try {
+ return (int) hashcodeHandle.invoke(o);
+ } catch (Throwable e) {
+ throw new FlinkRuntimeException(e);
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ hashcodeEvaluator.close();
+ equalityEvaluator.close();
+ }
+}
diff --git
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/ObjectContainer.java
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/ObjectContainer.java
new file mode 100644
index 00000000000..1a876ce0246
--- /dev/null
+++
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/ObjectContainer.java
@@ -0,0 +1,65 @@
+/*
+ * 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.util;
+
+import org.apache.flink.annotation.Internal;
+
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+/**
+ * This class is used for scalar function, in that it is used for comparing
Objects using code
+ * generated hashCode and equals instead of using the
Object#equals/Object#hashcode versions.
+ */
+@Internal
+public class ObjectContainer {
+
+ private final Object o;
+
+ private final BiFunction<Object, Object, Boolean> equalsMethod;
+
+ private final Function<Object, Integer> hashCodeMethod;
+
+ public ObjectContainer(
+ Object o,
+ BiFunction<Object, Object, Boolean> equalsMethod,
+ Function<Object, Integer> hashCodeMethod) {
+ this.o = o;
+ this.equalsMethod = equalsMethod;
+ this.hashCodeMethod = hashCodeMethod;
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (!(other instanceof ObjectContainer)) {
+ return false;
+ }
+ ObjectContainer that = (ObjectContainer) other;
+ return equalsMethod.apply(this.o, that.o);
+ }
+
+ @Override
+ public int hashCode() {
+ return hashCodeMethod.apply(o);
+ }
+}