You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by re...@apache.org on 2022/07/27 03:18:08 UTC

[flink] branch master updated: [FLINK-28416][table] Add (Async)LookupFunction and providers in replace of (Async)TableFunction as the API for lookup table (#20177)

This is an automated email from the ASF dual-hosted git repository.

renqs 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 3e2620bc785 [FLINK-28416][table] Add (Async)LookupFunction and providers in replace of (Async)TableFunction as the API for lookup table (#20177)
3e2620bc785 is described below

commit 3e2620bc785b3b4d82f1f188eb7b1e0e129b14d3
Author: Qingsheng Ren <re...@gmail.com>
AuthorDate: Wed Jul 27 11:18:00 2022 +0800

    [FLINK-28416][table] Add (Async)LookupFunction and providers in replace of (Async)TableFunction as the API for lookup table (#20177)
---
 .../source/AsyncTableFunctionProvider.java         |  5 ++
 .../table/connector/source/LookupTableSource.java  | 10 ++--
 .../connector/source/TableFunctionProvider.java    |  4 ++
 .../source/lookup/AsyncLookupFunctionProvider.java | 36 ++++++++++++
 .../source/lookup/LookupFunctionProvider.java      | 36 ++++++++++++
 .../flink/table/functions/AsyncLookupFunction.java | 64 ++++++++++++++++++++++
 .../flink/table/functions/LookupFunction.java      | 59 ++++++++++++++++++++
 7 files changed, 210 insertions(+), 4 deletions(-)

diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/AsyncTableFunctionProvider.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/AsyncTableFunctionProvider.java
index 491f3d336bf..8c118636e63 100644
--- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/AsyncTableFunctionProvider.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/AsyncTableFunctionProvider.java
@@ -19,6 +19,7 @@
 package org.apache.flink.table.connector.source;
 
 import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.connector.source.lookup.AsyncLookupFunctionProvider;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.functions.AsyncTableFunction;
 import org.apache.flink.table.functions.UserDefinedFunction;
@@ -36,8 +37,12 @@ import org.apache.flink.types.Row;
  * similar to other {@link UserDefinedFunction}s. However, for convenience, in a {@link
  * LookupTableSource} the output type can simply be a {@link Row} or {@link RowData} in which case
  * the input and output types are derived from the table's schema with default conversion.
+ *
+ * @deprecated Please use {@link AsyncLookupFunctionProvider} to implement asynchronous lookup
+ *     table.
  */
 @PublicEvolving
+@Deprecated
 public interface AsyncTableFunctionProvider<T> extends LookupTableSource.LookupRuntimeProvider {
 
     /** Helper method for creating a static provider. */
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/LookupTableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/LookupTableSource.java
index fb2c97074a0..457fffc5de3 100644
--- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/LookupTableSource.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/LookupTableSource.java
@@ -19,6 +19,8 @@
 package org.apache.flink.table.connector.source;
 
 import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.connector.source.lookup.AsyncLookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
 import org.apache.flink.types.RowKind;
 
 import java.io.Serializable;
@@ -56,8 +58,8 @@ public interface LookupTableSource extends DynamicTableSource {
      * <p>The given {@link LookupContext} offers utilities by the planner for creating runtime
      * implementation with minimal dependencies to internal data structures.
      *
-     * @see TableFunctionProvider
-     * @see AsyncTableFunctionProvider
+     * @see LookupFunctionProvider
+     * @see AsyncLookupFunctionProvider
      */
     LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context);
 
@@ -97,8 +99,8 @@ public interface LookupTableSource extends DynamicTableSource {
      * <p>There exist different interfaces for runtime implementation which is why {@link
      * LookupRuntimeProvider} serves as the base interface.
      *
-     * @see TableFunctionProvider
-     * @see AsyncTableFunctionProvider
+     * @see LookupFunctionProvider
+     * @see AsyncLookupFunctionProvider
      */
     @PublicEvolving
     interface LookupRuntimeProvider {
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/TableFunctionProvider.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/TableFunctionProvider.java
index b338eda5e54..811523c255c 100644
--- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/TableFunctionProvider.java
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/TableFunctionProvider.java
@@ -19,6 +19,7 @@
 package org.apache.flink.table.connector.source;
 
 import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.functions.TableFunction;
 import org.apache.flink.table.functions.UserDefinedFunction;
@@ -36,8 +37,11 @@ import org.apache.flink.types.Row;
  * similar to other {@link UserDefinedFunction}s. However, for convenience, in a {@link
  * LookupTableSource} the output type can simply be a {@link Row} or {@link RowData} in which case
  * the input and output types are derived from the table's schema with default conversion.
+ *
+ * @deprecated Please use {@link LookupFunctionProvider} to implement synchronous lookup table.
  */
 @PublicEvolving
+@Deprecated
 public interface TableFunctionProvider<T> extends LookupTableSource.LookupRuntimeProvider {
 
     /** Helper method for creating a static provider. */
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/AsyncLookupFunctionProvider.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/AsyncLookupFunctionProvider.java
new file mode 100644
index 00000000000..a958bd5117b
--- /dev/null
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/AsyncLookupFunctionProvider.java
@@ -0,0 +1,36 @@
+/*
+ * 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.connector.source.lookup;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.functions.AsyncLookupFunction;
+
+/** A provider for creating {@link AsyncLookupFunction}. */
+@PublicEvolving
+public interface AsyncLookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider {
+
+    /** Helper function for creating a static provider. */
+    static AsyncLookupFunctionProvider of(AsyncLookupFunction asyncLookupFunction) {
+        return () -> asyncLookupFunction;
+    }
+
+    /** Creates an {@link AsyncLookupFunction} instance. */
+    AsyncLookupFunction createAsyncLookupFunction();
+}
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/LookupFunctionProvider.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/LookupFunctionProvider.java
new file mode 100644
index 00000000000..0f6383ac26f
--- /dev/null
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/lookup/LookupFunctionProvider.java
@@ -0,0 +1,36 @@
+/*
+ * 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.connector.source.lookup;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.functions.LookupFunction;
+
+/** A provider for creating {@link LookupFunction}. */
+@PublicEvolving
+public interface LookupFunctionProvider extends LookupTableSource.LookupRuntimeProvider {
+
+    /** Helper function for creating a static provider. */
+    static LookupFunctionProvider of(LookupFunction lookupFunction) {
+        return () -> lookupFunction;
+    }
+
+    /** Creates an {@link LookupFunction} instance. */
+    LookupFunction createLookupFunction();
+}
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/AsyncLookupFunction.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/AsyncLookupFunction.java
new file mode 100644
index 00000000000..5e58b646d3b
--- /dev/null
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/AsyncLookupFunction.java
@@ -0,0 +1,64 @@
+/*
+ * 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.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * A wrapper class of {@link AsyncTableFunction} for asynchronously lookup rows matching the lookup
+ * keys from external system.
+ *
+ * <p>The output type of this table function is fixed as {@link RowData}.
+ */
+@PublicEvolving
+public abstract class AsyncLookupFunction extends AsyncTableFunction<RowData> {
+
+    /**
+     * Asynchronously lookup rows matching the lookup keys.
+     *
+     * @param keyRow - A {@link RowData} that wraps lookup keys.
+     * @return A collection of all matching rows in the lookup table.
+     */
+    public abstract CompletableFuture<Collection<RowData>> asyncLookup(RowData keyRow);
+
+    /** Invokes {@link #asyncLookup} and chains futures. */
+    public final void eval(CompletableFuture<Collection<RowData>> future, Object... keys) {
+        GenericRowData keyRow = GenericRowData.of(keys);
+        asyncLookup(keyRow)
+                .whenCompleteAsync(
+                        (result, exception) -> {
+                            if (exception != null) {
+                                future.completeExceptionally(
+                                        new TableException(
+                                                String.format(
+                                                        "Failed to asynchronously lookup entries with key '%s'",
+                                                        keyRow),
+                                                exception));
+                                return;
+                            }
+                            future.complete(result);
+                        });
+    }
+}
diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/LookupFunction.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/LookupFunction.java
new file mode 100644
index 00000000000..bef18923589
--- /dev/null
+++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/LookupFunction.java
@@ -0,0 +1,59 @@
+/*
+ * 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.functions;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * A wrapper class of {@link TableFunction} for synchronously lookup rows matching the lookup keys
+ * from external system.
+ *
+ * <p>The output type of this table function is fixed as {@link RowData}.
+ */
+@PublicEvolving
+public abstract class LookupFunction extends TableFunction<RowData> {
+
+    /**
+     * Synchronously lookup rows matching the lookup keys.
+     *
+     * @param keyRow - A {@link RowData} that wraps lookup keys.
+     * @return A collection of all matching rows in the lookup table.
+     */
+    public abstract Collection<RowData> lookup(RowData keyRow) throws IOException;
+
+    /** Invoke {@link #lookup} and handle exceptions. */
+    public final void eval(Object... keys) {
+        GenericRowData keyRow = GenericRowData.of(keys);
+        try {
+            Collection<RowData> lookup = lookup(keyRow);
+            if (lookup == null) {
+                return;
+            }
+            lookup.forEach(this::collect);
+        } catch (IOException e) {
+            throw new RuntimeException(
+                    String.format("Failed to lookup values with given key row '%s'", keyRow), e);
+        }
+    }
+}