twalthr commented on code in PR #25707:
URL: https://github.com/apache/flink/pull/25707#discussion_r1862114539


##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/ProcessTableFunction.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.annotation.ArgumentHint;
+import org.apache.flink.table.annotation.DataTypeHint;
+import org.apache.flink.table.annotation.FunctionHint;
+import org.apache.flink.table.catalog.DataTypeFactory;
+import org.apache.flink.table.types.inference.TypeInference;
+import org.apache.flink.util.Collector;
+
+/**
+ * Base class for a user-defined process table function. A process table 
function (PTF) maps zero,
+ * one, or multiple tables to zero, one, or multiple rows (or structured 
types). Scalar arguments
+ * are also supported. If the output record consists of only one field, the 
wrapper can be omitted,
+ * and a scalar value can be emitted that will be implicitly wrapped into a 
row by the runtime.
+ *
+ * <p>PTFs are the most powerful function kind for Flink SQL and Table API. 
They enable implementing
+ * user-defined operators that can be as feature-rich as built-in operations. 
PTFs have access to
+ * Flink's managed state, event-time and timer services, underlying table 
changelogs, and can take
+ * multiple ordered and/or partitioned tables to produce a new table.
+ *
+ * <h1>Table Semantics and Virtual Processors</h1>
+ *
+ * <p>PTFs can produce a new table by consuming tables as arguments. For 
scalability, input tables
+ * are distributed into virtual processors. Each virtual processor executes a 
PTF instance and has

Review Comment:
   A virtual processor could be translated into the Flink concept "key context" 
but this is rather an internal term. Virtual processor is neither TM, nor Task, 
nor Subtask. And I think this is also what confuses people when they work with 
KeyedProcessFunction for the first time. Understanding that state is scoped 
under the key.



-- 
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

Reply via email to