dawidwys commented on code in PR #26331:
URL: https://github.com/apache/flink/pull/26331#discussion_r2007445551
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java:
##########
@@ -965,6 +967,74 @@ void createTemporarySystemFunction(
*/
Table from(TableDescriptor descriptor);
+ /**
+ * Returns a {@link Table} backed by a call to a process table function
(PTF).
+ *
+ * <p>A PTF maps zero, one, or multiple tables to a new table. 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.
Review Comment:
Just confirming. We don't support ordering yet?
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/FunctionQueryOperation.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.operations;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.catalog.ContextResolvedFunction;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.TableReferenceExpression;
+import org.apache.flink.table.types.DataType;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/** Describes a relational operation that was created from applying a
(process) table function. */
+@Internal
+public class FunctionQueryOperation implements QueryOperation {
+
+ private static final String INPUT_ALIAS = "$$T_FUNC";
+
+ private final ContextResolvedFunction resolvedFunction;
+ private final List<ResolvedExpression> arguments;
+ private final ResolvedSchema resolvedSchema;
+
+ public FunctionQueryOperation(
+ ContextResolvedFunction resolvedFunction,
+ List<ResolvedExpression> arguments,
+ ResolvedSchema resolvedSchema) {
+ this.resolvedFunction = resolvedFunction;
+ this.arguments = arguments;
+ this.resolvedSchema = resolvedSchema;
+ }
+
+ public ContextResolvedFunction getResolvedFunction() {
+ return resolvedFunction;
+ }
+
+ public List<ResolvedExpression> getArguments() {
+ return arguments;
+ }
+
+ public DataType getOutputDataType() {
+ // Make sure time attributes are not erased
+ final List<String> fieldNames = resolvedSchema.getColumnNames();
+ final List<DataType> fieldTypes = resolvedSchema.getColumnDataTypes();
+ return DataTypes.ROW(
+ IntStream.range(0, fieldNames.size())
+ .mapToObj(
+ pos ->
+ DataTypes.FIELD(
+ fieldNames.get(pos),
fieldTypes.get(pos)))
+ .collect(Collectors.toList()))
+ .notNull();
Review Comment:
Could we extract that to `ResolvedSchema` or at least a util class? We do
that at least twice.
##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java:
##########
@@ -295,6 +297,17 @@ public String asSerializableString() {
duration.toMinutes() % 60,
duration.getSeconds() % 60,
duration.getNano() / 1_000_000);
+ case DESCRIPTOR:
+ final ColumnList columnList =
getValueAs(ColumnList.class).get();
+ if (!columnList.getDataTypes().isEmpty()) {
+ throw new TableException("Data types in DESCRIPTOR are not
supported yet.");
Review Comment:
I believe this answers one of my previous questions 😅
##########
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java:
##########
@@ -131,17 +134,28 @@ public static ApiExpression lit(Object v) {
}
/**
- * Creates a SQL literal of a given {@link DataType}.
+ * Creates a literal (i.e. a constant value) of a given {@link DataType}.
*
* <p>The method {@link #lit(Object)} is preferred as it extracts the
{@link DataType}
* automatically. Use this method only when necessary. The class of {@code
v} must be supported
- * according to the {@link
- *
org.apache.flink.table.types.logical.LogicalType#supportsInputConversion(Class)}.
+ * according to the {@link LogicalType#supportsInputConversion(Class)}.
*/
public static ApiExpression lit(Object v, DataType dataType) {
return new ApiExpression(valueLiteral(v, dataType));
}
+ /**
+ * Creates a literal describing an arbitrary, unvalidated list of column
names.
+ *
+ * <p>Passing a list of columns can be useful for parameterizing a
function. In particular, it
+ * enables declaring the {@code on_time} argument for {@link
ProcessTableFunction}.
+ *
+ * <p>The data type will be {@link DataTypes#DESCRIPTOR()}.
+ */
+ public static ApiExpression descriptor(String... columnNames) {
Review Comment:
probably just a question. Do we need a method with datatypes as well?
Equivalent to: `DESCRIPTOR(a INT, b STRING, c BOOLEAN)`
--
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]