...
SQL operation | Cluster Client Command |
ShowQueriesOperation | ClusterClient#listJobs |
DropQueryOperation | ClusterClient#stoplWithSavepoint |
DropQueryPurgeOperation | ClusterClient#cancel |
SavepointOperation | ClusterClient#triggerSavepoint |
SqlReleaseSavepointReleaseSavepointOperation | ClusterClient#disposeSavepoint |
...
- QUERY (new)
- QUERIES (new)
- RELEASE (new)
- SAVEPOINT (already reserved)
Rejected Alternatives
Book Keep Query Status in SQL Gateway
An alternative approach to query monitoring is that the SQL client or gateway book keeps every query and is responsible for updating the query status through polling or callbacks. In that way, the query status is better maintained, and we wouldn’t lose track of the queries in cases that they’re cleaned up by the cluster or the cluster is unavailable.
...
- Table/SQL API should provide the same capabilities as its peer DataStream API, thus show queries statement implement should be aligned with flink list in CLI as well.
- Maintaining query status at the client/gateway side requires additional work but brings little extra user value, since the client/gateway doesn’t persist metadata at the moment.
Savepoint Syntax: CREATE/DROP SAVEPOINT
An alternative syntax of savepoints is like:
Code Block | ||
---|---|---|
| ||
CREATE SAVEPOINT '<query_id>'
DROP SAVEPOINT '<savepoint_path>' |
But there are mainly two concerns:
- Savepoint keyword is already used in SQL standard (`SAVEPOINT '<savepoint_spec>'` / `RELEASE SAVEPOINT '<savepoint_spec>'`), so it might cause conflicts.
- Generally speaking, CREATE SAVEPOINT is more appropriate to be followed by a savepoint identifier instead of a query identifier.