1
0

[HUDI-1526] Translate the api partitionBy in spark datasource to hoodie.datasource.write.partitionpath.field (#2431)

This commit is contained in:
teeyog
2021-02-11 01:07:54 +08:00
committed by GitHub
parent a2f85d90de
commit 26da4f5462
3 changed files with 208 additions and 9 deletions

View File

@@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.streaming.Sink
import org.apache.spark.sql.sources._
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession}
import scala.collection.JavaConverters._
@@ -46,6 +46,14 @@ class DefaultSource extends RelationProvider
with StreamSinkProvider
with Serializable {
SparkSession.getActiveSession.foreach { spark =>
val sparkVersion = spark.version
if (sparkVersion.startsWith("0.") || sparkVersion.startsWith("1.") || sparkVersion.startsWith("2.")) {
// Enable "passPartitionByAsOptions" to support "write.partitionBy(...)"
spark.conf.set("spark.sql.legacy.sources.write.passPartitionByAsOptions", "true")
}
}
private val log = LogManager.getLogger(classOf[DefaultSource])
override def createRelation(sqlContext: SQLContext,
@@ -126,12 +134,13 @@ class DefaultSource extends RelationProvider
optParams: Map[String, String],
df: DataFrame): BaseRelation = {
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters)
val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*)
if (parameters(OPERATION_OPT_KEY).equals(BOOTSTRAP_OPERATION_OPT_VAL)) {
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, parameters, dfWithoutMetaCols)
if (translatedOptions(OPERATION_OPT_KEY).equals(BOOTSTRAP_OPERATION_OPT_VAL)) {
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, translatedOptions, dfWithoutMetaCols)
} else {
HoodieSparkSqlWriter.write(sqlContext, mode, parameters, dfWithoutMetaCols)
HoodieSparkSqlWriter.write(sqlContext, mode, translatedOptions, dfWithoutMetaCols)
}
new HoodieEmptyRelation(sqlContext, dfWithoutMetaCols.schema)
}
@@ -141,9 +150,10 @@ class DefaultSource extends RelationProvider
partitionColumns: Seq[String],
outputMode: OutputMode): Sink = {
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters)
new HoodieStreamingSink(
sqlContext,
parameters,
translatedOptions,
partitionColumns,
outputMode)
}