...
Also, sqlQuery provided a bridge from sql query to Table, but the name `sqlQuery` seems confusing on the behavior: do I really read something from the sql? To eliminate the ambiguity we suggest to provide a fromQuery method similar to `Table from(String path);` but obtain one table from one sql query stmt. Suggested methods:
/** * Get a Table from one sql query stmt, the sql must be a `select` stmt * and one single statement. */ Table fromQuery(String sql) |
`void sqlUpdate(String sql)`
Now `void sqlUpdate(String sql)` will execute DDLs right now while DMLs(such as `insert into xx`) must be triggered by `TableEnvironment.execute()`. Both behaviors should be kept consistent. So we propose a new blocking execute method with returned value to fetch execute the result and this method should be deprecated.
Suggested methods:
/** * ddl/dml/`select` but the sql argument must be a single statement. * @return null for `ddl` and `insert into` sql stmt * ResultTable for `show xx` `describe xx` `select xx` |
Suggested ResultTable Interface:
/** * result. |
`Table fromTableSource(TableSource<?> source)`
...
Notice, input SQLs can not be ddl or select sql similar to the JDBC statement[4]. We have such requirements for passed ddl will cause behavior unexpectedly. Suggested method:
/** |
Discuss a parse method for multiple statements execute in SQL CLI
...
Methods of TableEnvironment
Current Call | Replacement | Comment |
execute(String jobName) | deprecated | |
explain(boolean extended) | deprecated | |
sqlQuery(String sql) | fromQuery | |
sqlUpdate(String sql) | executeSql() | |
fromTableSource(TableSource tableSouce) |
New suggested TableEnvironment methods:
added method Call | Comment |
Table fromQuery(String sql) | Obtain a table from sql query stmt |
ResultTable executeSql(String sql) | |
void executeBatch(String... sql) | sql limit to insert |
Future Plan:
Notice: depends on FLIP-73/FLIP-74, not the scope of this flip.
...
Similarly as above, suggested methods:
/** * ddl/dml/`select` and the sql argument must be a single statement. |
We also should support execute batch sql asynchronous. Suggested method:
CompletableFuture<JobClient> asyncExecBatchSql(String... sql) |
Add an async execute method to org.apache.flink.table.delegation.Executor
/** |
Since we already have flips[7][8] to provide asynchronous management, it’s convenient and natural to provide such a method.
Add an async execute method to org.apache.flink.streaming.api.environment#StreamExecutionEnvironment
public abstract CompletableFuture<JobExecutionResult> asyncExecute(StreamGraph streamGraph) throws Exception; |
SQL CLI integrates with new API
...
If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.
References
[1] FLIP-69 Flink SQL DDL Enhancement
[2] discuss planner buffer execute
[3] FLIP-64: Support for Temporary Objects in Table module
[4] JDBC statement addBatch interface
[5] multiple statements in SQL CLI
[6] multiple statements in TableEnvironment
[7] flip-73 Introducing Executors for job submission