[ 
https://issues.apache.org/jira/browse/FLINK-6442?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16171256#comment-16171256
 ] 

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_r139621820
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
 ---
    @@ -130,6 +130,72 @@ abstract class StreamTableEnvironment(
       }
     
       /**
    +    * Registers an external [[TableSink]] with given field names and types 
in this
    +    * [[TableEnvironment]]'s catalog. Registered sink tables can be 
referenced in SQL DML clause.
    +    *
    +    * Examples:
    +    *
    +    * - predefine a table sink and its field names and types
    +    * {{{
    +    *   val fieldNames: Array[String]  = Array("a", "b", "c")
    +    *   val fieldTypes: Array[TypeInformation[_]]  = Array(Types.STRING, 
Types.INT, Types.LONG)
    +    *   val tableSink: TableSink = new YourTableSinkImpl(...)
    +    * }}}
    +    *
    +    * -  register an alias for this table sink to catalog
    +    * {{{
    +    *   tableEnv.registerTableSink("example_sink_table", fieldNames, 
fieldsTypes, tableSink)
    +    * }}}
    +    *
    +    * -  use the registered sink in SQL directly
    +    * {{{
    +    *   tableEnv.sqlInsert("INSERT INTO example_sink_table SELECT a, b, c 
FROM sourceTable")
    +    * }}}
    +    *
    +    * @param name The name under which the [[TableSink]] is registered.
    +    * @param fieldNames The field names related to the [[TableSink]].
    +    * @param fieldTypes The field types related to the [[TableSink]].
    +    * @param tableSink The [[TableSink]] to register.
    +    */
    +  def registerTableSink(
    +      name: String,
    +      fieldNames: Array[String],
    +      fieldTypes: Array[TypeInformation[_]],
    +      tableSink: TableSink[_]): Unit = {
    +    checkValidTableName(name)
    +    if (null == fieldNames || null == fieldTypes) {
    +      throw new TableException("fieldNames and fieldTypes should not be 
empty!")
    +    }
    +    if (fieldNames.length != fieldTypes.length) {
    +      val errorMsg = "Field names and types should have same length! 
Passing fieldNames " +
    +        s"length: ${fieldNames.length} but fieldTypes length: 
${fieldTypes.length}"
    +      throw new TableException(errorMsg)
    +    }
    +
    +    tableSink match {
    +      case streamTableSink@(_: AppendStreamTableSink[_] | _: 
UpsertStreamTableSink[_] |
    +              _: RetractStreamTableSink[_]) =>
    +        val configuredSink = streamTableSink.configure(fieldNames, 
fieldTypes)
    +        registerTableInternal(name, new TableSinkTable(configuredSink))
    +      case _ =>
    +        throw new TableException(
    +          "Only AppendStreamTableSink, UpsertStreamTableSink and 
RetractStreamTableSink can be " +
    +            "registered in StreamTableEnvironment.")
    +    }
    +  }
    +
    +  /**
    +    * Registers a [[Table]] under a unique name in the TableEnvironment's 
catalog.
    +    * Registered tables can be referenced in SQL queries.
    +    *
    +    * @param name  The name under which the table will be registered.
    +    * @param table The table to register.
    +    */
    +  override def registerTable(
    --- End diff --
    
    thx for the confirmation


> 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)

Reply via email to