flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [flink] KurtYoung commented on a change in pull request #9909: [FLINK-14381][table] Partition field names should be got from CatalogTable instead of source/sink
Date Fri, 25 Oct 2019 06:52:00 GMT
KurtYoung commented on a change in pull request #9909: [FLINK-14381][table] Partition field
names should be got from CatalogTable instead of source/sink
URL: https://github.com/apache/flink/pull/9909#discussion_r338906446
 
 

 ##########
 File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala
 ##########
 @@ -44,32 +42,36 @@ class BatchExecSinkRule extends ConverterRule(
     val sinkNode = rel.asInstanceOf[FlinkLogicalSink]
     val newTrait = rel.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL)
     var requiredTraitSet = sinkNode.getInput.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL)
-    sinkNode.sink match {
-      case partitionSink: PartitionableTableSink
-        if partitionSink.getPartitionFieldNames != null &&
-          partitionSink.getPartitionFieldNames.nonEmpty =>
-        val partitionFields = partitionSink.getPartitionFieldNames
-        val partitionIndices = partitionFields
-          .map(partitionSink.getTableSchema.getFieldNames.indexOf(_))
-        // validate
-        partitionIndices.foreach { idx =>
-          if (idx < 0) {
-            throw new TableException(s"Partitionable sink ${sinkNode.sinkName} field " +
-              s"${partitionFields.get(idx)} must be in the schema.")
-          }
-        }
+    if (sinkNode.catalogTable != null && sinkNode.catalogTable.isPartitioned) {
+      sinkNode.sink match {
+        case partitionSink: PartitionableTableSink =>
+          val partKeys = sinkNode.catalogTable.getPartitionKeys
+          if (!partKeys.isEmpty) {
+            val partitionIndices =
+              partKeys.map(partitionSink.getTableSchema.getFieldNames.indexOf(_))
+            // validate
+            partitionIndices.foreach { idx =>
+              if (idx < 0) {
+                throw new TableException(s"Partitionable sink ${sinkNode.sinkName} field
" +
+                    s"${partKeys.get(idx)} must be in the schema.")
+              }
+            }
 
-        requiredTraitSet = requiredTraitSet.plus(
-          FlinkRelDistribution.hash(partitionIndices
-            .map(Integer.valueOf), requireStrict = false))
+            requiredTraitSet = requiredTraitSet.plus(
+              FlinkRelDistribution.hash(partitionIndices
+                  .map(Integer.valueOf), requireStrict = false))
 
-        if (partitionSink.configurePartitionGrouping(true)) {
-          // default to asc.
-          val fieldCollations = partitionIndices.map(FlinkRelOptUtil.ofRelFieldCollation)
-          requiredTraitSet = requiredTraitSet.plus(RelCollations.of(fieldCollations: _*))
-        }
-      case _ =>
+            if (partitionSink.configurePartitionGrouping(true)) {
+              // default to asc.
+              val fieldCollations = partitionIndices.map(FlinkRelOptUtil.ofRelFieldCollation)
+              requiredTraitSet = requiredTraitSet.plus(RelCollations.of(fieldCollations:
_*))
+            }
+          }
+        case _ => throw new TableException(
+          s"Table(${sinkNode.sinkName}) with partition keys should be a PartitionableTableSink.")
 
 Review comment:
   We need PartitionableTableSink to write data to partitioned table: $tableName

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message