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


##########
flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java:
##########
@@ -632,7 +633,9 @@ private static Stream<TestSpec> functionSpecs() {
                 
TestSpec.forScalarFunction(FunctionHintTableArgScalarFunction.class)
                         .expectErrorMessage("Only scalar arguments are 
supported yet."),
                 
TestSpec.forScalarFunction(ArgumentHintTableArgScalarFunction.class)
-                        .expectErrorMessage("Only scalar arguments are 
supported yet."));
+                        .expectErrorMessage("Only scalar arguments are 
supported yet."),

Review Comment:
   You don't change the message, but I spotted the sentence is not 
grammatically correct. Do you think we could fix it?



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/annotation/StateHint.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.annotation;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.functions.AggregateFunction;
+import org.apache.flink.table.functions.ProcessTableFunction;
+import org.apache.flink.table.functions.TableAggregateFunction;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * A hint that declares an intermediate result (i.e. state entry) that is 
managed by the framework
+ * (i.e. Flink managed state).
+ *
+ * <p>State hints are primarily intended for {@link ProcessTableFunction}. A 
PTF supports multiple
+ * state entries at the beginning of an eval()/onTimer() method (after an 
optional context
+ * parameter).
+ *
+ * <p>Aggregating functions (i.e. {@link AggregateFunction} and {@link 
TableAggregateFunction})
+ * support a single state entry at the beginning of an accumulate()/retract() 
method (i.e. the
+ * accumulator).
+ *
+ * <p>For example, {@code @StateHint(name = "count", type = 
@DataTypeHint("BIGINT"))} is a state
+ * entry with the data type BIGINT named "count".
+ *
+ * <p>Note: Usually, a state entry is partitioned by a key and can not be 
accessed globally. The
+ * partitioning (or whether it is only a single partition) is defined by the 
corresponding function
+ * call.
+ *
+ * @see FunctionHint
+ */
+@PublicEvolving
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE, ElementType.METHOD, ElementType.PARAMETER})
+public @interface StateHint {
+
+    /**
+     * The name of the state entry.
+     *
+     * <p>This can be used to provide a descriptive name for the state entry. 
The name can be used
+     * for referencing the entry during clean up.
+     */
+    String name() default "";

Review Comment:
   Are there any uniqueness requirements? If so can we write them explicitly?



##########
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:
   Just writing out my thoughts...
   
   Is it a good idea to introduce a new term here? What is a "virtual 
processor"? How does it relate to Flink's TMs and/or Tasks? Maybe I am biased, 
but I find it raises more questions then provides answers.



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/TableSemantics.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.ArgumentTrait;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.types.DataType;
+
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Provides call information about the table that has been passed to a table 
argument.
+ *
+ * <p>This class if only available for table arguments (i.e. arguments of a 
{@link

Review Comment:
   ```suggestion
    * <p>This class is only available for table arguments (i.e. arguments of a 
{@link
   ```



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