[
https://issues.apache.org/jira/browse/FLINK-16366?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
godfrey he updated FLINK-16366:
-------------------------------
Description:
This issue aims to introduce {{executeSql}} which executes the given single
statement, and returns the execution result.
{code:java}
interface TableEnvironment {
/**
* Execute the given single statement and
* the statement can be DDL/DML/DQL/SHOW/DESCRIBE/EXPLAIN/USE.
*
* If the statement is translated to a Flink job (DML/DQL),
* the TableResult will be returned until the job is submitted, and
* contains a JobClient instance to associate the job.
* Else, the TableResult will be returned until the statement
* execution is finished, does not contain a JobClient instance.
*
* @return result for DQL/SHOW/DESCRIBE/EXPLAIN, the affected row count
* for `DML` (-1 means unknown),or a string message ("OK") for other
* statements.
*/
TableResult executeSql(String statement);
}
/**
* A TableResult is the representation of the statement execution result.
*/
interface TableResult {
/**
* return JobClient if a Flink job is submitted
* (for DML/DQL statement), else return empty (e.g. DDL).
*/
Optional<JobClient> getJobClient();
/**
* Get the schema of result.
*/
TableSchema getTableSchema();
/**
* return the ResultKind which can avoid custom parsing of
* an "OK" row in programming
*/
ResultKind getResultKind();
/**
* Get the result contents as an iterable rows.
*/
Iterator<Row> collect();
/**
* Print the result contents.
*/
void print();
}
/**
* ResultKind defines the types of the result.
*/
public enum ResultKind {
// for DDL, DCL and statements with a simple "OK"
SUCCESS,
// rows with important content are available (DML, DQL)
SUCCESS_WITH_CONTENT
}
{code}
was:
This issue aims to introduce {{executeStatement}} which synchronously executes
the given single statement immediately, and returns the execution result.
{code:java}
/**
* Synchronously execute the given single statement immediately and the
statement can be DDL/DML/SHOW/DESCRIBE/EXPLAIN/USE.
* If the statement is translated to a Flink job, the result will be returned
until the job is finished.
*
* @return result for SHOW/DESCRIBE/EXPLAIN, the affected row count for `DML`
(-1 means unknown), or a string message ("OK") for other statements.
* @throws Exception which occurs during the execution.
*/
ResultTable executeStatement(String statement) throws Exception;
/**
* A ResultTable is the representation of the statement execution result.
*/
public interface ResultTable {
/**
* Get the schema of ResultTable.
*/
TableSchema getResultSchema();
/**
*Get the result contents as an iterable rows.
*/
Iterable<Row> getResultRows();
}
{code}
> Introduce executeSql method in TableEnvironment
> -----------------------------------------------
>
> Key: FLINK-16366
> URL: https://issues.apache.org/jira/browse/FLINK-16366
> Project: Flink
> Issue Type: Sub-task
> Components: Table SQL / API
> Reporter: godfrey he
> Priority: Major
> Fix For: 1.11.0
>
>
> This issue aims to introduce {{executeSql}} which executes the given single
> statement, and returns the execution result.
> {code:java}
> interface TableEnvironment {
> /**
> * Execute the given single statement and
> * the statement can be DDL/DML/DQL/SHOW/DESCRIBE/EXPLAIN/USE.
> *
> * If the statement is translated to a Flink job (DML/DQL),
> * the TableResult will be returned until the job is submitted, and
> * contains a JobClient instance to associate the job.
> * Else, the TableResult will be returned until the statement
> * execution is finished, does not contain a JobClient instance.
> *
> * @return result for DQL/SHOW/DESCRIBE/EXPLAIN, the affected row count
> * for `DML` (-1 means unknown),or a string message ("OK") for other
> * statements.
> */
> TableResult executeSql(String statement);
> }
> /**
> * A TableResult is the representation of the statement execution result.
> */
> interface TableResult {
> /**
> * return JobClient if a Flink job is submitted
> * (for DML/DQL statement), else return empty (e.g. DDL).
> */
> Optional<JobClient> getJobClient();
>
> /**
> * Get the schema of result.
> */
> TableSchema getTableSchema();
>
> /**
> * return the ResultKind which can avoid custom parsing of
> * an "OK" row in programming
> */
> ResultKind getResultKind();
>
> /**
> * Get the result contents as an iterable rows.
> */
> Iterator<Row> collect();
>
> /**
> * Print the result contents.
> */
> void print();
> }
> /**
> * ResultKind defines the types of the result.
> */
> public enum ResultKind {
> // for DDL, DCL and statements with a simple "OK"
> SUCCESS,
>
> // rows with important content are available (DML, DQL)
> SUCCESS_WITH_CONTENT
> }
> {code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)