Interface TableResult

  • All Known Subinterfaces:
    TableResultInternal
    All Known Implementing Classes:
    TableResultImpl

    @PublicEvolving
    public interface TableResult
    A TableResult is the representation of the statement execution result.
    • Method Summary

      All Methods Instance Methods Abstract Methods Default Methods Deprecated Methods 
      Modifier and Type Method Description
      void await()
      Wait if necessary until the data is ready.
      void await​(long timeout, TimeUnit unit)
      Wait if necessary for at most the given time for the data to be ready.
      org.apache.flink.util.CloseableIterator<org.apache.flink.types.Row> collect()
      Get the result contents as a closeable row iterator.
      Optional<org.apache.flink.core.execution.JobClient> getJobClient()
      For DML and DQL statement, return the JobClient which associates the submitted Flink job.
      org.apache.flink.table.catalog.ResolvedSchema getResolvedSchema()
      Returns the schema of the result.
      ResultKind getResultKind()
      Return the ResultKind which represents the result type.
      default org.apache.flink.table.legacy.api.TableSchema getTableSchema()
      Deprecated.
      This method has been deprecated as part of FLIP-164.
      void print()
      Print the result contents as tableau form to client console.
    • Method Detail

      • getJobClient

        Optional<org.apache.flink.core.execution.JobClient> getJobClient()
        For DML and DQL statement, return the JobClient which associates the submitted Flink job. For other statements (e.g. DDL, DCL) return empty.
      • await

        void await()
            throws InterruptedException,
                   ExecutionException
        Wait if necessary until the data is ready.

        For a select operation, this method will wait until the first row can be accessed locally. For an insert operation, this method will wait for the job to finish, because the result contains only one row. For other operations, this method will return immediately, because the result is already available locally.

        Throws:
        ExecutionException - if a problem occurred
        InterruptedException - if the operation was interrupted while waiting
      • await

        void await​(long timeout,
                   TimeUnit unit)
            throws InterruptedException,
                   ExecutionException,
                   TimeoutException
        Wait if necessary for at most the given time for the data to be ready.

        For a select operation, this method will wait until the first row can be accessed locally. For an insert operation, this method will wait for the job to finish, because the result contains only one row. For other operations, this method will return immediately, because the result is already available locally.

        Parameters:
        timeout - the maximum time to wait
        unit - the time unit of the timeout argument
        Throws:
        ExecutionException - if a problem occurred
        InterruptedException - if the operation was interrupted while waiting
        TimeoutException - if the wait timed out
      • getResolvedSchema

        org.apache.flink.table.catalog.ResolvedSchema getResolvedSchema()
        Returns the schema of the result.

        The schema of DDL, USE, EXPLAIN:

         +-------------+-------------+----------+
         | column name | column type | comments |
         +-------------+-------------+----------+
         | result      | STRING      |          |
         +-------------+-------------+----------+
         

        The schema of SHOW:

         +---------------+-------------+----------+
         |  column name  | column type | comments |
         +---------------+-------------+----------+
         | <object name> | STRING      |          |
         +---------------+-------------+----------+
         The column name of `SHOW CATALOGS` is "catalog name",
         the column name of `SHOW DATABASES` is "database name",
         the column name of `SHOW TABLES` is "table name",
         the column name of `SHOW VIEWS` is "view name",
         the column name of `SHOW FUNCTIONS` is "function name",
         the column name of `SHOW MODULES` is "module name".
         

        The schema of DESCRIBE:

         +------------------+-------------+-----------------------------------------------------------------------------+
         | column name      | column type |                              comments                                       |
         +------------------+-------------+-----------------------------------------------------------------------------+
         | name             | STRING      | field name                                                                  |
         | type             | STRING      | field type expressed as a String                                            |
         | null             | BOOLEAN     | field nullability: true if a field is nullable, else false                  |
         | key              | BOOLEAN     | key constraint: 'PRI' for primary keys, 'UNQ' for unique keys, else null    |
         | extras           | STRING      | extras such as computed or metadata column information, else null           |
         | watermark        | STRING      | watermark: string expression if a field is watermark, else null             |
         +------------------+-------------+-----------------------------------------------------------------------------+
         

        The schema of INSERT: (one column per one sink)

         +----------------------------+-------------+-----------------------+
         | column name                | column type | comments              |
         +----------------------------+-------------+-----------------------+
         | (name of the insert table) | BIGINT      | the insert table name |
         +----------------------------+-------------+-----------------------+
         

        The schema of SELECT is the selected field names and types.

      • getTableSchema

        @Deprecated
        default org.apache.flink.table.legacy.api.TableSchema getTableSchema()
        Deprecated.
        This method has been deprecated as part of FLIP-164. TableSchema has been replaced by two more dedicated classes Schema and ResolvedSchema. Use Schema for declaration in APIs. ResolvedSchema is offered by the framework after resolution and validation.
        Returns the schema of the result.
      • collect

        org.apache.flink.util.CloseableIterator<org.apache.flink.types.Row> collect()
        Get the result contents as a closeable row iterator.

        NOTE:

        • For SELECT operation, the job will not be finished unless all result data has been collected. So we should actively close the job to avoid resource leak through CloseableIterator#close method. Calling CloseableIterator#close method will cancel the job and release related resources.
        • For DML operation, Flink does not support getting the real affected row count now. So the affected row count is always -1 (unknown) for every sink, and them will be returned until the job is finished. Calling CloseableIterator#close method will cancel the job.
        • For other operations, no flink job will be submitted (getJobClient() is always empty), and the result is bounded. Do nothing when calling CloseableIterator#close method.

        Recommended code to call CloseableIterator#close method looks like:

        
         TableResult result = tEnv.execute("select ...");
         // using try-with-resources statement
         try (CloseableIterator<Row> it = result.collect()) {
             it... // collect same data
         }
         

        This method has slightly different behaviors under different checkpointing settings (to enable checkpointing for a streaming job, set checkpointing properties through TableConfig).

        • For batch jobs or streaming jobs without checkpointing, this method has neither exactly-once nor at-least-once guarantee. Query results are immediately accessible by the clients once they're produced, but exceptions will be thrown when the job fails and restarts.
        • For streaming jobs with exactly-once checkpointing, this method guarantees an end-to-end exactly-once record delivery. A result will be accessible by clients only after its corresponding checkpoint completes.
        • For streaming jobs with at-least-once checkpointing, this method guarantees an end-to-end at-least-once record delivery. Query results are immediately accessible by the clients once they're produced, but it is possible for the same result to be delivered multiple times.

        In order to fetch result to local, you can call either collect() and print(). But, they can't be called both on the same TableResult instance, because the result can only be accessed once.

      • print

        void print()
        Print the result contents as tableau form to client console.

        This method has slightly different behaviors under different checkpointing settings (to enable checkpointing for a streaming job, set checkpointing properties through TableConfig).

        • For batch jobs or streaming jobs without checkpointing, this method has neither exactly-once nor at-least-once guarantee. Query results are immediately accessible by the clients once they're produced, but exceptions will be thrown when the job fails and restarts.
        • For streaming jobs with exactly-once checkpointing, this method guarantees an end-to-end exactly-once record delivery. A result will be accessible by clients only after its corresponding checkpoint completes.
        • For streaming jobs with at-least-once checkpointing, this method guarantees an end-to-end at-least-once record delivery. Query results are immediately accessible by the clients once they're produced, but it is possible for the same result to be delivered multiple times.

        In order to fetch result to local, you can call either collect() and print(). But, they can't be called both on the same TableResult instance, because the result can only be accessed once.