lihaosky commented on code in PR #27041:
URL: https://github.com/apache/flink/pull/27041#discussion_r2431030319
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMLPredictTableFunction.java:
##########
@@ -206,25 +200,15 @@ private Transformation<RowData> createModelPredict(
resultRowType,
mlPredictSpec.getFeatures(),
predictFunction,
- "MLPredict",
+
modelSpec.getContextResolvedModel().getIdentifier().asSummaryString(),
Review Comment:
Sorry didn't get it. Which required interface? Is it to improve error
message?
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/ml/AsyncMLPredictRunner.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.operators.ml;
+
+import org.apache.flink.api.common.functions.OpenContext;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.functions.async.CollectionSupplier;
+import org.apache.flink.streaming.api.functions.async.ResultFuture;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.JoinedRowData;
+import org.apache.flink.table.functions.AsyncPredictFunction;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.operators.AbstractAsyncFunctionRunner;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+
+/**
+ * Async function runner for {@link AsyncPredictFunction}, which takes the
generated function,
+ * instantiates it, and then calls its lifecycle methods.
+ */
+public class AsyncMLPredictRunner extends AbstractAsyncFunctionRunner<RowData>
{
+
+ private final int asyncBufferCapacity;
+
+ /**
+ * Buffers {@link ResultFuture} to avoid newInstance cost when processing
elements every time.
+ * We use {@link BlockingQueue} to make sure the head {@link
ResultFuture}s are available.
+ */
+ private transient BlockingQueue<JoinedRowResultFuture> resultFutureBuffer;
+
+ public AsyncMLPredictRunner(
+ GeneratedFunction<AsyncFunction<RowData, RowData>>
generatedFetcher,
+ int asyncBufferCapacity) {
+ super(generatedFetcher);
+ this.asyncBufferCapacity = asyncBufferCapacity;
+ }
+
+ @Override
+ public void open(OpenContext openContext) throws Exception {
+ super.open(openContext);
+ this.resultFutureBuffer = new ArrayBlockingQueue<>(asyncBufferCapacity
+ 1);
+ for (int i = 0; i < asyncBufferCapacity + 1; i++) {
+ JoinedRowResultFuture rf = new
JoinedRowResultFuture(resultFutureBuffer);
+ // add will throw exception immediately if the queue is full which
should never happen
+ resultFutureBuffer.add(rf);
+ }
+ registerMetric(getRuntimeContext().getMetricGroup());
+ }
+
+ @Override
+ public void asyncInvoke(RowData input, ResultFuture<RowData> resultFuture)
throws Exception {
+ try {
+ JoinedRowResultFuture buffer = resultFutureBuffer.take();
+ buffer.reset(input, resultFuture);
+ fetcher.asyncInvoke(input, buffer);
+ } catch (Throwable t) {
+ resultFuture.completeExceptionally(t);
+ }
+ }
+
+ private void registerMetric(MetricGroup metricGroup) {
+ metricGroup.gauge(
+ "ai_queue_length", () -> asyncBufferCapacity + 1 -
resultFutureBuffer.size());
Review Comment:
I mean `resultFutureBuffer.size()` is always equal to `asyncBufferCapacity +
1` from `open()`. So this is alway 0?
##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MLPredictCodeGenerator.scala:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.codegen
+
+import org.apache.flink.api.common.functions.FlatMapFunction
+import org.apache.flink.configuration.ReadableConfig
+import org.apache.flink.streaming.api.functions.async.AsyncFunction
+import org.apache.flink.table.catalog.DataTypeFactory
+import org.apache.flink.table.data.RowData
+import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction,
UserDefinedFunction}
+import
org.apache.flink.table.planner.codegen.FunctionCallCodeGenerator.GeneratedTableFunctionWithDataType
+import org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil
+import org.apache.flink.table.planner.functions.inference.FunctionCallContext
+import org.apache.flink.table.planner.plan.utils.FunctionCallUtil.FunctionParam
+import org.apache.flink.table.runtime.collector.ListenableCollector
+import org.apache.flink.table.runtime.generated.{GeneratedCollector,
GeneratedFunction}
+import org.apache.flink.table.types.inference.{TypeInference, TypeStrategies,
TypeTransformations}
+import org.apache.flink.table.types.logical.{LogicalType, RowType}
+import org.apache.flink.table.types.utils.DataTypeUtils.transform
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+object MLPredictCodeGenerator {
+
+ /** Generates a predict function ([[TableFunction]]) */
+ def generateSyncPredictFunction(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ predictFunctionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ features: util.List[FunctionParam],
+ syncPredictFunction: TableFunction[_],
+ functionName: String,
+ fieldCopy: Boolean
+ ): GeneratedFunction[FlatMapFunction[RowData, RowData]] = {
+ FunctionCallCodeGenerator
+ .generateSyncFunctionCall(
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ predictFunctionOutputType,
+ collectorOutputType,
+ features,
+ syncPredictFunction,
+ generateCallWithDataType(functionName, predictFunctionOutputType),
+ functionName,
+ "PredictFunction",
+ fieldCopy
+ )
+ .tableFunc
+ }
+
+ /** Generates a async predict function ([[AsyncTableFunction]]) */
+ def generateAsyncPredictFunction(
+ tableConfig: ReadableConfig,
+ classLoader: ClassLoader,
+ dataTypeFactory: DataTypeFactory,
+ inputType: LogicalType,
+ predictFunctionOutputType: LogicalType,
+ collectorOutputType: LogicalType,
+ features: util.List[FunctionParam],
+ asyncPredictFunction: AsyncTableFunction[_],
+ functionName: String):
GeneratedTableFunctionWithDataType[AsyncFunction[RowData, AnyRef]] = {
+ FunctionCallCodeGenerator.generateAsyncFunctionCall(
+ tableConfig,
+ classLoader,
+ dataTypeFactory,
+ inputType,
+ predictFunctionOutputType,
+ collectorOutputType,
+ features,
+ asyncPredictFunction,
+ generateCallWithDataType(functionName, predictFunctionOutputType),
+ functionName,
+ "AsyncPredictFunction"
+ )
+ }
+
+ /** Generate a collector to collect to join the input row and predicted
results. */
+ def generateCollector(
+ ctx: CodeGeneratorContext,
+ inputRowType: RowType,
+ predictFunctionOutputType: RowType,
+ collectorOutputType: RowType
+ ): GeneratedCollector[ListenableCollector[RowData]] = {
+ FunctionCallCodeGenerator.generateCollector(
+ ctx,
+ inputRowType,
+ predictFunctionOutputType,
+ collectorOutputType,
+ Option.empty,
+ Option.empty
+ )
+ }
+
+ private def generateCallWithDataType(
+ functionName: String,
+ modelOutputType: LogicalType
+ ) = (
+ ctx: CodeGeneratorContext,
+ callContext: FunctionCallContext,
+ udf: UserDefinedFunction,
+ operands: Seq[GeneratedExpression]) => {
+ val inference = TypeInference
Review Comment:
ml_predict has type inference defined already?
https://github.com/apache/flink/blob/master/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java#L740-L754
--
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]