[HUDI-3748] write and select hudi table when enable hoodie.datasource.write.drop.partition.columns (#5201)
This commit is contained in:
@@ -20,10 +20,11 @@ package org.apache.hudi
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
|
||||
import org.apache.hudi.HoodieBaseRelation.createBaseFileReader
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
|
||||
import org.apache.spark.sql.SQLContext
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.execution.datasources._
|
||||
import org.apache.spark.sql.sources.{BaseRelation, Filter}
|
||||
@@ -91,7 +92,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
|
||||
sparkSession = sparkSession,
|
||||
file = file,
|
||||
// TODO clarify why this is required
|
||||
partitionValues = InternalRow.empty
|
||||
partitionValues = getPartitionColumnsAsInternalRow(file)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -471,11 +471,10 @@ object DataSourceWriteOptions {
|
||||
.sinceVersion("0.9.0")
|
||||
.withDocumentation("This class is used by kafka client to deserialize the records")
|
||||
|
||||
val DROP_PARTITION_COLUMNS: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.write.drop.partition.columns")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("When set to true, will not write the partition columns into hudi. " +
|
||||
"By default, false.")
|
||||
val DROP_PARTITION_COLUMNS: ConfigProperty[Boolean] = ConfigProperty
|
||||
.key(HoodieTableConfig.DROP_PARTITION_COLUMNS.key())
|
||||
.defaultValue(HoodieTableConfig.DROP_PARTITION_COLUMNS.defaultValue().booleanValue())
|
||||
.withDocumentation(HoodieTableConfig.DROP_PARTITION_COLUMNS.doc())
|
||||
|
||||
/** @deprecated Use {@link HIVE_ASSUME_DATE_PARTITION} and its methods instead */
|
||||
@Deprecated
|
||||
|
||||
@@ -19,10 +19,12 @@ package org.apache.hudi
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig
|
||||
import org.apache.hadoop.mapred.JobConf
|
||||
|
||||
import org.apache.hudi.HoodieBaseRelation.getPartitionPath
|
||||
import org.apache.hudi.HoodieConversionUtils.toScalaOption
|
||||
import org.apache.hudi.common.config.SerializableConfiguration
|
||||
@@ -33,27 +35,29 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.StringUtils
|
||||
import org.apache.hudi.common.util.ValidationUtils.checkState
|
||||
import org.apache.hudi.hadoop.HoodieROTablePathFilter
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata
|
||||
|
||||
import org.apache.spark.TaskContext
|
||||
import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression}
|
||||
import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile}
|
||||
import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile, PartitioningUtils}
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
|
||||
import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.sql.{Row, SQLContext, SparkSession}
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import java.io.Closeable
|
||||
import java.net.URI
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.Try
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
trait HoodieFileSplit {}
|
||||
|
||||
@@ -140,6 +144,12 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
|
||||
protected val partitionColumns: Array[String] = tableConfig.getPartitionFields.orElse(Array.empty)
|
||||
|
||||
/**
|
||||
* if true, need to deal with schema for creating file reader.
|
||||
*/
|
||||
protected val dropPartitionColumnsWhenWrite: Boolean =
|
||||
metaClient.getTableConfig.isDropPartitionColumns && partitionColumns.nonEmpty
|
||||
|
||||
/**
|
||||
* NOTE: PLEASE READ THIS CAREFULLY
|
||||
*
|
||||
@@ -209,14 +219,37 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
|
||||
val fileSplits = collectFileSplits(partitionFilters, dataFilters)
|
||||
|
||||
val partitionSchema = StructType(Nil)
|
||||
val tableSchema = HoodieTableSchema(tableStructSchema, if (internalSchema.isEmptySchema) tableAvroSchema.toString else AvroInternalSchemaConverter.convert(internalSchema, tableAvroSchema.getName).toString, internalSchema)
|
||||
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString, requiredInternalSchema)
|
||||
val partitionSchema = if (dropPartitionColumnsWhenWrite) {
|
||||
// when hoodie.datasource.write.drop.partition.columns is true, partition columns can't be persisted in
|
||||
// data files.
|
||||
StructType(partitionColumns.map(StructField(_, StringType)))
|
||||
} else {
|
||||
StructType(Nil)
|
||||
}
|
||||
|
||||
val tableSchema = HoodieTableSchema(tableStructSchema, if (internalSchema.isEmptySchema) tableAvroSchema.toString else AvroInternalSchemaConverter.convert(internalSchema, tableAvroSchema.getName).toString, internalSchema)
|
||||
val dataSchema = if (dropPartitionColumnsWhenWrite) {
|
||||
val dataStructType = StructType(tableStructSchema.filterNot(f => partitionColumns.contains(f.name)))
|
||||
HoodieTableSchema(
|
||||
dataStructType,
|
||||
sparkAdapter.getAvroSchemaConverters.toAvroType(dataStructType, nullable = false, "record").toString()
|
||||
)
|
||||
} else {
|
||||
tableSchema
|
||||
}
|
||||
val requiredSchema = if (dropPartitionColumnsWhenWrite) {
|
||||
val requiredStructType = StructType(requiredStructSchema.filterNot(f => partitionColumns.contains(f.name)))
|
||||
HoodieTableSchema(
|
||||
requiredStructType,
|
||||
sparkAdapter.getAvroSchemaConverters.toAvroType(requiredStructType, nullable = false, "record").toString()
|
||||
)
|
||||
} else {
|
||||
HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString, requiredInternalSchema)
|
||||
}
|
||||
// Here we rely on a type erasure, to workaround inherited API restriction and pass [[RDD[InternalRow]]] back as [[RDD[Row]]]
|
||||
// Please check [[needConversion]] scala-doc for more details
|
||||
if (fileSplits.nonEmpty)
|
||||
composeRDD(fileSplits, partitionSchema, tableSchema, requiredSchema, filters).asInstanceOf[RDD[Row]]
|
||||
composeRDD(fileSplits, partitionSchema, dataSchema, requiredSchema, filters).asInstanceOf[RDD[Row]]
|
||||
else
|
||||
sparkSession.sparkContext.emptyRDD
|
||||
}
|
||||
@@ -286,8 +319,16 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
}
|
||||
|
||||
protected final def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = {
|
||||
val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col))
|
||||
requestedColumns ++ missing
|
||||
if (dropPartitionColumnsWhenWrite) {
|
||||
if (requestedColumns.isEmpty) {
|
||||
mandatoryColumns.toArray
|
||||
} else {
|
||||
requestedColumns
|
||||
}
|
||||
} else {
|
||||
val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col))
|
||||
requestedColumns ++ missing
|
||||
}
|
||||
}
|
||||
|
||||
protected def getTableState: HoodieTableState = {
|
||||
@@ -308,6 +349,38 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
// TODO(HUDI-3639) vectorized reader has to be disabled to make sure MORIncrementalRelation is working properly
|
||||
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "false")
|
||||
}
|
||||
|
||||
/**
|
||||
* For enable hoodie.datasource.write.drop.partition.columns, need to create an InternalRow on partition values
|
||||
* and pass this reader on parquet file. So that, we can query the partition columns.
|
||||
*/
|
||||
protected def getPartitionColumnsAsInternalRow(file: FileStatus): InternalRow = {
|
||||
try {
|
||||
val tableConfig = metaClient.getTableConfig
|
||||
if (dropPartitionColumnsWhenWrite) {
|
||||
val relativePath = new URI(metaClient.getBasePath).relativize(new URI(file.getPath.getParent.toString)).toString
|
||||
val hiveStylePartitioningEnabled = tableConfig.getHiveStylePartitioningEnable.toBoolean
|
||||
if (hiveStylePartitioningEnabled) {
|
||||
val partitionSpec = PartitioningUtils.parsePathFragment(relativePath)
|
||||
InternalRow.fromSeq(partitionColumns.map(partitionSpec(_)).map(UTF8String.fromString))
|
||||
} else {
|
||||
if (partitionColumns.length == 1) {
|
||||
InternalRow.fromSeq(Seq(UTF8String.fromString(relativePath)))
|
||||
} else {
|
||||
val parts = relativePath.split("/")
|
||||
assert(parts.size == partitionColumns.length)
|
||||
InternalRow.fromSeq(parts.map(UTF8String.fromString))
|
||||
}
|
||||
}
|
||||
} else {
|
||||
InternalRow.empty
|
||||
}
|
||||
} catch {
|
||||
case NonFatal(e) =>
|
||||
logWarning(s"Failed to get the right partition InternalRow for file : ${file.toString}")
|
||||
InternalRow.empty
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object HoodieBaseRelation {
|
||||
|
||||
@@ -160,6 +160,7 @@ object HoodieSparkSqlWriter {
|
||||
.setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HIVE_STYLE_PARTITIONING))
|
||||
.setUrlEncodePartitioning(hoodieConfig.getBoolean(URL_ENCODE_PARTITIONING))
|
||||
.setPartitionMetafileUseBaseFormat(useBaseFormatMetaFile)
|
||||
.setDropPartitionColumnsWhenWrite(hoodieConfig.getBooleanOrDefault(HoodieTableConfig.DROP_PARTITION_COLUMNS))
|
||||
.setCommitTimezone(HoodieTimelineTimeZone.valueOf(hoodieConfig.getStringOrDefault(HoodieTableConfig.TIMELINE_TIMEZONE)))
|
||||
.initTable(sparkContext.hadoopConfiguration, path)
|
||||
tableConfig = tableMetaClient.getTableConfig
|
||||
@@ -501,8 +502,8 @@ object HoodieSparkSqlWriter {
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val populateMetaFields = java.lang.Boolean.parseBoolean((parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(),
|
||||
String.valueOf(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()))))
|
||||
val dropPartitionColumns =
|
||||
parameters.getOrElse(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key(), DataSourceWriteOptions.DROP_PARTITION_COLUMNS.defaultValue()).toBoolean
|
||||
val dropPartitionColumns = parameters.get(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key()).map(_.toBoolean)
|
||||
.getOrElse(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.defaultValue())
|
||||
// register classes & schemas
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName)
|
||||
sparkContext.getConf.registerKryoClasses(
|
||||
|
||||
@@ -127,7 +127,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
|
||||
|
||||
val partitionedBaseFile = baseFile.map { file =>
|
||||
val filePath = getFilePath(file.getFileStatus.getPath)
|
||||
PartitionedFile(InternalRow.empty, filePath, 0, file.getFileLen)
|
||||
PartitionedFile(getPartitionColumnsAsInternalRow(file.getFileStatus), filePath, 0, file.getFileLen)
|
||||
}
|
||||
|
||||
HoodieMergeOnReadFileSplit(partitionedBaseFile, logFiles)
|
||||
|
||||
@@ -749,8 +749,17 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
@ParameterizedTest @ValueSource(booleans = Array(true, false))
|
||||
def testCopyOnWriteWithDropPartitionColumns(enableDropPartitionColumns: Boolean) {
|
||||
val resultContainPartitionColumn = copyOnWriteTableSelect(enableDropPartitionColumns)
|
||||
assertEquals(enableDropPartitionColumns, !resultContainPartitionColumn)
|
||||
val records1 = recordsToStrings(dataGen.generateInsertsContainsAllPartitions("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key, enableDropPartitionColumns)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi").load(basePath)
|
||||
assertEquals(snapshotDF1.count(), 100)
|
||||
assertEquals(3, snapshotDF1.select("partition").distinct().count())
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -863,22 +872,6 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
assertEquals(500, hoodieIncViewDF.count())
|
||||
}
|
||||
|
||||
def copyOnWriteTableSelect(enableDropPartitionColumns: Boolean): Boolean = {
|
||||
val records1 = recordsToStrings(dataGen.generateInsertsContainsAllPartitions("000", 3)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key, enableDropPartitionColumns)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.load(basePath + "/*/*/*/*")
|
||||
snapshotDF1.registerTempTable("tmptable")
|
||||
val result = spark.sql("select * from tmptable limit 1").collect()(0)
|
||||
result.schema.contains(new StructField("partition", StringType, true))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testWriteSmallPrecisionDecimalTable(): Unit = {
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList
|
||||
|
||||
@@ -630,4 +630,37 @@ class TestInsertTable extends TestHoodieSqlBase {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
test("Test enable hoodie.datasource.write.drop.partition.columns when write") {
|
||||
spark.sql("set hoodie.sql.bulk.insert.enable = false")
|
||||
Seq("mor", "cow").foreach { tableType =>
|
||||
withTempDir { tmp =>
|
||||
val tableName = generateTableName
|
||||
spark.sql(
|
||||
s"""
|
||||
| create table $tableName (
|
||||
| id int,
|
||||
| name string,
|
||||
| price double,
|
||||
| ts long,
|
||||
| dt string
|
||||
| ) using hudi
|
||||
| partitioned by (dt)
|
||||
| location '${tmp.getCanonicalPath}/$tableName'
|
||||
| tblproperties (
|
||||
| primaryKey = 'id',
|
||||
| preCombineField = 'ts',
|
||||
| type = '$tableType',
|
||||
| hoodie.datasource.write.drop.partition.columns = 'true'
|
||||
| )
|
||||
""".stripMargin)
|
||||
spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (1, 'a1', 10, 1000)")
|
||||
spark.sql(s"insert into $tableName partition(dt='2021-12-25') values (2, 'a2', 20, 1000)")
|
||||
checkAnswer(s"select id, name, price, ts, dt from $tableName")(
|
||||
Seq(1, "a1", 10, 1000, "2021-12-25"),
|
||||
Seq(2, "a2", 20, 1000, "2021-12-25")
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user