davidradl commented on code in PR #27130:
URL: https://github.com/apache/flink/pull/27130#discussion_r2444269447


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecVectorSearchTableFunction.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.plan.nodes.exec.stream;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.PipelineOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.streaming.api.operators.ProcessOperator;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.catalog.DataTypeFactory;
+import org.apache.flink.table.connector.source.VectorSearchTableSource;
+import 
org.apache.flink.table.connector.source.search.AsyncVectorSearchFunctionProvider;
+import 
org.apache.flink.table.connector.source.search.VectorSearchFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.functions.UserDefinedFunctionHelper;
+import org.apache.flink.table.functions.VectorSearchFunction;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.VectorSearchCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import 
org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.VectorSearchSpec;
+import 
org.apache.flink.table.planner.plan.nodes.exec.spec.VectorSearchTableSourceSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.FunctionCallUtil;
+import org.apache.flink.table.planner.plan.utils.VectorSearchUtil;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.collector.ListenableCollector;
+import org.apache.flink.table.runtime.generated.GeneratedCollector;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.operators.search.VectorSearchRunner;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.core.JoinRelType;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collections;
+
+/** Stream {@link ExecNode} for {@code VECTOR_SEARCH}. */
+public class StreamExecVectorSearchTableFunction extends ExecNodeBase<RowData>
+        implements MultipleTransformationTranslator<RowData>, 
StreamExecNode<RowData> {
+
+    public static final String VECTOR_SEARCH_TRANSFORMATION = 
"vector-search-table-function";
+    private final VectorSearchTableSourceSpec vectorSearchTableSourceSpec;
+    private final VectorSearchSpec vectorSearchSpec;
+    private final @Nullable FunctionCallUtil.AsyncOptions asyncOptions;
+
+    public StreamExecVectorSearchTableFunction(
+            ReadableConfig tableConfig,
+            VectorSearchTableSourceSpec vectorSearchTableSourceSpec,
+            VectorSearchSpec vectorSearchSpec,
+            @Nullable FunctionCallUtil.AsyncOptions asyncOptions,
+            InputProperty inputProperty,
+            RowType outputType,
+            String description) {
+        super(
+                ExecNodeContext.newNodeId(),
+                
ExecNodeContext.newContext(StreamExecVectorSearchTableFunction.class),
+                ExecNodeContext.newPersistedConfig(
+                        StreamExecVectorSearchTableFunction.class, 
tableConfig),
+                Collections.singletonList(inputProperty),
+                outputType,
+                description);
+        this.vectorSearchTableSourceSpec = vectorSearchTableSourceSpec;
+        this.vectorSearchSpec = vectorSearchSpec;
+        this.asyncOptions = asyncOptions;
+    }
+
+    @Override
+    protected Transformation<RowData> translateToPlanInternal(
+            PlannerBase planner, ExecNodeConfig config) {
+        // 1. translate input node
+        ExecEdge inputEdge = getInputEdges().get(0);
+        Transformation<RowData> inputTransformation =
+                (Transformation<RowData>) inputEdge.translateToPlan(planner);
+        // 2. extract search function
+        TableSourceTable searchTable = 
vectorSearchTableSourceSpec.getSearchTable();
+        boolean isAsyncEnabled = asyncOptions != null;
+        UserDefinedFunction vectorSearchFunction =
+                findVectorSearchFunction(
+                        VectorSearchUtil.createVectorSearchRuntimeProvider(
+                                searchTable, 
vectorSearchSpec.getSearchColumns().keySet()),
+                        isAsyncEnabled);
+        UserDefinedFunctionHelper.prepareInstance(config, 
vectorSearchFunction);
+        // 3. build the operator
+        RowType inputType = (RowType) inputEdge.getOutputType();
+        RowType outputType = (RowType) getOutputType();
+        StreamOperatorFactory<RowData> operatorFactory =
+                isAsyncEnabled
+                        ? createAsyncVectorSearchOperator()

Review Comment:
   I am curious what this is doing , there is one implementation that throws an 
Exception. I thought this change is adding options for the async search.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to