@PublicEvolving public interface TableResult
Modifier and Type | Method and 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.
|
CloseableIterator<Row> |
collect()
Get the result contents as a closeable row iterator.
|
Optional<JobClient> |
getJobClient()
For DML and DQL statement, return the
JobClient which associates the submitted Flink
job. |
ResolvedSchema |
getResolvedSchema()
Returns the schema of the result.
|
ResultKind |
getResultKind()
Return the
ResultKind which represents the result type. |
default 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. |
void |
print()
Print the result contents as tableau form to client console.
|
Optional<JobClient> getJobClient()
JobClient
which associates the submitted Flink
job. For other statements (e.g. DDL, DCL) return empty.void await() throws InterruptedException, ExecutionException
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.
ExecutionException
- if a problem occurredInterruptedException
- if the operation was interrupted while waitingvoid await(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
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.
timeout
- the maximum time to waitunit
- the time unit of the timeout argumentExecutionException
- if a problem occurredInterruptedException
- if the operation was interrupted while waitingTimeoutException
- if the wait timed outResolvedSchema getResolvedSchema()
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.
@Deprecated default TableSchema getTableSchema()
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.ResultKind getResultKind()
ResultKind
which represents the result type.
For DDL operation and USE operation, the result kind is always ResultKind.SUCCESS
.
For other operations, the result kind is always ResultKind.SUCCESS_WITH_CONTENT
.
CloseableIterator<Row> collect()
NOTE:
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
).
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.
void print()
This method has slightly different behaviors under different checkpointing settings (to
enable checkpointing for a streaming job, set checkpointing properties through TableConfig
).
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.
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.