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.

Main Differences Between the Two Planners

  1. Blink treats batch jobs as a special case of streaming. As such, the conversion between Table and DataSet is also not supported, and batch jobs will not be translated into DateSet programs but translated into DataStream programs, the same as the streaming jobs.
  2. The Blink planner does not support BatchTableSource, use bounded StreamTableSource instead of it.
  3. The implementations of FilterableTableSource for the old planner and the Blink planner are incompatible. The old planner will push down PlannerExpressions into FilterableTableSource, while the Blink planner will push down Expressions.
  4. String based key-value config options (Please see the documentation about Configuration for details) are only used for the Blink planner.
  5. The implementation(CalciteConfig) of PlannerConfig in two planners is different.
  6. The Blink planner will optimize multiple-sinks into one DAG on both TableEnvironment and StreamTableEnvironment. The old planner will always optimize each sink into a new DAG, where all DAGs are independent of each other.
  7. The old planner does not support catalog statistics now, while the Blink planner does.

Structure of Table API and SQL Programs

All Table API and SQL programs for batch and streaming follow the same pattern. The following code example shows the common structure of Table API and SQL programs.

  1. // create a TableEnvironment for specific planner batch or streaming
  2. TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // create a Table
  4. tableEnv.connect(...).createTemporaryTable("table1");
  5. // register an output Table
  6. tableEnv.connect(...).createTemporaryTable("outputTable");
  7. // create a Table object from a Table API query
  8. Table tapiResult = tableEnv.from("table1").select(...);
  9. // create a Table object from a SQL query
  10. Table sqlResult = tableEnv.sqlQuery("SELECT ... FROM table1 ... ");
  11. // emit a Table API result Table to a TableSink, same for SQL result
  12. TableResult tableResult = tapiResult.executeInsert("outputTable");
  13. tableResult...
  1. // create a TableEnvironment for specific planner batch or streaming
  2. val tableEnv = ... // see "Create a TableEnvironment" section
  3. // create a Table
  4. tableEnv.connect(...).createTemporaryTable("table1")
  5. // register an output Table
  6. tableEnv.connect(...).createTemporaryTable("outputTable")
  7. // create a Table from a Table API query
  8. val tapiResult = tableEnv.from("table1").select(...)
  9. // create a Table from a SQL query
  10. val sqlResult = tableEnv.sqlQuery("SELECT ... FROM table1 ...")
  11. // emit a Table API result Table to a TableSink, same for SQL result
  12. val tableResult = tapiResult.executeInsert("outputTable")
  13. tableResult...
  1. # create a TableEnvironment for specific planner batch or streaming
  2. table_env = ... # see "Create a TableEnvironment" section
  3. # register a Table
  4. table_env.connect(...).create_temporary_table("table1")
  5. # register an output Table
  6. table_env.connect(...).create_temporary_table("outputTable")
  7. # create a Table from a Table API query
  8. tapi_result = table_env.from_path("table1").select(...)
  9. # create a Table from a SQL query
  10. sql_result = table_env.sql_query("SELECT ... FROM table1 ...")
  11. # emit a Table API result Table to a TableSink, same for SQL result
  12. table_result = tapi_result.execute_insert("outputTable")
  13. table_result...

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

Back to top

Create a TableEnvironment

The TableEnvironment is a central concept of the Table API and SQL integration. It 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 a DataStream or DataSet into a Table
  • Holding a reference to an ExecutionEnvironment or StreamExecutionEnvironment

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 BatchTableEnvironment.create() or StreamTableEnvironment.create() method with a StreamExecutionEnvironment or an ExecutionEnvironment and an optional TableConfig. The TableConfig can be used to configure the TableEnvironment or to customize the query optimization and translation process (see Query Optimization).

Make sure to choose the specific planner BatchTableEnvironment/StreamTableEnvironment that matches your programming language.

If both planner jars are on the classpath (the default behavior), you should explicitly set which planner to use in the current program.

  1. // **********************
  2. // FLINK STREAMING QUERY
  3. // **********************
  4. import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  5. import org.apache.flink.table.api.EnvironmentSettings;
  6. import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
  7. EnvironmentSettings fsSettings = EnvironmentSettings.newInstance().useOldPlanner().inStreamingMode().build();
  8. StreamExecutionEnvironment fsEnv = StreamExecutionEnvironment.getExecutionEnvironment();
  9. StreamTableEnvironment fsTableEnv = StreamTableEnvironment.create(fsEnv, fsSettings);
  10. // or TableEnvironment fsTableEnv = TableEnvironment.create(fsSettings);
  11. // ******************
  12. // FLINK BATCH QUERY
  13. // ******************
  14. import org.apache.flink.api.java.ExecutionEnvironment;
  15. import org.apache.flink.table.api.bridge.java.BatchTableEnvironment;
  16. ExecutionEnvironment fbEnv = ExecutionEnvironment.getExecutionEnvironment();
  17. BatchTableEnvironment fbTableEnv = BatchTableEnvironment.create(fbEnv);
  18. // **********************
  19. // BLINK STREAMING QUERY
  20. // **********************
  21. import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
  22. import org.apache.flink.table.api.EnvironmentSettings;
  23. import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
  24. StreamExecutionEnvironment bsEnv = StreamExecutionEnvironment.getExecutionEnvironment();
  25. EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
  26. StreamTableEnvironment bsTableEnv = StreamTableEnvironment.create(bsEnv, bsSettings);
  27. // or TableEnvironment bsTableEnv = TableEnvironment.create(bsSettings);
  28. // ******************
  29. // BLINK BATCH QUERY
  30. // ******************
  31. import org.apache.flink.table.api.EnvironmentSettings;
  32. import org.apache.flink.table.api.TableEnvironment;
  33. EnvironmentSettings bbSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build();
  34. TableEnvironment bbTableEnv = TableEnvironment.create(bbSettings);
  1. // **********************
  2. // FLINK STREAMING QUERY
  3. // **********************
  4. import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
  5. import org.apache.flink.table.api.EnvironmentSettings
  6. import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment
  7. val fsSettings = EnvironmentSettings.newInstance().useOldPlanner().inStreamingMode().build()
  8. val fsEnv = StreamExecutionEnvironment.getExecutionEnvironment
  9. val fsTableEnv = StreamTableEnvironment.create(fsEnv, fsSettings)
  10. // or val fsTableEnv = TableEnvironment.create(fsSettings)
  11. // ******************
  12. // FLINK BATCH QUERY
  13. // ******************
  14. import org.apache.flink.api.scala.ExecutionEnvironment
  15. import org.apache.flink.table.api.bridge.scala.BatchTableEnvironment
  16. val fbEnv = ExecutionEnvironment.getExecutionEnvironment
  17. val fbTableEnv = BatchTableEnvironment.create(fbEnv)
  18. // **********************
  19. // BLINK STREAMING QUERY
  20. // **********************
  21. import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
  22. import org.apache.flink.table.api.EnvironmentSettings
  23. import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment
  24. val bsEnv = StreamExecutionEnvironment.getExecutionEnvironment
  25. val bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build()
  26. val bsTableEnv = StreamTableEnvironment.create(bsEnv, bsSettings)
  27. // or val bsTableEnv = TableEnvironment.create(bsSettings)
  28. // ******************
  29. // BLINK BATCH QUERY
  30. // ******************
  31. import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment}
  32. val bbSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build()
  33. val bbTableEnv = TableEnvironment.create(bbSettings)
  1. # **********************
  2. # FLINK STREAMING QUERY
  3. # **********************
  4. from pyflink.datastream import StreamExecutionEnvironment
  5. from pyflink.table import StreamTableEnvironment, EnvironmentSettings
  6. f_s_env = StreamExecutionEnvironment.get_execution_environment()
  7. f_s_settings = EnvironmentSettings.new_instance().use_old_planner().in_streaming_mode().build()
  8. f_s_t_env = StreamTableEnvironment.create(f_s_env, environment_settings=f_s_settings)
  9. # ******************
  10. # FLINK BATCH QUERY
  11. # ******************
  12. from pyflink.dataset import ExecutionEnvironment
  13. from pyflink.table import BatchTableEnvironment
  14. f_b_env = ExecutionEnvironment.get_execution_environment()
  15. f_b_t_env = BatchTableEnvironment.create(f_b_env, table_config)
  16. # **********************
  17. # BLINK STREAMING QUERY
  18. # **********************
  19. from pyflink.datastream import StreamExecutionEnvironment
  20. from pyflink.table import StreamTableEnvironment, EnvironmentSettings
  21. b_s_env = StreamExecutionEnvironment.get_execution_environment()
  22. b_s_settings = EnvironmentSettings.new_instance().use_blink_planner().in_streaming_mode().build()
  23. b_s_t_env = StreamTableEnvironment.create(b_s_env, environment_settings=b_s_settings)
  24. # ******************
  25. # BLINK BATCH QUERY
  26. # ******************
  27. from pyflink.table import EnvironmentSettings, BatchTableEnvironment
  28. b_b_settings = EnvironmentSettings.new_instance().use_blink_planner().in_batch_mode().build()
  29. b_b_t_env = BatchTableEnvironment.create(environment_settings=b_b_settings)

Note: If there is only one planner jar in /lib directory, you can use useAnyPlanner (use_any_planner for python) to create specific EnvironmentSettings.

Back to top

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:

  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);
  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)
  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.

Back to top

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 Apacha Kafka or a regular file system can be declared here.

  1. tableEnvironment
  2. .connect(...)
  3. .withFormat(...)
  4. .withSchema(...)
  5. .inAppendMode()
  6. .createTemporaryTable("MyTable")
  1. tableEnvironment
  2. .connect(...)
  3. .withFormat(...)
  4. .withSchema(...)
  5. .inAppendMode()
  6. .createTemporaryTable("MyTable")
  1. table_environment \
  2. .connect(...) \
  3. .with_format(...) \
  4. .with_schema(...) \
  5. .in_append_mode() \
  6. .create_temporary_table("MyTable")
  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 (` ).

  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);
  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:

  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
  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.
  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("cCountry === 'FRANCE'") \
  9. .group_by("cID, cName") \
  10. .select("cID, cName, revenue.sum AS revSum")
  11. # emit or convert Table
  12. # execute query

Back to top

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.

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

  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. );
  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)
  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. )

Back to top

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.

Back to top

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

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:

  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.LONG());
  8. tableEnv.connect(new FileSystem("/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");
  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.LONG())
  8. tableEnv.connect(new FileSystem("/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")
  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")

Back to top

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 and DataSet API). Once translated, it’s a regular DataStream program and is executed when StreamExecutionEnvironment.execute() is called.

Attention Since 1.11 version, sqlUpdate() method and insertInto() method are deprecated. If the Table program is built from these two methods, we must use StreamTableEnvironment.execute() method instead of StreamExecutionEnvironment.execute() method to execute it.

Table API and SQL queries are translated into DataStream or DataSet programs depending on whether their input is a streaming or batch input. 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 or DataSet 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. Each sink will be optimized independently. The execution graph contains multiple independent sub-DAGs.
  • For streaming, a Table is translated when it is converted into a DataStream (see Integration with DataStream and DataSet API). Once translated, it’s a regular DataStream program and is executed when StreamExecutionEnvironment.execute() is called. For batch, a Table is translated when it is converted into a DataSet (see Integration with DataStream and DataSet API). Once translated, it’s a regular DataSet program and is executed when ExecutionEnvironment.execute() is called.

Attention Since 1.11 version, sqlUpdate() method and insertInto() method are deprecated. For streaming, if the Table program is built from these two methods, we must use StreamTableEnvironment.execute() method instead of StreamExecutionEnvironment.execute() method to execute it. For batch, if the Table program is built from these two methods, we must use BatchTableEnvironment.execute() method instead of ExecutionEnvironment.execute() method to execute it.

Back to top

Integration with DataStream and DataSet API

Both planners on stream can integrate with the DataStream API. Only old planner can integrate with the DataSet API, Blink planner on batch could not be combined with both. Note: The DataSet API discussed below is only relevant for the old planner on batch.

Table API and SQL queries can be easily integrated with and embedded into DataStream and DataSet programs. For instance, it is possible to query an external table (for example from a RDBMS), do some pre-processing, such as filtering, projecting, aggregating, or joining with meta data, and then further process the data with either the DataStream or DataSet API (and any of the libraries built on top of these APIs, such as CEP or Gelly). Inversely, a Table API or SQL query can also be applied on the result of a DataStream or DataSet program.

This interaction can be achieved by converting a DataStream or DataSet into a Table and vice versa. In this section, we describe how these conversions are done.

Implicit Conversion for Scala

The Scala Table API features implicit conversions for the DataSet, DataStream, and Table classes. These conversions are enabled by importing the package org.apache.flink.table.api.bridge.scala._ in addition to org.apache.flink.api.scala._ for the Scala DataStream API.

Create a View from a DataStream or DataSet

A DataStream or DataSet can be registered in a TableEnvironment as a View. The schema of the resulting view depends on the data type of the registered DataStream or DataSet. Please check the section about mapping of data types to table schema for details.

Note: Views created from a DataStream or DataSet can be registered as temporary views only.

  1. // get StreamTableEnvironment
  2. // registration of a DataSet in a BatchTableEnvironment is equivalent
  3. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  4. DataStream<Tuple2<Long, String>> stream = ...
  5. // register the DataStream as View "myTable" with fields "f0", "f1"
  6. tableEnv.createTemporaryView("myTable", stream);
  7. // register the DataStream as View "myTable2" with fields "myLong", "myString"
  8. tableEnv.createTemporaryView("myTable2", stream, $("myLong"), $("myString"));
  1. // get TableEnvironment
  2. // registration of a DataSet is equivalent
  3. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  4. val stream: DataStream[(Long, String)] = ...
  5. // register the DataStream as View "myTable" with fields "f0", "f1"
  6. tableEnv.createTemporaryView("myTable", stream)
  7. // register the DataStream as View "myTable2" with fields "myLong", "myString"
  8. tableEnv.createTemporaryView("myTable2", stream, 'myLong, 'myString)

Back to top

Convert a DataStream or DataSet into a Table

Instead of registering a DataStream or DataSet in a TableEnvironment, it can also be directly converted into a Table. This is convenient if you want to use the Table in a Table API query.

  1. // get StreamTableEnvironment
  2. // registration of a DataSet in a BatchTableEnvironment is equivalent
  3. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  4. DataStream<Tuple2<Long, String>> stream = ...
  5. // Convert the DataStream into a Table with default fields "f0", "f1"
  6. Table table1 = tableEnv.fromDataStream(stream);
  7. // Convert the DataStream into a Table with fields "myLong", "myString"
  8. Table table2 = tableEnv.fromDataStream(stream, $("myLong"), $("myString"));
  1. // get TableEnvironment
  2. // registration of a DataSet is equivalent
  3. val tableEnv = ... // see "Create a TableEnvironment" section
  4. val stream: DataStream[(Long, String)] = ...
  5. // convert the DataStream into a Table with default fields "_1", "_2"
  6. val table1: Table = tableEnv.fromDataStream(stream)
  7. // convert the DataStream into a Table with fields "myLong", "myString"
  8. val table2: Table = tableEnv.fromDataStream(stream, $"myLong", $"myString")

Back to top

Convert a Table into a DataStream or DataSet

A Table can be converted into a DataStream or DataSet. In this way, custom DataStream or DataSet programs can be run on the result of a Table API or SQL query.

When converting a Table into a DataStream or DataSet, you need to specify the data type of the resulting DataStream or DataSet, i.e., the data type into which the rows of the Table are to be converted. Often the most convenient conversion type is Row. The following list gives an overview of the features of the different options:

  • Row: fields are mapped by position, arbitrary number of fields, support for null values, no type-safe access.
  • POJO: fields are mapped by name (POJO fields must be named as Table fields), arbitrary number of fields, support for null values, type-safe access.
  • Case Class: fields are mapped by position, no support for null values, type-safe access.
  • Tuple: fields are mapped by position, limitation to 22 (Scala) or 25 (Java) fields, no support for null values, type-safe access.
  • Atomic Type: Table must have a single field, no support for null values, type-safe access.

Convert a Table into a DataStream

A Table that is the result of a streaming query will be updated dynamically, i.e., it is changing as new records arrive on the query’s input streams. Hence, the DataStream into which such a dynamic query is converted needs to encode the updates of the table.

There are two modes to convert a Table into a DataStream:

  1. Append Mode: This mode can only be used if the dynamic Table is only modified by INSERT changes, i.e, it is append-only and previously emitted results are never updated.
  2. Retract Mode: This mode can always be used. It encodes INSERT and DELETE changes with a boolean flag.
  1. // get StreamTableEnvironment.
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // Table with two fields (String name, Integer age)
  4. Table table = ...
  5. // convert the Table into an append DataStream of Row by specifying the class
  6. DataStream<Row> dsRow = tableEnv.toAppendStream(table, Row.class);
  7. // convert the Table into an append DataStream of Tuple2<String, Integer>
  8. // via a TypeInformation
  9. TupleTypeInfo<Tuple2<String, Integer>> tupleType = new TupleTypeInfo<>(
  10. Types.STRING(),
  11. Types.INT());
  12. DataStream<Tuple2<String, Integer>> dsTuple =
  13. tableEnv.toAppendStream(table, tupleType);
  14. // convert the Table into a retract DataStream of Row.
  15. // A retract stream of type X is a DataStream<Tuple2<Boolean, X>>.
  16. // The boolean field indicates the type of the change.
  17. // True is INSERT, false is DELETE.
  18. DataStream<Tuple2<Boolean, Row>> retractStream =
  19. tableEnv.toRetractStream(table, Row.class);
  1. // get TableEnvironment.
  2. // registration of a DataSet is equivalent
  3. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  4. // Table with two fields (String name, Integer age)
  5. val table: Table = ...
  6. // convert the Table into an append DataStream of Row
  7. val dsRow: DataStream[Row] = tableEnv.toAppendStream[Row](table)
  8. // convert the Table into an append DataStream of Tuple2[String, Int]
  9. val dsTuple: DataStream[(String, Int)] dsTuple =
  10. tableEnv.toAppendStream[(String, Int)](table)
  11. // convert the Table into a retract DataStream of Row.
  12. // A retract stream of type X is a DataStream[(Boolean, X)].
  13. // The boolean field indicates the type of the change.
  14. // True is INSERT, false is DELETE.
  15. val retractStream: DataStream[(Boolean, Row)] = tableEnv.toRetractStream[Row](table)

Note: A detailed discussion about dynamic tables and their properties is given in the Dynamic Tables document.

Attention Once the Table is converted to a DataStream, please use the StreamExecutionEnvironment.execute() method to execute the DataStream program.

Convert a Table into a DataSet

A Table is converted into a DataSet as follows:

  1. // get BatchTableEnvironment
  2. BatchTableEnvironment tableEnv = BatchTableEnvironment.create(env);
  3. // Table with two fields (String name, Integer age)
  4. Table table = ...
  5. // convert the Table into a DataSet of Row by specifying a class
  6. DataSet<Row> dsRow = tableEnv.toDataSet(table, Row.class);
  7. // convert the Table into a DataSet of Tuple2<String, Integer> via a TypeInformation
  8. TupleTypeInfo<Tuple2<String, Integer>> tupleType = new TupleTypeInfo<>(
  9. Types.STRING(),
  10. Types.INT());
  11. DataSet<Tuple2<String, Integer>> dsTuple =
  12. tableEnv.toDataSet(table, tupleType);
  1. // get TableEnvironment
  2. // registration of a DataSet is equivalent
  3. val tableEnv = BatchTableEnvironment.create(env)
  4. // Table with two fields (String name, Integer age)
  5. val table: Table = ...
  6. // convert the Table into a DataSet of Row
  7. val dsRow: DataSet[Row] = tableEnv.toDataSet[Row](table)
  8. // convert the Table into a DataSet of Tuple2[String, Int]
  9. val dsTuple: DataSet[(String, Int)] = tableEnv.toDataSet[(String, Int)](table)

Attention Once the Table is converted to a DataSet, we must use the ExecutionEnvironment.execute method to execute the DataSet program.

Back to top

Mapping of Data Types to Table Schema

Flink’s DataStream and DataSet APIs support very diverse types. Composite types such as Tuples (built-in Scala and Flink Java tuples), POJOs, Scala case classes, and Flink’s Row type allow for nested data structures with multiple fields that can be accessed in table expressions. Other types are treated as atomic types. In the following, we describe how the Table API converts these types into an internal row representation and show examples of converting a DataStream into a Table.

The mapping of a data type to a table schema can happen in two ways: based on the field positions or based on the field names.

Position-based Mapping

Position-based mapping can be used to give fields a more meaningful name while keeping the field order. This mapping is available for composite data types with a defined field order as well as atomic types. Composite data types such as tuples, rows, and case classes have such a field order. However, fields of a POJO must be mapped based on the field names (see next section). Fields can be projected out but can’t be renamed using an alias as.

When defining a position-based mapping, the specified names must not exist in the input data type, otherwise the API will assume that the mapping should happen based on the field names. If no field names are specified, the default field names and field order of the composite type are used or f0 for atomic types.

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section;
  3. DataStream<Tuple2<Long, Integer>> stream = ...
  4. // convert DataStream into Table with default field names "f0" and "f1"
  5. Table table = tableEnv.fromDataStream(stream);
  6. // convert DataStream into Table with field "myLong" only
  7. Table table = tableEnv.fromDataStream(stream, $("myLong"));
  8. // convert DataStream into Table with field names "myLong" and "myInt"
  9. Table table = tableEnv.fromDataStream(stream, $("myLong"), $("myInt"));
  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. val stream: DataStream[(Long, Int)] = ...
  4. // convert DataStream into Table with default field names "_1" and "_2"
  5. val table: Table = tableEnv.fromDataStream(stream)
  6. // convert DataStream into Table with field "myLong" only
  7. val table: Table = tableEnv.fromDataStream(stream, $"myLong")
  8. // convert DataStream into Table with field names "myLong" and "myInt"
  9. val table: Table = tableEnv.fromDataStream(stream, $"myLong", $"myInt")

Name-based Mapping

Name-based mapping can be used for any data type including POJOs. It is the most flexible way of defining a table schema mapping. All fields in the mapping are referenced by name and can be possibly renamed using an alias as. Fields can be reordered and projected out.

If no field names are specified, the default field names and field order of the composite type are used or f0 for atomic types.

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. DataStream<Tuple2<Long, Integer>> stream = ...
  4. // convert DataStream into Table with default field names "f0" and "f1"
  5. Table table = tableEnv.fromDataStream(stream);
  6. // convert DataStream into Table with field "f1" only
  7. Table table = tableEnv.fromDataStream(stream, $("f1"));
  8. // convert DataStream into Table with swapped fields
  9. Table table = tableEnv.fromDataStream(stream, $("f1"), $("f0"));
  10. // convert DataStream into Table with swapped fields and field names "myInt" and "myLong"
  11. Table table = tableEnv.fromDataStream(stream, $("f1").as("myInt"), $("f0").as("myLong"));
  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. val stream: DataStream[(Long, Int)] = ...
  4. // convert DataStream into Table with default field names "_1" and "_2"
  5. val table: Table = tableEnv.fromDataStream(stream)
  6. // convert DataStream into Table with field "_2" only
  7. val table: Table = tableEnv.fromDataStream(stream, $"_2")
  8. // convert DataStream into Table with swapped fields
  9. val table: Table = tableEnv.fromDataStream(stream, $"_2", $"_1")
  10. // convert DataStream into Table with swapped fields and field names "myInt" and "myLong"
  11. val table: Table = tableEnv.fromDataStream(stream, $"_2" as "myInt", $"_1" as "myLong")

Atomic Types

Flink treats primitives (Integer, Double, String) or generic types (types that cannot be analyzed and decomposed) as atomic types. A DataStream or DataSet of an atomic type is converted into a Table with a single attribute. The type of the attribute is inferred from the atomic type and the name of the attribute can be specified.

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. DataStream<Long> stream = ...
  4. // convert DataStream into Table with default field name "f0"
  5. Table table = tableEnv.fromDataStream(stream);
  6. // convert DataStream into Table with field name "myLong"
  7. Table table = tableEnv.fromDataStream(stream, $("myLong"));
  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. val stream: DataStream[Long] = ...
  4. // convert DataStream into Table with default field name "f0"
  5. val table: Table = tableEnv.fromDataStream(stream)
  6. // convert DataStream into Table with field name "myLong"
  7. val table: Table = tableEnv.fromDataStream(stream, $"myLong")

Tuples (Scala and Java) and Case Classes (Scala only)

Flink supports Scala’s built-in tuples and provides its own tuple classes for Java. DataStreams and DataSets of both kinds of tuples can be converted into tables. Fields can be renamed by providing names for all fields (mapping based on position). If no field names are specified, the default field names are used. If the original field names (f0, f1, … for Flink Tuples and _1, _2, … for Scala Tuples) are referenced, the API assumes that the mapping is name-based instead of position-based. Name-based mapping allows for reordering fields and projection with alias (as).

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. DataStream<Tuple2<Long, String>> stream = ...
  4. // convert DataStream into Table with default field names "f0", "f1"
  5. Table table = tableEnv.fromDataStream(stream);
  6. // convert DataStream into Table with renamed field names "myLong", "myString" (position-based)
  7. Table table = tableEnv.fromDataStream(stream, $("myLong"), $("myString"));
  8. // convert DataStream into Table with reordered fields "f1", "f0" (name-based)
  9. Table table = tableEnv.fromDataStream(stream, $("f1"), $("f0"));
  10. // convert DataStream into Table with projected field "f1" (name-based)
  11. Table table = tableEnv.fromDataStream(stream, $("f1"));
  12. // convert DataStream into Table with reordered and aliased fields "myString", "myLong" (name-based)
  13. Table table = tableEnv.fromDataStream(stream, $("f1").as("myString"), $("f0").as("myLong"));
  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. val stream: DataStream[(Long, String)] = ...
  4. // convert DataStream into Table with renamed default field names '_1, '_2
  5. val table: Table = tableEnv.fromDataStream(stream)
  6. // convert DataStream into Table with field names "myLong", "myString" (position-based)
  7. val table: Table = tableEnv.fromDataStream(stream, $"myLong", $"myString")
  8. // convert DataStream into Table with reordered fields "_2", "_1" (name-based)
  9. val table: Table = tableEnv.fromDataStream(stream, $"_2", $"_1")
  10. // convert DataStream into Table with projected field "_2" (name-based)
  11. val table: Table = tableEnv.fromDataStream(stream, $"_2")
  12. // convert DataStream into Table with reordered and aliased fields "myString", "myLong" (name-based)
  13. val table: Table = tableEnv.fromDataStream(stream, $"_2" as "myString", $"_1" as "myLong")
  14. // define case class
  15. case class Person(name: String, age: Int)
  16. val streamCC: DataStream[Person] = ...
  17. // convert DataStream into Table with default field names 'name, 'age
  18. val table = tableEnv.fromDataStream(streamCC)
  19. // convert DataStream into Table with field names 'myName, 'myAge (position-based)
  20. val table = tableEnv.fromDataStream(streamCC, $"myName", $"myAge")
  21. // convert DataStream into Table with reordered and aliased fields "myAge", "myName" (name-based)
  22. val table: Table = tableEnv.fromDataStream(stream, $"age" as "myAge", $"name" as "myName")

POJO (Java and Scala)

Flink supports POJOs as composite types. The rules for what determines a POJO are documented here.

When converting a POJO DataStream or DataSet into a Table without specifying field names, the names of the original POJO fields are used. The name mapping requires the original names and cannot be done by positions. Fields can be renamed using an alias (with the as keyword), reordered, and projected.

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // Person is a POJO with fields "name" and "age"
  4. DataStream<Person> stream = ...
  5. // convert DataStream into Table with default field names "age", "name" (fields are ordered by name!)
  6. Table table = tableEnv.fromDataStream(stream);
  7. // convert DataStream into Table with renamed fields "myAge", "myName" (name-based)
  8. Table table = tableEnv.fromDataStream(stream, $("age").as("myAge"), $("name").as("myName"));
  9. // convert DataStream into Table with projected field "name" (name-based)
  10. Table table = tableEnv.fromDataStream(stream, $("name"));
  11. // convert DataStream into Table with projected and renamed field "myName" (name-based)
  12. Table table = tableEnv.fromDataStream(stream, $("name").as("myName"));
  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. // Person is a POJO with field names "name" and "age"
  4. val stream: DataStream[Person] = ...
  5. // convert DataStream into Table with default field names "age", "name" (fields are ordered by name!)
  6. val table: Table = tableEnv.fromDataStream(stream)
  7. // convert DataStream into Table with renamed fields "myAge", "myName" (name-based)
  8. val table: Table = tableEnv.fromDataStream(stream, $"age" as "myAge", $"name" as "myName")
  9. // convert DataStream into Table with projected field "name" (name-based)
  10. val table: Table = tableEnv.fromDataStream(stream, $"name")
  11. // convert DataStream into Table with projected and renamed field "myName" (name-based)
  12. val table: Table = tableEnv.fromDataStream(stream, $"name" as "myName")

Row

The Row data type supports an arbitrary number of fields and fields with null values. Field names can be specified via a RowTypeInfo or when converting a Row DataStream or DataSet into a Table. The row type supports mapping of fields by position and by name. Fields can be renamed by providing names for all fields (mapping based on position) or selected individually for projection/ordering/renaming (mapping based on name).

  1. // get a StreamTableEnvironment, works for BatchTableEnvironment equivalently
  2. StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section
  3. // DataStream of Row with two fields "name" and "age" specified in `RowTypeInfo`
  4. DataStream<Row> stream = ...
  5. // convert DataStream into Table with default field names "name", "age"
  6. Table table = tableEnv.fromDataStream(stream);
  7. // convert DataStream into Table with renamed field names "myName", "myAge" (position-based)
  8. Table table = tableEnv.fromDataStream(stream, $("myName"), $("myAge"));
  9. // convert DataStream into Table with renamed fields "myName", "myAge" (name-based)
  10. Table table = tableEnv.fromDataStream(stream, $("name").as("myName"), $("age").as("myAge"));
  11. // convert DataStream into Table with projected field "name" (name-based)
  12. Table table = tableEnv.fromDataStream(stream, $("name"));
  13. // convert DataStream into Table with projected and renamed field "myName" (name-based)
  14. Table table = tableEnv.fromDataStream(stream, $("name").as("myName"));
  1. // get a TableEnvironment
  2. val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" section
  3. // DataStream of Row with two fields "name" and "age" specified in `RowTypeInfo`
  4. val stream: DataStream[Row] = ...
  5. // convert DataStream into Table with default field names "name", "age"
  6. val table: Table = tableEnv.fromDataStream(stream)
  7. // convert DataStream into Table with renamed field names "myName", "myAge" (position-based)
  8. val table: Table = tableEnv.fromDataStream(stream, $"myName", $"myAge")
  9. // convert DataStream into Table with renamed fields "myName", "myAge" (name-based)
  10. val table: Table = tableEnv.fromDataStream(stream, $"name" as "myName", $"age" as "myAge")
  11. // convert DataStream into Table with projected field "name" (name-based)
  12. val table: Table = tableEnv.fromDataStream(stream, $"name")
  13. // convert DataStream into Table with projected and renamed field "myName" (name-based)
  14. val table: Table = tableEnv.fromDataStream(stream, $"name" as "myName")

Back to top

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.

Apache Flink leverages Apache Calcite to optimize and translate queries. The optimization currently performed include projection and filter push-down, subquery decorrelation, and other kinds of query rewriting. Old planner does not yet optimize the order of joins, but executes them in the same order as defined in the query (order of Tables in the FROM clause and/or order of join predicates in the WHERE clause).

It is possible to tweak the set of optimization rules which are applied in different phases by providing a CalciteConfig object. This can be created via a builder by calling CalciteConfig.createBuilder()) and is provided to the TableEnvironment by calling tableEnv.getConfig.setPlannerConfig(calciteConfig).

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:

  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());
  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())
  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("LIKE(word, 'F%')") \
  7. .union_all(table2)
  8. print(table.explain())

The result of the above exmaple is

  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:

  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("/source/path1"))
  7. .withFormat(new Csv().deriveSchema())
  8. .withSchema(schema)
  9. .createTemporaryTable("MySource1");
  10. tEnv.connect(new FileSystem("/source/path2"))
  11. .withFormat(new Csv().deriveSchema())
  12. .withSchema(schema)
  13. .createTemporaryTable("MySource2");
  14. tEnv.connect(new FileSystem("/sink/path1"))
  15. .withFormat(new Csv().deriveSchema())
  16. .withSchema(schema)
  17. .createTemporaryTable("MySink1");
  18. tEnv.connect(new FileSystem("/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);
  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("/source/path1"))
  7. .withFormat(new Csv().deriveSchema())
  8. .withSchema(schema)
  9. .createTemporaryTable("MySource1")
  10. tEnv.connect(new FileSystem("/source/path2"))
  11. .withFormat(new Csv().deriveSchema())
  12. .withSchema(schema)
  13. .createTemporaryTable("MySource2")
  14. tEnv.connect(new FileSystem("/sink/path1"))
  15. .withFormat(new Csv().deriveSchema())
  16. .withSchema(schema)
  17. .createTemporaryTable("MySink1")
  18. tEnv.connect(new FileSystem("/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)
  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("LIKE(word, '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

  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

Back to top