[HUDI-2053] Insert Static Partition With DateType Return Incorrect Partition Value (#3133)
This commit is contained in:
@@ -20,7 +20,6 @@ package org.apache.spark.sql.hudi
|
||||
import scala.collection.JavaConverters._
|
||||
import java.net.URI
|
||||
import java.util.Locale
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.SparkAdapterSupport
|
||||
@@ -30,7 +29,7 @@ import org.apache.spark.sql.{Column, DataFrame, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
|
||||
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
|
||||
import org.apache.spark.sql.catalyst.expressions.{And, Cast, Expression, Literal}
|
||||
import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Cast, Expression, Literal}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
|
||||
import org.apache.spark.sql.execution.datasources.LogicalRelation
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
@@ -106,6 +105,10 @@ object HoodieSqlUtils extends SparkAdapterSupport {
|
||||
}
|
||||
}
|
||||
|
||||
def removeMetaFields(attrs: Seq[Attribute]): Seq[Attribute] = {
|
||||
attrs.filterNot(attr => isMetaField(attr.name))
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the table location.
|
||||
* @param tableId
|
||||
|
||||
@@ -128,6 +128,11 @@ object InsertIntoHoodieTableCommand {
|
||||
s"Required partition columns is: ${targetPartitionSchema.json}, Current static partitions " +
|
||||
s"is: ${staticPartitionValues.mkString("," + "")}")
|
||||
|
||||
assert(staticPartitionValues.size + query.output.size == table.schema.size,
|
||||
s"Required select columns count: ${removeMetaFields(table.schema).size}, " +
|
||||
s"Current select columns(including static partition column) count: " +
|
||||
s"${staticPartitionValues.size + removeMetaFields(query.output).size},columns: " +
|
||||
s"(${(removeMetaFields(query.output).map(_.name) ++ staticPartitionValues.keys).mkString(",")})")
|
||||
val queryDataFields = if (staticPartitionValues.isEmpty) { // insert dynamic partition
|
||||
query.output.dropRight(targetPartitionSchema.fields.length)
|
||||
} else { // insert static partition
|
||||
@@ -156,7 +161,7 @@ object InsertIntoHoodieTableCommand {
|
||||
targetPartitionSchema.fields.map(f => {
|
||||
val staticPartitionValue = staticPartitionValues.getOrElse(f.name,
|
||||
s"Missing static partition value for: ${f.name}")
|
||||
val castAttr = Literal.create(staticPartitionValue, f.dataType)
|
||||
val castAttr = castIfNeeded(Literal.create(staticPartitionValue), f.dataType, conf)
|
||||
Alias(castAttr, f.name)()
|
||||
})
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user