Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 1BB51200BD3 for ; Tue, 6 Dec 2016 13:42:57 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 19CF9160B1B; Tue, 6 Dec 2016 12:42:57 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 3EA53160B0C for ; Tue, 6 Dec 2016 13:42:56 +0100 (CET) Received: (qmail 9666 invoked by uid 500); 6 Dec 2016 12:42:54 -0000 Mailing-List: contact commits-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list commits@flink.apache.org Received: (qmail 9390 invoked by uid 99); 6 Dec 2016 12:42:49 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 06 Dec 2016 12:42:49 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 5F2C9F16A5; Tue, 6 Dec 2016 12:42:49 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: fhueske@apache.org To: commits@flink.apache.org Date: Tue, 06 Dec 2016 12:42:52 -0000 Message-Id: <65c54b27404a4ff48ca635d4f4e42634@git.apache.org> In-Reply-To: <203a1fa116244c44b39e7ac989247e40@git.apache.org> References: <203a1fa116244c44b39e7ac989247e40@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [4/4] flink git commit: [FLINK-5251] [table] Decouple StreamTableSourceScan from TableSourceTable. archived-at: Tue, 06 Dec 2016 12:42:57 -0000 [FLINK-5251] [table] Decouple StreamTableSourceScan from TableSourceTable. This closes #2934. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/98d18260 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/98d18260 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/98d18260 Branch: refs/heads/master Commit: 98d1826030d1486a3d64466aff1c909a41e2de10 Parents: 6f9633c Author: Kurt Young Authored: Mon Dec 5 09:43:13 2016 +0800 Committer: Fabian Hueske Committed: Tue Dec 6 11:23:28 2016 +0100 ---------------------------------------------------------------------- .../plan/nodes/datastream/DataStreamScan.scala | 6 ++++-- .../table/plan/nodes/datastream/StreamScan.scala | 5 +---- .../nodes/datastream/StreamTableSourceScan.scala | 19 ++++++++++--------- .../datastream/StreamTableSourceScanRule.scala | 6 +++++- 4 files changed, 20 insertions(+), 16 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/98d18260/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamScan.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamScan.scala index 463e1bc..da83b64 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamScan.scala @@ -35,11 +35,13 @@ class DataStreamScan( cluster: RelOptCluster, traitSet: RelTraitSet, table: RelOptTable, - rowType: RelDataType) - extends StreamScan(cluster, traitSet, table, rowType) { + rowRelDataType: RelDataType) + extends StreamScan(cluster, traitSet, table) { val dataStreamTable: DataStreamTable[Any] = getTable.unwrap(classOf[DataStreamTable[Any]]) + override def deriveRowType() = rowRelDataType + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { new DataStreamScan( cluster, http://git-wip-us.apache.org/repos/asf/flink/blob/98d18260/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala index 17620d0..b13770e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala @@ -37,13 +37,10 @@ import scala.collection.JavaConverters._ abstract class StreamScan( cluster: RelOptCluster, traitSet: RelTraitSet, - table: RelOptTable, - rowRelDataType: RelDataType) + table: RelOptTable) extends TableScan(cluster, traitSet, table) with DataStreamRel { - override def deriveRowType() = rowRelDataType - protected def convertToExpectedType( input: DataStream[Any], flinkTable: FlinkTable[_], http://git-wip-us.apache.org/repos/asf/flink/blob/98d18260/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala index 21b8a63..8201070 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala @@ -20,11 +20,10 @@ package org.apache.flink.api.table.plan.nodes.datastream import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.`type`.RelDataType import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.table.StreamTableEnvironment import org.apache.flink.api.table.plan.schema.TableSourceTable import org.apache.flink.api.table.sources.StreamTableSource +import org.apache.flink.api.table.{FlinkTypeFactory, StreamTableEnvironment} import org.apache.flink.streaming.api.datastream.DataStream /** Flink RelNode to read data from an external source defined by a [[StreamTableSource]]. */ @@ -32,18 +31,20 @@ class StreamTableSourceScan( cluster: RelOptCluster, traitSet: RelTraitSet, table: RelOptTable, - rowType: RelDataType) - extends StreamScan(cluster, traitSet, table, rowType) { + tableSource: StreamTableSource[_]) + extends StreamScan(cluster, traitSet, table) { - val tableSourceTable = table.unwrap(classOf[TableSourceTable]) - val tableSource = tableSourceTable.tableSource.asInstanceOf[StreamTableSource[_]] + override def deriveRowType() = { + val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] + flinkTypeFactory.buildRowDataType(tableSource.getFieldsNames, tableSource.getFieldTypes) + } override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { new StreamTableSourceScan( cluster, traitSet, - table, - rowType + getTable, + tableSource ) } @@ -55,7 +56,7 @@ class StreamTableSourceScan( val inputDataStream: DataStream[Any] = tableSource .getDataStream(tableEnv.execEnv).asInstanceOf[DataStream[Any]] - convertToExpectedType(inputDataStream, tableSourceTable, expectedType, config) + convertToExpectedType(inputDataStream, new TableSourceTable(tableSource), expectedType, config) } } http://git-wip-us.apache.org/repos/asf/flink/blob/98d18260/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala ---------------------------------------------------------------------- diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala index 9d8075c..91dd255 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala @@ -58,11 +58,15 @@ class StreamTableSourceScanRule val scan: LogicalTableScan = rel.asInstanceOf[LogicalTableScan] val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE) + // The original registered table source + val table: TableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable]) + val tableSource: StreamTableSource[_] = table.tableSource.asInstanceOf[StreamTableSource[_]] + new StreamTableSourceScan( rel.getCluster, traitSet, scan.getTable, - rel.getRowType + tableSource ) } }