Concepts & Common API

The Table API and SQL are integrated in a joint API. The central concept of this API is a Table which serves as input and output of queries. This document shows the common structure of programs with Table API and SQL queries, how to register a Table, how to query a Table, and how to emit a Table.

Structure of Table API and SQL Programs

The following code example shows the common structure of Table API and SQL programs.

Java

  1. // create a TableEnvironment for batch or streaming execution
  2. TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // create an input Table
  4. tableEnv.executeSql("CREATE TEMPORARY TABLE table1 ... WITH ( 'connector' = ... )");
  5. // register an output Table
  6. tableEnv.executeSql("CREATE TEMPORARY TABLE outputTable ... WITH ( 'connector' = ... )");
  7. // create a Table object from a Table API query
  8. Table table2 = tableEnv.from("table1").select(...);
  9. // create a Table object from a SQL query
  10. Table table3 = tableEnv.sqlQuery("SELECT ... FROM table1 ... ");
  11. // emit a Table API result Table to a TableSink, same for SQL result
  12. TableResult tableResult = table2.executeInsert("outputTable");
  13. tableResult...

Scala

  1. // create a TableEnvironment for batch or streaming execution
  2. val tableEnv = ... // see "Create a TableEnvironment" section
  3. // create an input Table
  4. tableEnv.executeSql("CREATE TEMPORARY TABLE table1 ... WITH ( 'connector' = ... )")
  5. // register an output Table
  6. tableEnv.executeSql("CREATE TEMPORARY TABLE outputTable ... WITH ( 'connector' = ... )")
  7. // create a Table from a Table API query
  8. val table2 = tableEnv.from("table1").select(...)
  9. // create a Table from a SQL query
  10. val table3 = tableEnv.sqlQuery("SELECT ... FROM table1 ...")
  11. // emit a Table API result Table to a TableSink, same for SQL result
  12. val tableResult = table2.executeInsert("outputTable")
  13. tableResult...

Python

  1. # create a TableEnvironment for batch or streaming execution
  2. table_env = ... # see "Create a TableEnvironment" section
  3. # register an input Table
  4. table_env.executeSql("CREATE TEMPORARY TABLE table1 ... WITH ( 'connector' = ... )")
  5. # register an output Table
  6. table_env.executeSql("CREATE TEMPORARY TABLE outputTable ... WITH ( 'connector' = ... )")
  7. # create a Table from a Table API query
  8. table2 = table_env.from_path("table1").select(...)
  9. # create a Table from a SQL query
  10. table3 = table_env.sql_query("SELECT ... FROM table1 ...")
  11. # emit a Table API result Table to a TableSink, same for SQL result
  12. table_result = table3.execute_insert("outputTable")
  13. table_result...

Table API and SQL queries can be easily integrated with and embedded into DataStream programs. Have a look at the DataStream API Integration page to learn how DataStreams can be converted into Tables and vice versa.

Create a TableEnvironment

The TableEnvironment is the entrypoint for Table API and SQL integration and is responsible for:

  • Registering a Table in the internal catalog
  • Registering catalogs
  • Loading pluggable modules
  • Executing SQL queries
  • Registering a user-defined (scalar, table, or aggregation) function
  • Converting between DataStream and Table (in case of StreamTableEnvironment)

A Table is always bound to a specific TableEnvironment. It is not possible to combine tables of different TableEnvironments in the same query, e.g., to join or union them. A TableEnvironment is created by calling the static TableEnvironment.create() method.

Java

  1. import org.apache.flink.table.api.EnvironmentSettings;
  2. import org.apache.flink.table.api.TableEnvironment;
  3. EnvironmentSettings settings = EnvironmentSettings
  4. .newInstance()
  5. .inStreamingMode()
  6. //.inBatchMode()
  7. .build();
  8. TableEnvironment tEnv = TableEnvironment.create(setting);

Scala

  1. import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment}
  2. val settings = EnvironmentSettings
  3. .newInstance()
  4. .inStreamingMode()
  5. //.inBatchMode()
  6. .build()
  7. val tEnv = TableEnvironment.create(setting)

Python

  1. from pyflink.table import EnvironmentSettings, TableEnvironment
  2. # create a blink streaming TableEnvironment
  3. env_settings = EnvironmentSettings.new_instance().use_blink_planner().build()
  4. table_env = TableEnvironment.create(env_settings)
  5. # create a blink batch TableEnvironment
  6. env_settings = EnvironmentSettings.new_instance().in_batch_mode().build()
  7. table_env = TableEnvironment.create(env_settings)

Alternatively, users can create a StreamTableEnvironment from an existing StreamExecutionEnvironment to interoperate with the DataStream API.

Java

  1. import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  2. import org.apache.flink.table.api.EnvironmentSettings;
  3. import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
  4. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  5. StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);

Scala

  1. import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
  2. import org.apache.flink.table.api.EnvironmentSettings
  3. import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment
  4. val env = StreamExecutionEnvironment.getExecutionEnvironment
  5. val tEnv = StreamTableEnvironment.create(env)

Python

  1. from pyflink.datastream import StreamExecutionEnvironment
  2. from pyflink.table import StreamTableEnvironment
  3. s_env = StreamExecutionEnvironment.get_execution_environment()
  4. t_env = StreamTableEnvironment.create(s_env)

Create Tables in the Catalog

A TableEnvironment maintains a map of catalogs of tables which are created with an identifier. Each identifier consists of 3 parts: catalog name, database name and object name. If a catalog or database is not specified, the current default value will be used (see examples in the Table identifier expanding section).

Tables can be either virtual (VIEWS) or regular (TABLES). VIEWS can be created from an existing Table object, usually the result of a Table API or SQL query. TABLES describe external data, such as a file, database table, or message queue.

Temporary vs Permanent tables.

Tables may either be temporary, and tied to the lifecycle of a single Flink session, or permanent, and visible across multiple Flink sessions and clusters.

Permanent tables require a catalog (such as Hive Metastore) to maintain metadata about the table. Once a permanent table is created, it is visible to any Flink session that is connected to the catalog and will continue to exist until the table is explicitly dropped.

On the other hand, temporary tables are always stored in memory and only exist for the duration of the Flink session they are created within. These tables are not visible to other sessions. They are not bound to any catalog or database but can be created in the namespace of one. Temporary tables are not dropped if their corresponding database is removed.

Shadowing

It is possible to register a temporary table with the same identifier as an existing permanent table. The temporary table shadows the permanent one and makes the permanent table inaccessible as long as the temporary one exists. All queries with that identifier will be executed against the temporary table.

This might be useful for experimentation. It allows running exactly the same query first against a temporary table that e.g. has just a subset of data, or the data is obfuscated. Once verified that the query is correct it can be run against the real production table.

Create a Table

Virtual Tables

A Table API object corresponds to a VIEW (virtual table) in a SQL terms. It encapsulates a logical query plan. It can be created in a catalog as follows:

Java

  1. // get a TableEnvironment
  2. TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // table is the result of a simple projection query
  4. Table projTable = tableEnv.from("X").select(...);
  5. // register the Table projTable as table "projectedTable"
  6. tableEnv.createTemporaryView("projectedTable", projTable);

Scala

  1. // get a TableEnvironment
  2. val tableEnv = ... // see "Create a TableEnvironment" section
  3. // table is the result of a simple projection query
  4. val projTable: Table = tableEnv.from("X").select(...)
  5. // register the Table projTable as table "projectedTable"
  6. tableEnv.createTemporaryView("projectedTable", projTable)

Python

  1. # get a TableEnvironment
  2. table_env = ... # see "Create a TableEnvironment" section
  3. # table is the result of a simple projection query
  4. proj_table = table_env.from_path("X").select(...)
  5. # register the Table projTable as table "projectedTable"
  6. table_env.register_table("projectedTable", proj_table)

Note: Table objects are similar to VIEW’s from relational database systems, i.e., the query that defines the Table is not optimized but will be inlined when another query references the registered Table. If multiple queries reference the same registered Table, it will be inlined for each referencing query and executed multiple times, i.e., the result of the registered Table will not be shared.

Connector Tables

It is also possible to create a TABLE as known from relational databases from a connector declaration. The connector describes the external system that stores the data of a table. Storage systems such as Apache Kafka or a regular file system can be declared here.

  1. tableEnvironment.executeSql("CREATE [TEMPORARY] TABLE MyTable (...) WITH (...)")

Expanding Table identifiers

Tables are always registered with a 3-part identifier consisting of catalog, database, and table name.

Users can set one catalog and one database inside it to be the “current catalog” and “current database”. With them, the first two parts in the 3-parts identifier mentioned above can be optional - if they are not provided, the current catalog and current database will be referred. Users can switch the current catalog and current database via table API or SQL.

Identifiers follow SQL requirements which means that they can be escaped with a backtick character (` ).

Java

  1. TableEnvironment tEnv = ...;
  2. tEnv.useCatalog("custom_catalog");
  3. tEnv.useDatabase("custom_database");
  4. Table table = ...;
  5. // register the view named 'exampleView' in the catalog named 'custom_catalog'
  6. // in the database named 'custom_database'
  7. tableEnv.createTemporaryView("exampleView", table);
  8. // register the view named 'exampleView' in the catalog named 'custom_catalog'
  9. // in the database named 'other_database'
  10. tableEnv.createTemporaryView("other_database.exampleView", table);
  11. // register the view named 'example.View' in the catalog named 'custom_catalog'
  12. // in the database named 'custom_database'
  13. tableEnv.createTemporaryView("`example.View`", table);
  14. // register the view named 'exampleView' in the catalog named 'other_catalog'
  15. // in the database named 'other_database'
  16. tableEnv.createTemporaryView("other_catalog.other_database.exampleView", table);

Scala

  1. // get a TableEnvironment
  2. val tEnv: TableEnvironment = ...;
  3. tEnv.useCatalog("custom_catalog")
  4. tEnv.useDatabase("custom_database")
  5. val table: Table = ...;
  6. // register the view named 'exampleView' in the catalog named 'custom_catalog'
  7. // in the database named 'custom_database'
  8. tableEnv.createTemporaryView("exampleView", table)
  9. // register the view named 'exampleView' in the catalog named 'custom_catalog'
  10. // in the database named 'other_database'
  11. tableEnv.createTemporaryView("other_database.exampleView", table)
  12. // register the view named 'example.View' in the catalog named 'custom_catalog'
  13. // in the database named 'custom_database'
  14. tableEnv.createTemporaryView("`example.View`", table)
  15. // register the view named 'exampleView' in the catalog named 'other_catalog'
  16. // in the database named 'other_database'
  17. tableEnv.createTemporaryView("other_catalog.other_database.exampleView", table)

Query a Table

Table API

The Table API is a language-integrated query API for Scala and Java. In contrast to SQL, queries are not specified as Strings but are composed step-by-step in the host language.

The API is based on the Table class which represents a table (streaming or batch) and offers methods to apply relational operations. These methods return a new Table object, which represents the result of applying the relational operation on the input Table. Some relational operations are composed of multiple method calls such as table.groupBy(...).select(), where groupBy(...) specifies a grouping of table, and select(...) the projection on the grouping of table.

The Table API document describes all Table API operations that are supported on streaming and batch tables.

The following example shows a simple Table API aggregation query:

Java

  1. // get a TableEnvironment
  2. TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // register Orders table
  4. // scan registered Orders table
  5. Table orders = tableEnv.from("Orders");
  6. // compute revenue for all customers from France
  7. Table revenue = orders
  8. .filter($("cCountry").isEqual("FRANCE"))
  9. .groupBy($("cID"), $("cName"))
  10. .select($("cID"), $("cName"), $("revenue").sum().as("revSum"));
  11. // emit or convert Table
  12. // execute query

Scala

  1. // get a TableEnvironment
  2. val tableEnv = ... // see "Create a TableEnvironment" section
  3. // register Orders table
  4. // scan registered Orders table
  5. val orders = tableEnv.from("Orders")
  6. // compute revenue for all customers from France
  7. val revenue = orders
  8. .filter($"cCountry" === "FRANCE")
  9. .groupBy($"cID", $"cName")
  10. .select($"cID", $"cName", $"revenue".sum AS "revSum")
  11. // emit or convert Table
  12. // execute query

Note: The Scala Table API uses Scala String interpolation that starts with a dollar sign ($) to reference the attributes of a Table. The Table API uses Scala implicits. Make sure to import

  • org.apache.flink.table.api._ - for implicit expression conversions
  • org.apache.flink.api.scala._ and org.apache.flink.table.api.bridge.scala._ if you want to convert from/to DataStream.

    Python

  1. # get a TableEnvironment
  2. table_env = # see "Create a TableEnvironment" section
  3. # register Orders table
  4. # scan registered Orders table
  5. orders = table_env.from_path("Orders")
  6. # compute revenue for all customers from France
  7. revenue = orders \
  8. .filter(orders.cCountry == 'FRANCE') \
  9. .group_by(orders.cID, orders.cName) \
  10. .select(orders.cID, orders.cName, orders.revenue.sum.alias('revSum'))
  11. # emit or convert Table
  12. # execute query

SQL

Flink’s SQL integration is based on Apache Calcite, which implements the SQL standard. SQL queries are specified as regular Strings.

The SQL document describes Flink’s SQL support for streaming and batch tables.

The following example shows how to specify a query and return the result as a Table.

Java

  1. // get a TableEnvironment
  2. TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // register Orders table
  4. // compute revenue for all customers from France
  5. Table revenue = tableEnv.sqlQuery(
  6. "SELECT cID, cName, SUM(revenue) AS revSum " +
  7. "FROM Orders " +
  8. "WHERE cCountry = 'FRANCE' " +
  9. "GROUP BY cID, cName"
  10. );
  11. // emit or convert Table
  12. // execute query

Scala

  1. // get a TableEnvironment
  2. val tableEnv = ... // see "Create a TableEnvironment" section
  3. // register Orders table
  4. // compute revenue for all customers from France
  5. val revenue = tableEnv.sqlQuery("""
  6. |SELECT cID, cName, SUM(revenue) AS revSum
  7. |FROM Orders
  8. |WHERE cCountry = 'FRANCE'
  9. |GROUP BY cID, cName
  10. """.stripMargin)
  11. // emit or convert Table
  12. // execute query

Python

  1. # get a TableEnvironment
  2. table_env = ... # see "Create a TableEnvironment" section
  3. # register Orders table
  4. # compute revenue for all customers from France
  5. revenue = table_env.sql_query(
  6. "SELECT cID, cName, SUM(revenue) AS revSum "
  7. "FROM Orders "
  8. "WHERE cCountry = 'FRANCE' "
  9. "GROUP BY cID, cName"
  10. )
  11. # emit or convert Table
  12. # execute query

The following example shows how to specify an update query that inserts its result into a registered table.

Java

  1. // get a TableEnvironment
  2. TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // register "Orders" table
  4. // register "RevenueFrance" output table
  5. // compute revenue for all customers from France and emit to "RevenueFrance"
  6. tableEnv.executeSql(
  7. "INSERT INTO RevenueFrance " +
  8. "SELECT cID, cName, SUM(revenue) AS revSum " +
  9. "FROM Orders " +
  10. "WHERE cCountry = 'FRANCE' " +
  11. "GROUP BY cID, cName"
  12. );

Scala

  1. // get a TableEnvironment
  2. val tableEnv = ... // see "Create a TableEnvironment" section
  3. // register "Orders" table
  4. // register "RevenueFrance" output table
  5. // compute revenue for all customers from France and emit to "RevenueFrance"
  6. tableEnv.executeSql("""
  7. |INSERT INTO RevenueFrance
  8. |SELECT cID, cName, SUM(revenue) AS revSum
  9. |FROM Orders
  10. |WHERE cCountry = 'FRANCE'
  11. |GROUP BY cID, cName
  12. """.stripMargin)

Python

  1. # get a TableEnvironment
  2. table_env = ... # see "Create a TableEnvironment" section
  3. # register "Orders" table
  4. # register "RevenueFrance" output table
  5. # compute revenue for all customers from France and emit to "RevenueFrance"
  6. table_env.execute_sql(
  7. "INSERT INTO RevenueFrance "
  8. "SELECT cID, cName, SUM(revenue) AS revSum "
  9. "FROM Orders "
  10. "WHERE cCountry = 'FRANCE' "
  11. "GROUP BY cID, cName"
  12. )

Mixing Table API and SQL

Table API and SQL queries can be easily mixed because both return Table objects:

  • A Table API query can be defined on the Table object returned by a SQL query.
  • A SQL query can be defined on the result of a Table API query by registering the resulting Table in the TableEnvironment and referencing it in the FROM clause of the SQL query.

Emit a Table

A Table is emitted by writing it to a TableSink. A TableSink is a generic interface to support a wide variety of file formats (e.g. CSV, Apache Parquet, Apache Avro), storage systems (e.g., JDBC, Apache HBase, Apache Cassandra, Elasticsearch), or messaging systems (e.g., Apache Kafka, RabbitMQ).

A batch Table can only be written to a BatchTableSink, while a streaming Table requires either an AppendStreamTableSink, a RetractStreamTableSink, or an UpsertStreamTableSink.

Please see the documentation about Table Sources & Sinks for details about available sinks and instructions for how to implement a custom DynamicTableSink.

The Table.executeInsert(String tableName) method emits the Table to a registered TableSink. The method looks up the TableSink from the catalog by the name and validates that the schema of the Table is identical to the schema of the TableSink.

The following examples shows how to emit a Table:

Java

  1. // get a TableEnvironment
  2. TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // create an output Table
  4. final Schema schema = new Schema()
  5. .field("a", DataTypes.INT())
  6. .field("b", DataTypes.STRING())
  7. .field("c", DataTypes.BIGINT());
  8. tableEnv.connect(new FileSystem().path("/path/to/file"))
  9. .withFormat(new Csv().fieldDelimiter('|').deriveSchema())
  10. .withSchema(schema)
  11. .createTemporaryTable("CsvSinkTable");
  12. // compute a result Table using Table API operators and/or SQL queries
  13. Table result = ...
  14. // emit the result Table to the registered TableSink
  15. result.executeInsert("CsvSinkTable");

Scala

  1. // get a TableEnvironment
  2. val tableEnv = ... // see "Create a TableEnvironment" section
  3. // create an output Table
  4. val schema = new Schema()
  5. .field("a", DataTypes.INT())
  6. .field("b", DataTypes.STRING())
  7. .field("c", DataTypes.BIGINT())
  8. tableEnv.connect(new FileSystem().path("/path/to/file"))
  9. .withFormat(new Csv().fieldDelimiter('|').deriveSchema())
  10. .withSchema(schema)
  11. .createTemporaryTable("CsvSinkTable")
  12. // compute a result Table using Table API operators and/or SQL queries
  13. val result: Table = ...
  14. // emit the result Table to the registered TableSink
  15. result.executeInsert("CsvSinkTable")

Python

  1. # get a TableEnvironment
  2. table_env = ... # see "Create a TableEnvironment" section
  3. # create a TableSink
  4. table_env.connect(FileSystem().path("/path/to/file")))
  5. .with_format(Csv()
  6. .field_delimiter(',')
  7. .deriveSchema())
  8. .with_schema(Schema()
  9. .field("a", DataTypes.INT())
  10. .field("b", DataTypes.STRING())
  11. .field("c", DataTypes.BIGINT()))
  12. .create_temporary_table("CsvSinkTable")
  13. # compute a result Table using Table API operators and/or SQL queries
  14. result = ...
  15. # emit the result Table to the registered TableSink
  16. result.execute_insert("CsvSinkTable")

Translate and Execute a Query

The behavior of translating and executing a query is different for the two planners. Table API and SQL queries are translated into DataStream programs whether their input is streaming or batch. A query is internally represented as a logical query plan and is translated in two phases:

  1. Optimization of the logical plan,
  2. Translation into a DataStream program.

a Table API or SQL query is translated when:

  • TableEnvironment.executeSql() is called. This method is used for executing a given statement, and the sql query is translated immediately once this method is called.
  • Table.executeInsert() is called. This method is used for inserting the table content to the given sink path, and the Table API is translated immediately once this method is called.
  • Table.execute() is called. This method is used for collecting the table content to local client, and the Table API is translated immediately once this method is called.
  • StatementSet.execute() is called. A Table (emitted to a sink through StatementSet.addInsert()) or an INSERT statement (specified through StatementSet.addInsertSql()) will be buffered in StatementSet first. They are translated once StatementSet.execute() is called. All sinks will be optimized into one DAG.
  • A Table is translated when it is converted into a DataStream (see Integration with DataStream). Once translated, it’s a regular DataStream program and is executed when StreamExecutionEnvironment.execute() is called.

Query Optimization

Apache Flink leverages and extends Apache Calcite to perform sophisticated query optimization. This includes a series of rule and cost-based optimizations such as:

  • Subquery decorrelation based on Apache Calcite
  • Project pruning
  • Partition pruning
  • Filter push-down
  • Sub-plan deduplication to avoid duplicate computation
  • Special subquery rewriting, including two parts:
    • Converts IN and EXISTS into left semi-joins
    • Converts NOT IN and NOT EXISTS into left anti-join
  • Optional join reordering
    • Enabled via table.optimizer.join-reorder-enabled

Note: IN/EXISTS/NOT IN/NOT EXISTS are currently only supported in conjunctive conditions in subquery rewriting.

The optimizer makes intelligent decisions, based not only on the plan but also rich statistics available from the data sources and fine-grain costs for each operator such as io, cpu, network, and memory.

Advanced users may provide custom optimizations via a CalciteConfig object that can be provided to the table environment by calling TableEnvironment#getConfig#setPlannerConfig.

Explaining a Table

The Table API provides a mechanism to explain the logical and optimized query plans to compute a Table. This is done through the Table.explain() method or StatementSet.explain() method. Table.explain()returns the plan of a Table. StatementSet.explain() returns the plan of multiple sinks. It returns a String describing three plans:

  1. the Abstract Syntax Tree of the relational query, i.e., the unoptimized logical query plan,
  2. the optimized logical query plan, and
  3. the physical execution plan.

TableEnvironment.explainSql() and TableEnvironment.executeSql() support execute a EXPLAIN statement to get the plans, Please refer to EXPLAIN page.

The following code shows an example and the corresponding output for given Table using Table.explain() method:

Java

  1. StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
  2. StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
  3. DataStream<Tuple2<Integer, String>> stream1 = env.fromElements(new Tuple2<>(1, "hello"));
  4. DataStream<Tuple2<Integer, String>> stream2 = env.fromElements(new Tuple2<>(1, "hello"));
  5. // explain Table API
  6. Table table1 = tEnv.fromDataStream(stream1, $("count"), $("word"));
  7. Table table2 = tEnv.fromDataStream(stream2, $("count"), $("word"));
  8. Table table = table1
  9. .where($("word").like("F%"))
  10. .unionAll(table2);
  11. System.out.println(table.explain());

Scala

  1. val env = StreamExecutionEnvironment.getExecutionEnvironment
  2. val tEnv = StreamTableEnvironment.create(env)
  3. val table1 = env.fromElements((1, "hello")).toTable(tEnv, $"count", $"word")
  4. val table2 = env.fromElements((1, "hello")).toTable(tEnv, $"count", $"word")
  5. val table = table1
  6. .where($"word".like("F%"))
  7. .unionAll(table2)
  8. println(table.explain())

Python

  1. env = StreamExecutionEnvironment.get_execution_environment()
  2. t_env = StreamTableEnvironment.create(env)
  3. table1 = t_env.from_elements([(1, "hello")], ["count", "word"])
  4. table2 = t_env.from_elements([(1, "hello")], ["count", "word"])
  5. table = table1 \
  6. .where(table1.word.like('F%')) \
  7. .union_all(table2)
  8. print(table.explain())

The result of the above exmaple is

Explain ↕

  1. == Abstract Syntax Tree ==
  2. LogicalUnion(all=[true])
  3. LogicalFilter(condition=[LIKE($1, _UTF-16LE'F%')])
  4. FlinkLogicalDataStreamScan(id=[1], fields=[count, word])
  5. FlinkLogicalDataStreamScan(id=[2], fields=[count, word])
  6. == Optimized Logical Plan ==
  7. DataStreamUnion(all=[true], union all=[count, word])
  8. DataStreamCalc(select=[count, word], where=[LIKE(word, _UTF-16LE'F%')])
  9. DataStreamScan(id=[1], fields=[count, word])
  10. DataStreamScan(id=[2], fields=[count, word])
  11. == Physical Execution Plan ==
  12. Stage 1 : Data Source
  13. content : collect elements with CollectionInputFormat
  14. Stage 2 : Data Source
  15. content : collect elements with CollectionInputFormat
  16. Stage 3 : Operator
  17. content : from: (count, word)
  18. ship_strategy : REBALANCE
  19. Stage 4 : Operator
  20. content : where: (LIKE(word, _UTF-16LE'F%')), select: (count, word)
  21. ship_strategy : FORWARD
  22. Stage 5 : Operator
  23. content : from: (count, word)
  24. ship_strategy : REBALANCE

The following code shows an example and the corresponding output for multiple-sinks plan using StatementSet.explain() method:

Java

  1. EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
  2. TableEnvironment tEnv = TableEnvironment.create(settings);
  3. final Schema schema = new Schema()
  4. .field("count", DataTypes.INT())
  5. .field("word", DataTypes.STRING());
  6. tEnv.connect(new FileSystem().path("/source/path1"))
  7. .withFormat(new Csv().deriveSchema())
  8. .withSchema(schema)
  9. .createTemporaryTable("MySource1");
  10. tEnv.connect(new FileSystem().path("/source/path2"))
  11. .withFormat(new Csv().deriveSchema())
  12. .withSchema(schema)
  13. .createTemporaryTable("MySource2");
  14. tEnv.connect(new FileSystem().path("/sink/path1"))
  15. .withFormat(new Csv().deriveSchema())
  16. .withSchema(schema)
  17. .createTemporaryTable("MySink1");
  18. tEnv.connect(new FileSystem().path("/sink/path2"))
  19. .withFormat(new Csv().deriveSchema())
  20. .withSchema(schema)
  21. .createTemporaryTable("MySink2");
  22. StatementSet stmtSet = tEnv.createStatementSet();
  23. Table table1 = tEnv.from("MySource1").where($("word").like("F%"));
  24. stmtSet.addInsert("MySink1", table1);
  25. Table table2 = table1.unionAll(tEnv.from("MySource2"));
  26. stmtSet.addInsert("MySink2", table2);
  27. String explanation = stmtSet.explain();
  28. System.out.println(explanation);

Scala

  1. val settings = EnvironmentSettings.newInstance.useBlinkPlanner.inStreamingMode.build
  2. val tEnv = TableEnvironment.create(settings)
  3. val schema = new Schema()
  4. .field("count", DataTypes.INT())
  5. .field("word", DataTypes.STRING())
  6. tEnv.connect(new FileSystem().path("/source/path1"))
  7. .withFormat(new Csv().deriveSchema())
  8. .withSchema(schema)
  9. .createTemporaryTable("MySource1")
  10. tEnv.connect(new FileSystem().path("/source/path2"))
  11. .withFormat(new Csv().deriveSchema())
  12. .withSchema(schema)
  13. .createTemporaryTable("MySource2")
  14. tEnv.connect(new FileSystem().path("/sink/path1"))
  15. .withFormat(new Csv().deriveSchema())
  16. .withSchema(schema)
  17. .createTemporaryTable("MySink1")
  18. tEnv.connect(new FileSystem().path("/sink/path2"))
  19. .withFormat(new Csv().deriveSchema())
  20. .withSchema(schema)
  21. .createTemporaryTable("MySink2")
  22. val stmtSet = tEnv.createStatementSet()
  23. val table1 = tEnv.from("MySource1").where($"word".like("F%"))
  24. stmtSet.addInsert("MySink1", table1)
  25. val table2 = table1.unionAll(tEnv.from("MySource2"))
  26. stmtSet.addInsert("MySink2", table2)
  27. val explanation = stmtSet.explain()
  28. println(explanation)

Python

  1. settings = EnvironmentSettings.new_instance().use_blink_planner().in_streaming_mode().build()
  2. t_env = TableEnvironment.create(environment_settings=settings)
  3. schema = Schema()
  4. .field("count", DataTypes.INT())
  5. .field("word", DataTypes.STRING())
  6. t_env.connect(FileSystem().path("/source/path1")))
  7. .with_format(Csv().deriveSchema())
  8. .with_schema(schema)
  9. .create_temporary_table("MySource1")
  10. t_env.connect(FileSystem().path("/source/path2")))
  11. .with_format(Csv().deriveSchema())
  12. .with_schema(schema)
  13. .create_temporary_table("MySource2")
  14. t_env.connect(FileSystem().path("/sink/path1")))
  15. .with_format(Csv().deriveSchema())
  16. .with_schema(schema)
  17. .create_temporary_table("MySink1")
  18. t_env.connect(FileSystem().path("/sink/path2")))
  19. .with_format(Csv().deriveSchema())
  20. .with_schema(schema)
  21. .create_temporary_table("MySink2")
  22. stmt_set = t_env.create_statement_set()
  23. table1 = t_env.from_path("MySource1").where(col('word').like('F%'))
  24. stmt_set.add_insert("MySink1", table1)
  25. table2 = table1.union_all(t_env.from_path("MySource2"))
  26. stmt_set.add_insert("MySink2", table2)
  27. explanation = stmt_set.explain()
  28. print(explanation)

the result of multiple-sinks plan is

MultiTable Explain ↕

  1. == Abstract Syntax Tree ==
  2. LogicalLegacySink(name=[MySink1], fields=[count, word])
  3. +- LogicalFilter(condition=[LIKE($1, _UTF-16LE'F%')])
  4. +- LogicalTableScan(table=[[default_catalog, default_database, MySource1, source: [CsvTableSource(read fields: count, word)]]])
  5. LogicalLegacySink(name=[MySink2], fields=[count, word])
  6. +- LogicalUnion(all=[true])
  7. :- LogicalFilter(condition=[LIKE($1, _UTF-16LE'F%')])
  8. : +- LogicalTableScan(table=[[default_catalog, default_database, MySource1, source: [CsvTableSource(read fields: count, word)]]])
  9. +- LogicalTableScan(table=[[default_catalog, default_database, MySource2, source: [CsvTableSource(read fields: count, word)]]])
  10. == Optimized Logical Plan ==
  11. Calc(select=[count, word], where=[LIKE(word, _UTF-16LE'F%')], reuse_id=[1])
  12. +- TableSourceScan(table=[[default_catalog, default_database, MySource1, source: [CsvTableSource(read fields: count, word)]]], fields=[count, word])
  13. LegacySink(name=[MySink1], fields=[count, word])
  14. +- Reused(reference_id=[1])
  15. LegacySink(name=[MySink2], fields=[count, word])
  16. +- Union(all=[true], union=[count, word])
  17. :- Reused(reference_id=[1])
  18. +- TableSourceScan(table=[[default_catalog, default_database, MySource2, source: [CsvTableSource(read fields: count, word)]]], fields=[count, word])
  19. == Physical Execution Plan ==
  20. Stage 1 : Data Source
  21. content : collect elements with CollectionInputFormat
  22. Stage 2 : Operator
  23. content : CsvTableSource(read fields: count, word)
  24. ship_strategy : REBALANCE
  25. Stage 3 : Operator
  26. content : SourceConversion(table:Buffer(default_catalog, default_database, MySource1, source: [CsvTableSource(read fields: count, word)]), fields:(count, word))
  27. ship_strategy : FORWARD
  28. Stage 4 : Operator
  29. content : Calc(where: (word LIKE _UTF-16LE'F%'), select: (count, word))
  30. ship_strategy : FORWARD
  31. Stage 5 : Operator
  32. content : SinkConversionToRow
  33. ship_strategy : FORWARD
  34. Stage 6 : Operator
  35. content : Map
  36. ship_strategy : FORWARD
  37. Stage 8 : Data Source
  38. content : collect elements with CollectionInputFormat
  39. Stage 9 : Operator
  40. content : CsvTableSource(read fields: count, word)
  41. ship_strategy : REBALANCE
  42. Stage 10 : Operator
  43. content : SourceConversion(table:Buffer(default_catalog, default_database, MySource2, source: [CsvTableSource(read fields: count, word)]), fields:(count, word))
  44. ship_strategy : FORWARD
  45. Stage 12 : Operator
  46. content : SinkConversionToRow
  47. ship_strategy : FORWARD
  48. Stage 13 : Operator
  49. content : Map
  50. ship_strategy : FORWARD
  51. Stage 7 : Data Sink
  52. content : Sink: CsvTableSink(count, word)
  53. ship_strategy : FORWARD
  54. Stage 14 : Data Sink
  55. content : Sink: CsvTableSink(count, word)
  56. ship_strategy : FORWARD