@@ -40,7 +40,7 @@ import org.apache.flink.table.planner.plan.nodes.FlinkConventions
40
40
import org .apache .flink .table .planner .plan .nodes .logical ._
41
41
import org .apache .flink .table .planner .plan .nodes .physical .batch ._
42
42
import org .apache .flink .table .planner .plan .nodes .physical .stream ._
43
- import org .apache .flink .table .planner .plan .schema .{FlinkPreparingTableBase , IntermediateRelTable }
43
+ import org .apache .flink .table .planner .plan .schema .{FlinkPreparingTableBase , IntermediateRelTable , TableSourceTable }
44
44
import org .apache .flink .table .planner .plan .stream .sql .join .TestTemporalTable
45
45
import org .apache .flink .table .planner .plan .utils ._
46
46
import org .apache .flink .table .planner .utils .Top3
@@ -171,6 +171,37 @@ class FlinkRelMdHandlerTestBase {
171
171
protected lazy val empStreamScan : StreamPhysicalDataStreamScan =
172
172
createDataStreamScan(ImmutableList .of(" emp" ), streamPhysicalTraits)
173
173
174
+ protected lazy val tableSourceTableLogicalScan : LogicalTableScan =
175
+ createTableSourceTable(ImmutableList .of(" TableSourceTable1" ), logicalTraits)
176
+ protected lazy val tableSourceTableFlinkLogicalScan : FlinkLogicalDataStreamTableScan =
177
+ createTableSourceTable(ImmutableList .of(" TableSourceTable1" ), flinkLogicalTraits)
178
+ protected lazy val tableSourceTableBatchScan : BatchPhysicalBoundedStreamScan =
179
+ createTableSourceTable(ImmutableList .of(" TableSourceTable1" ), batchPhysicalTraits)
180
+ protected lazy val tableSourceTableStreamScan : StreamPhysicalDataStreamScan =
181
+ createTableSourceTable(ImmutableList .of(" TableSourceTable1" ), streamPhysicalTraits)
182
+
183
+ protected lazy val tablePartiallyProjectedKeyLogicalScan : LogicalTableScan =
184
+ createTableSourceTable(ImmutableList .of(" projected_table_source_table_with_partial_pk" ),
185
+ logicalTraits)
186
+ protected lazy val tablePartiallyProjectedKeyFlinkLogicalScan : FlinkLogicalDataStreamTableScan =
187
+ createTableSourceTable(ImmutableList .of(" projected_table_source_table_with_partial_pk" ),
188
+ flinkLogicalTraits)
189
+ protected lazy val tablePartiallyProjectedKeyBatchScan : BatchPhysicalBoundedStreamScan =
190
+ createTableSourceTable(ImmutableList .of(" projected_table_source_table_with_partial_pk" ),
191
+ batchPhysicalTraits)
192
+ protected lazy val tablePartiallyProjectedKeyStreamScan : StreamPhysicalDataStreamScan =
193
+ createTableSourceTable(ImmutableList .of(" projected_table_source_table_with_partial_pk" ),
194
+ streamPhysicalTraits)
195
+
196
+ protected lazy val tableSourceTableNonKeyLogicalScan : LogicalTableScan =
197
+ createTableSourceTable(ImmutableList .of(" TableSourceTable3" ), logicalTraits)
198
+ protected lazy val tableSourceTableNonKeyFlinkLogicalScan : FlinkLogicalDataStreamTableScan =
199
+ createTableSourceTable(ImmutableList .of(" TableSourceTable3" ), flinkLogicalTraits)
200
+ protected lazy val tableSourceTableNonKeyBatchScan : BatchPhysicalBoundedStreamScan =
201
+ createTableSourceTable(ImmutableList .of(" TableSourceTable3" ), batchPhysicalTraits)
202
+ protected lazy val tableSourceTableNonKeyStreamScan : StreamPhysicalDataStreamScan =
203
+ createTableSourceTable(ImmutableList .of(" TableSourceTable3" ), streamPhysicalTraits)
204
+
174
205
private lazy val valuesType = relBuilder.getTypeFactory
175
206
.builder()
176
207
.add(" a" , SqlTypeName .BIGINT )
@@ -2727,6 +2758,51 @@ class FlinkRelMdHandlerTestBase {
2727
2758
}
2728
2759
}
2729
2760
2761
+ // select * from TableSourceTable1
2762
+ // left join TableSourceTable2 on TableSourceTable1.b = TableSourceTable2.b
2763
+ protected lazy val logicalLeftJoinOnContainedUniqueKeys : RelNode = relBuilder
2764
+ .scan(" TableSourceTable1" )
2765
+ .scan(" TableSourceTable2" )
2766
+ .join(
2767
+ JoinRelType .LEFT ,
2768
+ relBuilder.call(
2769
+ EQUALS ,
2770
+ relBuilder.field(2 , 0 , 1 ),
2771
+ relBuilder.field(2 , 1 , 1 )
2772
+ )
2773
+ )
2774
+ .build
2775
+
2776
+ // select * from TableSourceTable1
2777
+ // left join TableSourceTable2 on TableSourceTable1.a = TableSourceTable2.a
2778
+ protected lazy val logicalLeftJoinOnDisjointUniqueKeys : RelNode = relBuilder
2779
+ .scan(" TableSourceTable1" )
2780
+ .scan(" TableSourceTable2" )
2781
+ .join(
2782
+ JoinRelType .LEFT ,
2783
+ relBuilder.call(
2784
+ EQUALS ,
2785
+ relBuilder.field(2 , 0 , 0 ),
2786
+ relBuilder.field(2 , 1 , 0 )
2787
+ )
2788
+ )
2789
+ .build
2790
+
2791
+ // select * from TableSourceTable1
2792
+ // left join TableSourceTable3 on TableSourceTable1.a = TableSourceTable3.a
2793
+ protected lazy val logicalLeftJoinWithNoneKeyTableUniqueKeys : RelNode = relBuilder
2794
+ .scan(" TableSourceTable1" )
2795
+ .scan(" TableSourceTable3" )
2796
+ .join(
2797
+ JoinRelType .LEFT ,
2798
+ relBuilder.call(
2799
+ EQUALS ,
2800
+ relBuilder.field(2 , 0 , 0 ),
2801
+ relBuilder.field(2 , 1 , 0 )
2802
+ )
2803
+ )
2804
+ .build
2805
+
2730
2806
protected def createDataStreamScan [T ](
2731
2807
tableNames : util.List [String ], traitSet : RelTraitSet ): T = {
2732
2808
val table = relBuilder
@@ -2754,6 +2830,33 @@ class FlinkRelMdHandlerTestBase {
2754
2830
scan.asInstanceOf [T ]
2755
2831
}
2756
2832
2833
+ protected def createTableSourceTable [T ](
2834
+ tableNames : util.List [String ], traitSet : RelTraitSet ): T = {
2835
+ val table = relBuilder
2836
+ .getRelOptSchema
2837
+ .asInstanceOf [CalciteCatalogReader ]
2838
+ .getTable(tableNames)
2839
+ .asInstanceOf [TableSourceTable ]
2840
+ val conventionTrait = traitSet.getTrait(ConventionTraitDef .INSTANCE )
2841
+ val scan = conventionTrait match {
2842
+ case Convention .NONE =>
2843
+ relBuilder.clear()
2844
+ val scan = relBuilder.scan(tableNames).build()
2845
+ scan.copy(traitSet, scan.getInputs)
2846
+ case FlinkConventions .LOGICAL =>
2847
+ new FlinkLogicalDataStreamTableScan (
2848
+ cluster, traitSet, Collections .emptyList[RelHint ](), table)
2849
+ case FlinkConventions .BATCH_PHYSICAL =>
2850
+ new BatchPhysicalBoundedStreamScan (
2851
+ cluster, traitSet, Collections .emptyList[RelHint ](), table, table.getRowType)
2852
+ case FlinkConventions .STREAM_PHYSICAL =>
2853
+ new StreamPhysicalDataStreamScan (
2854
+ cluster, traitSet, Collections .emptyList[RelHint ](), table, table.getRowType)
2855
+ case _ => throw new TableException (s " Unsupported convention trait: $conventionTrait" )
2856
+ }
2857
+ scan.asInstanceOf [T ]
2858
+ }
2859
+
2757
2860
protected def createLiteralList (
2758
2861
rowType : RelDataType ,
2759
2862
literalValues : Seq [String ]): util.List [RexLiteral ] = {
0 commit comments