Table API & SQL
Apache Flink features two relational APIs - the Table API and SQL - for unified stream and batch processing. The Table API is a language-integrated query API for Scala and Java that allows the composition of queries from relational operators such as selection, filter, and join in a very intuitive way. Flink’s SQL support is based on Apache Calcite which implements the SQL standard. Queries specified in either interface have the same semantics and specify the same result regardless whether the input is a batch input (DataSet) or a stream input (DataStream).
The Table API and the SQL interfaces are tightly integrated with each other as well as Flink’s DataStream and DataSet APIs. You can easily switch between all APIs and libraries which build upon the APIs. For instance, you can extract patterns from a DataStream using the CEP library and later use the Table API to analyze the patterns, or you might scan, filter, and aggregate a batch table using a SQL query before running a Gelly graph algorithm on the preprocessed data.
Please note that the Table API and SQL are not yet feature complete and are being actively developed. Not all operations are supported by every combination of [Table API, SQL] and [stream, batch] input.
Dependency Structure
Starting from Flink 1.9, Flink provides two different planner implementations for evaluating Table & SQL API programs: the Blink planner and the old planner that was available before Flink 1.9. Planners are responsible fortranslating relational operators into an executable, optimized Flink job. Both of the planners come with different optimization rules and runtime classes.They may also differ in the set of supported features.
Attention For production use cases, we recommend the old planner that was present before Flink 1.9 for now.
All Table API and SQL components are bundled in the flink-table
or flink-table-blink
Maven artifacts.
The following dependencies are relevant for most projects:
flink-table-common
: A common module for extending the table ecosystem by custom functions, formats, etc.flink-table-api-java
: The Table & SQL API for pure table programs using the Java programming language (in early development stage, not recommended!).flink-table-api-scala
: The Table & SQL API for pure table programs using the Scala programming language (in early development stage, not recommended!).flink-table-api-java-bridge
: The Table & SQL API with DataStream/DataSet API support using the Java programming language.flink-table-api-scala-bridge
: The Table & SQL API with DataStream/DataSet API support using the Scala programming language.flink-table-planner
: The table program planner and runtime. This was the only planner of Flink before the 1.9 release. It is still the recommended one.flink-table-planner-blink
: The new Blink planner.flink-table-runtime-blink
: The new Blink runtime.flink-table-uber
: Packages the API modules above plus the old planner into a distribution for most Table & SQL API use cases. The uber JAR fileflink-table-*.jar
is located in the/lib
directory of a Flink release by default.flink-table-uber-blink
: Packages the API modules above plus the Blink specific modules into a distribution for most Table & SQL API use cases. The uber JAR fileflink-table-blink-*.jar
is located in the/lib
directory of a Flink release by default.See the common API page for more information about how to switch between the old and new Blink planner in table programs.
Table Program Dependencies
Depending on the target programming language, you need to add the Java or Scala API to a project in order to use the Table API & SQL for defining pipelines:
<!-- Either... -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge_2.11</artifactId>
<version>1.9.0</version>
<scope>provided</scope>
</dependency>
<!-- or... -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-scala-bridge_2.11</artifactId>
<version>1.9.0</version>
<scope>provided</scope>
</dependency>
Additionally, if you want to run the Table API & SQL programs locally within your IDE, you must add one of thefollowing set of modules, depending which planner you want to use:
<!-- Either... (for the old planner that was available before Flink 1.9) -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner_2.11</artifactId>
<version>1.9.0</version>
<scope>provided</scope>
</dependency>
<!-- or.. (for the new Blink planner) -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner-blink_2.11</artifactId>
<version>1.9.0</version>
<scope>provided</scope>
</dependency>
Internally, parts of the table ecosystem are implemented in Scala. Therefore, please make sure to add the following dependency for both batch and streaming applications:
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-scala_2.11</artifactId>
<version>1.9.0</version>
<scope>provided</scope>
</dependency>
Extension Dependencies
If you want to implement a custom format for interacting with Kafka or a set of user-defined functions, the following dependency is sufficient and can be used for JAR files for the SQL Client:
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>1.9.0</version>
<scope>provided</scope>
</dependency>
Currently, the module includes extension points for:
Where to go next?
- Concepts & Common API: Shared concepts and APIs of the Table API and SQL.
- Data Types: Lists pre-defined data types and their properties.
- Streaming Concepts: Streaming-specific documentation for the Table API or SQL such as configuration of time attributes and handling of updating results.
- Connect to External Systems: Available connectors and formats for reading and writing data to external systems.
- Table API: Supported operations and API for the Table API.
- SQL: Supported operations and syntax for SQL.
- Built-in Functions: Supported functions in Table API and SQL.
- SQL Client: Play around with Flink SQL and submit a table program to a cluster without programming knowledge.