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

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_r136124917
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala
 ---
    @@ -0,0 +1,33 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.schema
    +
    +import org.apache.flink.table.api.TableEnvironment
    +import org.apache.flink.table.plan.stats.FlinkStatistic
    +import org.apache.flink.table.sinks.TableSink
    +
    +/** Table which defines an external table via a [[TableSink]] */
    +class TableSinkTable[T](
    +    val tableSink: TableSink[T],
    +    override val statistic: FlinkStatistic = FlinkStatistic.UNKNOWN)
    +  extends FlinkTable[T](
    --- End diff --
    
    I don't think we should extend `FlinkTable` but go directly against 
Calcite's `AbstractTable`. `FlinkTable` deals with many issues like POJO types 
that are not required for table sinks.
    
    `TableSinkTable` can override `getRowType()` by requesting field names and 
types directly from the `TableSink`:
    
    ```
    override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = {
      val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory]
      flinkTypeFactory.buildLogicalRowType(
        tableSink.getFieldNames,
        tableSink.getFieldTypes,
        None,
        None)
    }
    ```


> 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