-
Notifications
You must be signed in to change notification settings - Fork 13k
/
PhysicalTableSourceScan.scala
79 lines (68 loc) · 3.17 KB
/
PhysicalTableSourceScan.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
/*
* 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.nodes.physical
import org.apache.flink.api.common.io.InputFormat
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.dag.Transformation
import org.apache.flink.core.io.InputSplit
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
import org.apache.flink.table.calcite.FlinkTypeFactory
import org.apache.flink.table.plan.schema.{FlinkRelOptTable, TableSourceTable}
import org.apache.flink.table.sources.{InputFormatTableSource, StreamTableSource, TableSource}
import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo
import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
import org.apache.calcite.rel.RelWriter
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.core.TableScan
import scala.collection.JavaConverters._
/**
* Base physical RelNode to read data from an external source defined by a [[TableSource]].
*/
abstract class PhysicalTableSourceScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
relOptTable: FlinkRelOptTable)
extends TableScan(cluster, traitSet, relOptTable) {
// cache table source transformation.
protected var sourceTransform: Transformation[_] = _
protected val tableSourceTable: TableSourceTable[_] =
relOptTable.unwrap(classOf[TableSourceTable[_]])
protected[flink] val tableSource: TableSource[_] = tableSourceTable.tableSource
override def deriveRowType(): RelDataType = {
val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
tableSourceTable.getRowType(flinkTypeFactory)
}
override def explainTerms(pw: RelWriter): RelWriter = {
super.explainTerms(pw).item("fields", getRowType.getFieldNames.asScala.mkString(", "))
}
def getSourceTransformation(
streamEnv: StreamExecutionEnvironment): Transformation[_] = {
if (sourceTransform == null) {
sourceTransform = tableSource match {
case source: InputFormatTableSource[_] =>
val resultType = fromDataTypeToLegacyInfo(source.getProducedDataType)
.asInstanceOf[TypeInformation[Any]]
streamEnv.createInput(
source.getInputFormat.asInstanceOf[InputFormat[Any, _ <: InputSplit]],
resultType).getTransformation
case source: StreamTableSource[_] => source.getDataStream(streamEnv).getTransformation
}
}
sourceTransform
}
}