1
0

[HUDI-1376] Drop Hudi metadata cols at the beginning of Spark datasource writing (#2233)

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2020-12-15 16:20:48 -08:00
committed by GitHub
parent 93d9c25aee
commit 26cdc457f6
3 changed files with 46 additions and 10 deletions

View File

@@ -19,7 +19,7 @@ package org.apache.hudi
import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceReadOptions._
import org.apache.hudi.common.model.HoodieTableType
import org.apache.hudi.common.model.{HoodieRecord, HoodieTableType}
import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION_OPT_KEY}
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.table.HoodieTableMetaClient
@@ -33,6 +33,8 @@ import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
import scala.collection.JavaConverters._
/**
* Hoodie Spark Datasource, for reading and writing hoodie tables
*
@@ -119,12 +121,14 @@ class DefaultSource extends RelationProvider
optParams: Map[String, String],
df: DataFrame): BaseRelation = {
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*)
if (parameters(OPERATION_OPT_KEY).equals(BOOTSTRAP_OPERATION_OPT_VAL)) {
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, parameters, df)
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, parameters, dfWithoutMetaCols)
} else {
HoodieSparkSqlWriter.write(sqlContext, mode, parameters, df)
HoodieSparkSqlWriter.write(sqlContext, mode, parameters, dfWithoutMetaCols)
}
new HoodieEmptyRelation(sqlContext, df.schema)
new HoodieEmptyRelation(sqlContext, dfWithoutMetaCols.schema)
}
override def createSink(sqlContext: SQLContext,