• SQL 客户端 Beta
    • Getting Started
      • Starting the SQL Client CLI
      • Running SQL Queries
    • Configuration
      • Environment Files
        • Restart Strategies
      • Dependencies
      • User-defined Functions
        • Constructor Parameters
    • Catalogs
    • Detached SQL Queries
    • SQL Views
    • Temporal Tables
    • Limitations & Future

    SQL 客户端 Beta

    Flink’s Table & SQL API makes it possible to work with queries written in the SQL language, but these queries need to be embedded within a table program that is written in either Java or Scala. Moreover, these programs need to be packaged with a build tool before being submitted to a cluster. This more or less limits the usage of Flink to Java/Scala programmers.

    The SQL Client aims to provide an easy way of writing, debugging, and submitting table programs to a Flink cluster without a single line of Java or Scala code. The SQL Client CLI allows for retrieving and visualizing real-time results from the running distributed application on the command line.

    Animated demo of the Flink SQL Client CLI running table programs on a cluster

    Attention The SQL Client is in an early development phase. Even though the application is not production-ready yet, it can be a quite useful tool for prototyping and playing around with Flink SQL. In the future, the community plans to extend its functionality by providing a REST-based SQL Client Gateway.

    • Getting Started
      • Starting the SQL Client CLI
      • Running SQL Queries
    • Configuration
      • Environment Files
      • Dependencies
      • User-defined Functions
    • Catalogs
    • Detached SQL Queries
    • SQL Views
    • Temporal Tables
    • Limitations & Future

    Getting Started

    This section describes how to setup and run your first Flink SQL program from the command-line.

    The SQL Client is bundled in the regular Flink distribution and thus runnable out-of-the-box. It requires only a running Flink cluster where table programs can be executed. For more information about setting up a Flink cluster see the Cluster & Deployment part. If you simply want to try out the SQL Client, you can also start a local cluster with one worker using the following command:

    1. ./bin/start-cluster.sh

    Starting the SQL Client CLI

    The SQL Client scripts are also located in the binary directory of Flink. In the future, a user will have two possibilities of starting the SQL Client CLI either by starting an embedded standalone process or by connecting to a remote SQL Client Gateway. At the moment only the embedded mode is supported. You can start the CLI by calling:

    1. ./bin/sql-client.sh embedded

    By default, the SQL Client will read its configuration from the environment file located in ./conf/sql-client-defaults.yaml. See the configuration part for more information about the structure of environment files.

    Running SQL Queries

    Once the CLI has been started, you can use the HELP command to list all available SQL statements. For validating your setup and cluster connection, you can enter your first SQL query and press the Enter key to execute it:

    1. SELECT 'Hello World';

    This query requires no table source and produces a single row result. The CLI will retrieve results from the cluster and visualize them. You can close the result view by pressing the Q key.

    The CLI supports two modes for maintaining and visualizing results.

    The table mode materializes results in memory and visualizes them in a regular, paginated table representation. It can be enabled by executing the following command in the CLI:

    1. SET execution.result-mode=table;

    The changelog mode does not materialize results and visualizes the result stream that is produced by a continuous query consisting of insertions (+) and retractions (-).

    1. SET execution.result-mode=changelog;

    You can use the following query to see both result modes in action:

    1. SELECT name, COUNT(*) AS cnt FROM (VALUES ('Bob'), ('Alice'), ('Greg'), ('Bob')) AS NameTable(name) GROUP BY name;

    This query performs a bounded word count example.

    In changelog mode, the visualized changelog should be similar to:

    1. + Bob, 1
    2. + Alice, 1
    3. + Greg, 1
    4. - Bob, 1
    5. + Bob, 2

    In table mode, the visualized result table is continuously updated until the table program ends with:

    1. Bob, 2
    2. Alice, 1
    3. Greg, 1

    Both result modes can be useful during the prototyping of SQL queries. In both modes, results are stored in the Java heap memory of the SQL Client. In order to keep the CLI interface responsive, the changelog mode only shows the latest 1000 changes. The table mode allows for navigating through bigger results that are only limited by the available main memory and the configured maximum number of rows (max-table-result-rows).

    Attention Queries that are executed in a batch environment, can only be retrieved using the table result mode.

    After a query is defined, it can be submitted to the cluster as a long-running, detached Flink job. For this, a target system that stores the results needs to be specified using the INSERT INTO statement. The configuration section explains how to declare table sources for reading data, how to declare table sinks for writing data, and how to configure other table program properties.

    Configuration

    The SQL Client can be started with the following optional CLI commands. They are discussed in detail in the subsequent paragraphs.

    1. ./bin/sql-client.sh embedded --help
    2. Mode "embedded" submits Flink jobs from the local machine.
    3. Syntax: embedded [OPTIONS]
    4. "embedded" mode options:
    5. -d,--defaults <environment file> The environment properties with which
    6. every new session is initialized.
    7. Properties might be overwritten by
    8. session properties.
    9. -e,--environment <environment file> The environment properties to be
    10. imported into the session. It might
    11. overwrite default environment
    12. properties.
    13. -h,--help Show the help message with
    14. descriptions of all options.
    15. -j,--jar <JAR file> A JAR file to be imported into the
    16. session. The file might contain
    17. user-defined classes needed for the
    18. execution of statements such as
    19. functions, table sources, or sinks.
    20. Can be used multiple times.
    21. -l,--library <JAR directory> A JAR file directory with which every
    22. new session is initialized. The files
    23. might contain user-defined classes
    24. needed for the execution of
    25. statements such as functions, table
    26. sources, or sinks. Can be used
    27. multiple times.
    28. -s,--session <session identifier> The identifier for a session.
    29. 'default' is the default identifier.

    Environment Files

    A SQL query needs a configuration environment in which it is executed. The so-called environment files define available table sources and sinks, external catalogs, user-defined functions, and other properties required for execution and deployment.

    Every environment file is a regular YAML file. An example of such a file is presented below.

    1. # Define tables here such as sources, sinks, views, or temporal tables.
    2. tables:
    3. - name: MyTableSource
    4. type: source-table
    5. update-mode: append
    6. connector:
    7. type: filesystem
    8. path: "/path/to/something.csv"
    9. format:
    10. type: csv
    11. fields:
    12. - name: MyField1
    13. type: INT
    14. - name: MyField2
    15. type: VARCHAR
    16. line-delimiter: "\n"
    17. comment-prefix: "#"
    18. schema:
    19. - name: MyField1
    20. type: INT
    21. - name: MyField2
    22. type: VARCHAR
    23. - name: MyCustomView
    24. type: view
    25. query: "SELECT MyField2 FROM MyTableSource"
    26. # Define user-defined functions here.
    27. functions:
    28. - name: myUDF
    29. from: class
    30. class: foo.bar.AggregateUDF
    31. constructor:
    32. - 7.6
    33. - false
    34. # Execution properties allow for changing the behavior of a table program.
    35. execution:
    36. type: streaming # required: execution mode either 'batch' or 'streaming'
    37. result-mode: table # required: either 'table' or 'changelog'
    38. max-table-result-rows: 1000000 # optional: maximum number of maintained rows in
    39. # 'table' mode (1000000 by default, smaller 1 means unlimited)
    40. time-characteristic: event-time # optional: 'processing-time' or 'event-time' (default)
    41. parallelism: 1 # optional: Flink's parallelism (1 by default)
    42. periodic-watermarks-interval: 200 # optional: interval for periodic watermarks (200 ms by default)
    43. max-parallelism: 16 # optional: Flink's maximum parallelism (128 by default)
    44. min-idle-state-retention: 0 # optional: table program's minimum idle state time
    45. max-idle-state-retention: 0 # optional: table program's maximum idle state time
    46. restart-strategy: # optional: restart strategy
    47. type: fallback # "fallback" to global restart strategy by default
    48. # Deployment properties allow for describing the cluster to which table programs are submitted to.
    49. deployment:
    50. response-timeout: 5000

    This configuration:

    • defines an environment with a table source MyTableSource that reads from a CSV file,
    • defines a view MyCustomView that declares a virtual table using a SQL query,
    • defines a user-defined function myUDF that can be instantiated using the class name and two constructor parameters,
    • specifies a parallelism of 1 for queries executed in this streaming environment,
    • specifies an event-time characteristic, and
    • runs queries in the table result mode.

    Depending on the use case, a configuration can be split into multiple files. Therefore, environment files can be created for general purposes (defaults environment file using —defaults) as well as on a per-session basis (session environment file using —environment). Every CLI session is initialized with the default properties followed by the session properties. For example, the defaults environment file could specify all table sources that should be available for querying in every session whereas the session environment file only declares a specific state retention time and parallelism. Both default and session environment files can be passed when starting the CLI application. If no default environment file has been specified, the SQL Client searches for ./conf/sql-client-defaults.yaml in Flink’s configuration directory.

    Attention Properties that have been set within a CLI session (e.g. using the SET command) have highest precedence:

    1. CLI commands > session environment file > defaults environment file

    Restart Strategies

    Restart strategies control how Flink jobs are restarted in case of a failure. Similar to global restart strategies for a Flink cluster, a more fine-grained restart configuration can be declared in an environment file.

    The following strategies are supported:

    1. execution:
    2. # falls back to the global strategy defined in flink-conf.yaml
    3. restart-strategy:
    4. type: fallback
    5. # job fails directly and no restart is attempted
    6. restart-strategy:
    7. type: none
    8. # attempts a given number of times to restart the job
    9. restart-strategy:
    10. type: fixed-delay
    11. attempts: 3 # retries before job is declared as failed (default: Integer.MAX_VALUE)
    12. delay: 10000 # delay in ms between retries (default: 10 s)
    13. # attempts as long as the maximum number of failures per time interval is not exceeded
    14. restart-strategy:
    15. type: failure-rate
    16. max-failures-per-interval: 1 # retries in interval until failing (default: 1)
    17. failure-rate-interval: 60000 # measuring interval in ms for failure rate
    18. delay: 10000 # delay in ms between retries (default: 10 s)

    Dependencies

    The SQL Client does not require to setup a Java project using Maven or SBT. Instead, you can pass the dependencies as regular JAR files that get submitted to the cluster. You can either specify each JAR file separately (using —jar) or define entire library directories (using —library). For connectors to external systems (such as Apache Kafka) and corresponding data formats (such as JSON), Flink provides ready-to-use JAR bundles. These JAR files can be downloaded for each release from the Maven central repository.

    The full list of offered SQL JARs and documentation about how to use them can be found on the connection to external systems page.

    The following example shows an environment file that defines a table source reading JSON data from Apache Kafka.

    1. tables:
    2. - name: TaxiRides
    3. type: source-table
    4. update-mode: append
    5. connector:
    6. property-version: 1
    7. type: kafka
    8. version: "0.11"
    9. topic: TaxiRides
    10. startup-mode: earliest-offset
    11. properties:
    12. - key: zookeeper.connect
    13. value: localhost:2181
    14. - key: bootstrap.servers
    15. value: localhost:9092
    16. - key: group.id
    17. value: testGroup
    18. format:
    19. property-version: 1
    20. type: json
    21. schema: "ROW<rideId LONG, lon FLOAT, lat FLOAT, rideTime TIMESTAMP>"
    22. schema:
    23. - name: rideId
    24. type: LONG
    25. - name: lon
    26. type: FLOAT
    27. - name: lat
    28. type: FLOAT
    29. - name: rowTime
    30. type: TIMESTAMP
    31. rowtime:
    32. timestamps:
    33. type: "from-field"
    34. from: "rideTime"
    35. watermarks:
    36. type: "periodic-bounded"
    37. delay: "60000"
    38. - name: procTime
    39. type: TIMESTAMP
    40. proctime: true

    The resulting schema of the TaxiRide table contains most of the fields of the JSON schema. Furthermore, it adds a rowtime attribute rowTime and processing-time attribute procTime.

    Both connector and format allow to define a property version (which is currently version 1) for future backwards compatibility.

    User-defined Functions

    The SQL Client allows users to create custom, user-defined functions to be used in SQL queries. Currently, these functions are restricted to be defined programmatically in Java/Scala classes.

    In order to provide a user-defined function, you need to first implement and compile a function class that extends ScalarFunction, AggregateFunction or TableFunction (see User-defined Functions). One or more functions can then be packaged into a dependency JAR for the SQL Client.

    All functions must be declared in an environment file before being called. For each item in the list of functions, one must specify

    • a name under which the function is registered,
    • the source of the function using from (restricted to be class for now),
    • the class which indicates the fully qualified class name of the function and an optional list of constructor parameters for instantiation.
    1. functions:
    2. - name: ... # required: name of the function
    3. from: class # required: source of the function (can only be "class" for now)
    4. class: ... # required: fully qualified class name of the function
    5. constructor: # optimal: constructor parameters of the function class
    6. - ... # optimal: a literal parameter with implicit type
    7. - class: ... # optimal: full class name of the parameter
    8. constructor: # optimal: constructor parameters of the parameter's class
    9. - type: ... # optimal: type of the literal parameter
    10. value: ... # optimal: value of the literal parameter

    Make sure that the order and types of the specified parameters strictly match one of the constructors of your function class.

    Constructor Parameters

    Depending on the user-defined function, it might be necessary to parameterize the implementation before using it in SQL statements.

    As shown in the example before, when declaring a user-defined function, a class can be configured using constructor parameters in one of the following three ways:

    A literal value with implicit type: The SQL Client will automatically derive the type according to the literal value itself. Currently, only values of BOOLEAN, INT, DOUBLE and VARCHAR are supported here.If the automatic derivation does not work as expected (e.g., you need a VARCHAR false), use explicit types instead.

    1. - true # -> BOOLEAN (case sensitive)
    2. - 42 # -> INT
    3. - 1234.222 # -> DOUBLE
    4. - foo # -> VARCHAR

    A literal value with explicit type: Explicitly declare the parameter with type and value properties for type-safety.

    1. - type: DECIMAL
    2. value: 11111111111111111

    The table below illustrates the supported Java parameter types and the corresponding SQL type strings.

    Java typeSQL type
    java.math.BigDecimalDECIMAL
    java.lang.BooleanBOOLEAN
    java.lang.ByteTINYINT
    java.lang.DoubleDOUBLE
    java.lang.FloatREAL, FLOAT
    java.lang.IntegerINTEGER, INT
    java.lang.LongBIGINT
    java.lang.ShortSMALLINT
    java.lang.StringVARCHAR

    More types (e.g., TIMESTAMP or ARRAY), primitive types, and null are not supported yet.

    A (nested) class instance: Besides literal values, you can also create (nested) class instances for constructor parameters by specifying the class and constructor properties.This process can be recursively performed until all the constructor parameters are represented with literal values.

    1. - class: foo.bar.paramClass
    2. constructor:
    3. - StarryName
    4. - class: java.lang.Integer
    5. constructor:
    6. - class: java.lang.String
    7. constructor:
    8. - type: VARCHAR
    9. value: 3

    Catalogs

    Catalogs can be defined as a set of YAML properties and are automatically registered to the environment upon starting SQL Client.

    Users can specify which catalog they want to use as the current catalog in SQL CLI, and which database of the catalog they want to use as the current database.

    1. catalogs:
    2. - name: catalog_1
    3. type: hive
    4. property-version: 1
    5. default-database: mydb2
    6. hive-version: 1.2.1
    7. hive-conf-dir: <path of Hive conf directory>
    8. - name: catalog_2
    9. type: hive
    10. property-version: 1
    11. hive-conf-dir: <path of Hive conf directory>
    12. execution:
    13. ...
    14. current-catalog: catalog_1
    15. current-database: mydb1

    For more information about catalogs, see Catalogs.

    Detached SQL Queries

    In order to define end-to-end SQL pipelines, SQL’s INSERT INTO statement can be used for submitting long-running, detached queries to a Flink cluster. These queries produce their results into an external system instead of the SQL Client. This allows for dealing with higher parallelism and larger amounts of data. The CLI itself does not have any control over a detached query after submission.

    1. INSERT INTO MyTableSink SELECT * FROM MyTableSource

    The table sink MyTableSink has to be declared in the environment file. See the connection page for more information about supported external systems and their configuration. An example for an Apache Kafka table sink is shown below.

    1. tables:
    2. - name: MyTableSink
    3. type: sink-table
    4. update-mode: append
    5. connector:
    6. property-version: 1
    7. type: kafka
    8. version: "0.11"
    9. topic: OutputTopic
    10. properties:
    11. - key: zookeeper.connect
    12. value: localhost:2181
    13. - key: bootstrap.servers
    14. value: localhost:9092
    15. - key: group.id
    16. value: testGroup
    17. format:
    18. property-version: 1
    19. type: json
    20. derive-schema: true
    21. schema:
    22. - name: rideId
    23. type: LONG
    24. - name: lon
    25. type: FLOAT
    26. - name: lat
    27. type: FLOAT
    28. - name: rideTime
    29. type: TIMESTAMP

    The SQL Client makes sure that a statement is successfully submitted to the cluster. Once the query is submitted, the CLI will show information about the Flink job.

    1. [INFO] Table update statement has been successfully submitted to the cluster:
    2. Cluster ID: StandaloneClusterId
    3. Job ID: 6f922fe5cba87406ff23ae4a7bb79044
    4. Web interface: http://localhost:8081

    Attention The SQL Client does not track the status of the running Flink job after submission. The CLI process can be shutdown after the submission without affecting the detached query. Flink’s restart strategy takes care of the fault-tolerance. A query can be cancelled using Flink’s web interface, command-line, or REST API.

    SQL Views

    Views allow to define virtual tables from SQL queries. The view definition is parsed and validated immediately. However, the actual execution happens when the view is accessed during the submission of a general INSERT INTO or SELECT statement.

    Views can either be defined in environment files or within the CLI session.

    The following example shows how to define multiple views in a file. The views are registered in the order in which they are defined in the environment file. Reference chains such as view A depends on view B depends on view C are supported.

    1. tables:
    2. - name: MyTableSource
    3. # ...
    4. - name: MyRestrictedView
    5. type: view
    6. query: "SELECT MyField2 FROM MyTableSource"
    7. - name: MyComplexView
    8. type: view
    9. query: >
    10. SELECT MyField2 + 42, CAST(MyField1 AS VARCHAR)
    11. FROM MyTableSource
    12. WHERE MyField2 > 200

    Similar to table sources and sinks, views defined in a session environment file have highest precedence.

    Views can also be created within a CLI session using the CREATE VIEW statement:

    1. CREATE VIEW MyNewView AS SELECT MyField2 FROM MyTableSource;

    Views created within a CLI session can also be removed again using the DROP VIEW statement:

    1. DROP VIEW MyNewView;

    Attention The definition of views in the CLI is limited to the mentioned syntax above. Defining a schema for views or escaping whitespaces in table names will be supported in future versions.

    Temporal Tables

    A temporal table allows for a (parameterized) view on a changing history table that returns the content of a table at a specific point in time. This is especially useful for joining a table with the content of another table at a particular timestamp. More information can be found in the temporal table joins page.

    The following example shows how to define a temporal table SourceTemporalTable:

    1. tables:
    2. # Define the table source (or view) that contains updates to a temporal table
    3. - name: HistorySource
    4. type: source-table
    5. update-mode: append
    6. connector: # ...
    7. format: # ...
    8. schema:
    9. - name: integerField
    10. type: INT
    11. - name: stringField
    12. type: VARCHAR
    13. - name: rowtimeField
    14. type: TIMESTAMP
    15. rowtime:
    16. timestamps:
    17. type: from-field
    18. from: rowtimeField
    19. watermarks:
    20. type: from-source
    21. # Define a temporal table over the changing history table with time attribute and primary key
    22. - name: SourceTemporalTable
    23. type: temporal-table
    24. history-table: HistorySource
    25. primary-key: integerField
    26. time-attribute: rowtimeField # could also be a proctime field

    As shown in the example, definitions of table sources, views, and temporal tables can be mixed with each other. They are registered in the order in which they are defined in the environment file. For example, a temporal table can reference a view which can depend on another view or table source.

    Limitations & Future

    The current SQL Client implementation is in a very early development stage and might change in the future as part of the bigger Flink Improvement Proposal 24 (FLIP-24). Feel free to join the discussion and open issue about bugs and features that you find useful.