| /* |
| * 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.api.functions; |
| |
| import org.apache.flink.table.api.types.DataType; |
| import org.apache.flink.util.Collector; |
| |
| /** |
| * Base class for a user-defined table function (UDTF). A user-defined table functions works on |
| * zero, one, or multiple scalar values as input and returns multiple rows as output. |
| * |
| * <p>The behavior of a {@link TableFunction} can be defined by implementing a custom evaluation |
| * method. An evaluation method must be declared publicly, not static and named "eval". |
| * Evaluation methods can also be overloaded by implementing multiple methods named "eval". |
| * |
| * <p>User-defined functions must have a default constructor and must be instantiable during |
| * runtime. |
| * |
| * <p>By default the result type of an evaluation method is determined by Flink's type extraction |
| * facilities. This is sufficient for basic types or simple POJOs but might be wrong for more |
| * complex, custom, or composite types. In these cases {@link DataType} of the result type |
| * can be manually defined by overriding {@link #getResultType}. |
| * |
| * <p>Internally, the Table/SQL API code generation works with primitive values as much as possible. |
| * If a user-defined table function should not introduce much overhead during runtime, it is |
| * recommended to declare parameters and result types as primitive types instead of their boxed |
| * classes. DATE/TIME is equal to int, TIMESTAMP is equal to long. |
| * |
| * <p>Example: |
| * |
| * {@code |
| * |
| * public class Split extends TableFunction<String> { |
| * |
| * // implement an "eval" method with as many parameters as you want |
| * public void eval(String str) { |
| * for (String s : str.split(" ")) { |
| * collect(s); // use collect(...) to emit an output row |
| * } |
| * } |
| * |
| * // you can overload the eval method here ... |
| * } |
| * |
| * val tEnv: TableEnvironment = ... |
| * val table: Table = ... // schema: [a: String] |
| * |
| * // for Scala users |
| * val split = new Split() |
| * table.join(split('c) as ('s)).select('a, 's) |
| * |
| * // for Java users |
| * tEnv.registerFunction("split", new Split()) // register table function first |
| * table.join(new Table(tEnv, "split(a) as (s)")).select("a, s") |
| * |
| * // for SQL users |
| * tEnv.registerFunction("split", new Split()) // register table function first |
| * tEnv.sql("SELECT a, s FROM MyTable, LATERAL TABLE(split(a)) as T(s)") |
| * |
| * } |
| * |
| * @param <T> The type of the output row |
| */ |
| public abstract class TableFunction<T> extends CustomTypeDefinedFunction { |
| |
| /** |
| * Emit an output row. |
| * |
| * @param row the output row |
| */ |
| public void collect(T row) { |
| collector.collect(row); |
| } |
| |
| // ---------------------------------------------------------------------------------------------- |
| |
| /** |
| * The code generated collector used to emit row. |
| */ |
| private Collector<T> collector = null; |
| |
| /** |
| * Internal use. Sets the current collector. |
| */ |
| public final void setCollector(Collector<T> collector) { |
| this.collector = collector; |
| } |
| } |