[
https://issues.apache.org/jira/browse/FLINK-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16170682#comment-16170682
]
ASF GitHub Bot commented on FLINK-6442:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/3829#discussion_r139524452
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
---
@@ -502,23 +521,130 @@ abstract class TableEnvironment(val config:
TableConfig) {
* @param query The SQL query to evaluate.
* @return The result of the query as Table.
*/
+ @deprecated("use [[sqlQuery()]] instead")
def sql(query: String): Table = {
+ sqlQuery(query)
+ }
+
+ /**
+ * Evaluates a SQL Select query on registered tables and retrieves the
result as a
+ * [[Table]].
+ *
+ * All tables referenced by the query must be registered in the
TableEnvironment. But
+ * [[Table.toString]] will automatically register an unique table name
and return the
+ * table name. So it allows to call SQL directly on tables like this:
+ *
+ * {{{
+ * val table: Table = ...
+ * // the table is not registered to the table environment
+ * tEnv.sqlSelect(s"SELECT * FROM $table")
+ * }}}
+ *
+ * @param query The SQL query to evaluate.
+ * @return The result of the query as Table
+ */
+ def sqlQuery(query: String): Table = {
val planner = new FlinkPlannerImpl(getFrameworkConfig, getPlanner,
getTypeFactory)
// parse the sql query
val parsed = planner.parse(query)
- // validate the sql query
- val validated = planner.validate(parsed)
- // transform to a relational tree
- val relational = planner.rel(validated)
+ if (null != parsed && parsed.getKind.belongsTo(SqlKind.QUERY)) {
+ // validate the sql query
+ val validated = planner.validate(parsed)
+ // transform to a relational tree
+ val relational = planner.rel(validated)
+ new Table(this, LogicalRelNode(relational.rel))
+ } else {
+ throw new TableException(
+ "Unsupported SQL query! sqlQuery() only accepts SQL queries of
type SELECT, UNION, " +
+ "INTERSECT, EXCEPT, VALUES, WITH, ORDER_BY, and EXPLICIT_TABLE.")
+ }
+ }
+
+ /**
+ * Evaluates a SQL statement such as INSERT, UPDATE or DELETE; or a DDL
statement;
+ * Currently only SQL INSERT statement on registered tables are
supported.
--- End diff --
+s `statementS`
> Extend TableAPI Support Sink Table Registration and ‘insert into’ Clause in
> SQL
> -------------------------------------------------------------------------------
>
> Key: FLINK-6442
> URL: https://issues.apache.org/jira/browse/FLINK-6442
> Project: Flink
> Issue Type: New Feature
> Components: Table API & SQL
> Reporter: lincoln.lee
> Assignee: lincoln.lee
> Priority: Minor
>
> Currently in TableAPI there’s only registration method for source table,
> when we use SQL writing a streaming job, we should add additional part for
> the sink, like TableAPI does:
> {code}
> val sqlQuery = "SELECT * FROM MyTable WHERE _1 = 3"
> val t = StreamTestData.getSmall3TupleDataStream(env)
> tEnv.registerDataStream("MyTable", t)
> // one way: invoke tableAPI’s writeToSink method directly
> val result = tEnv.sql(sqlQuery)
> result.writeToSink(new YourStreamSink)
> // another way: convert to datastream first and then invoke addSink
> val result = tEnv.sql(sqlQuery).toDataStream[Row]
> result.addSink(new StreamITCase.StringSink)
> {code}
> From the api we can see the sink table always be a derived table because its
> 'schema' is inferred from the result type of upstream query.
> Compare to traditional RDBMS which support DML syntax, a query with a target
> output could be written like this:
> {code}
> insert into table target_table_name
> [(column_name [ ,...n ])]
> query
> {code}
> The equivalent form of the example above is as follows:
> {code}
> tEnv.registerTableSink("targetTable", new YourSink)
> val sql = "INSERT INTO targetTable SELECT a, b, c FROM sourceTable"
> val result = tEnv.sql(sql)
> {code}
> It is supported by Calcite’s grammar:
> {code}
> insert:( INSERT | UPSERT ) INTO tablePrimary
> [ '(' column [, column ]* ')' ]
> query
> {code}
> I'd like to extend Flink TableAPI to support such feature. see design doc:
> https://goo.gl/n3phK5
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)