[HUDI-4178] Addressing performance regressions in Spark DataSourceV2 Integration (#5737)
There are multiple issues with our current DataSource V2 integrations: b/c we advertise Hudi tables as V2, Spark expects it to implement certain APIs which are not implemented at the moment, instead we're using custom Resolution rule (in HoodieSpark3Analysis) to instead manually fallback to V1 APIs. This commit fixes the issue by reverting DSv2 APIs and making Spark use V1, except for schema evaluation logic.
This commit is contained in:
@@ -19,7 +19,7 @@ package org.apache.hudi
|
||||
|
||||
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, QUERY_TYPE_SNAPSHOT_OPT_VAL}
|
||||
import org.apache.hudi.HoodieConversionUtils.toScalaOption
|
||||
import org.apache.hudi.common.config.{ConfigProperty, HoodieConfig}
|
||||
import org.apache.hudi.common.config.{ConfigProperty, HoodieCommonConfig, HoodieConfig}
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig
|
||||
import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType}
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
@@ -142,6 +142,9 @@ object DataSourceReadOptions {
|
||||
.key("hoodie.datasource.read.incr.fallback.fulltablescan.enable")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("When doing an incremental query whether we should fall back to full table scans if file does not exist.")
|
||||
|
||||
val SCHEMA_EVOLUTION_ENABLED: ConfigProperty[Boolean] = HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE
|
||||
|
||||
/** @deprecated Use {@link QUERY_TYPE} and its methods instead */
|
||||
@Deprecated
|
||||
val QUERY_TYPE_OPT_KEY = QUERY_TYPE.key()
|
||||
|
||||
@@ -25,11 +25,10 @@ import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.config.HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.internal.schema.InternalSchema
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql.execution.streaming.{Sink, Source}
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isUsingHiveCatalog
|
||||
import org.apache.spark.sql.hudi.streaming.HoodieStreamSource
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.streaming.OutputMode
|
||||
@@ -100,9 +99,18 @@ class DefaultSource extends RelationProvider
|
||||
val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent
|
||||
val tableType = metaClient.getTableType
|
||||
val queryType = parameters(QUERY_TYPE.key)
|
||||
val userSchema = if (schema == null) Option.empty[StructType] else Some(schema)
|
||||
// NOTE: In cases when Hive Metastore is used as catalog and the table is partitioned, schema in the HMS might contain
|
||||
// Hive-specific partitioning columns created specifically for HMS to handle partitioning appropriately. In that
|
||||
// case we opt in to not be providing catalog's schema, and instead force Hudi relations to fetch the schema
|
||||
// from the table itself
|
||||
val userSchema = if (isUsingHiveCatalog(sqlContext.sparkSession)) {
|
||||
None
|
||||
} else {
|
||||
Option(schema)
|
||||
}
|
||||
|
||||
log.info(s"Is bootstrapped table => $isBootstrappedTable, tableType is: $tableType, queryType is: $queryType")
|
||||
|
||||
if (metaClient.getCommitsTimeline.filterCompletedInstants.countInstants() == 0) {
|
||||
new EmptyRelation(sqlContext, metaClient)
|
||||
} else {
|
||||
|
||||
@@ -34,7 +34,7 @@ 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.internal.schema.InternalSchema
|
||||
import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema}
|
||||
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader
|
||||
import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex
|
||||
@@ -74,7 +74,7 @@ case class HoodieTableState(tablePath: String,
|
||||
abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
val metaClient: HoodieTableMetaClient,
|
||||
val optParams: Map[String, String],
|
||||
userSchema: Option[StructType])
|
||||
schemaSpec: Option[StructType])
|
||||
extends BaseRelation
|
||||
with FileRelation
|
||||
with PrunedFilteredScan
|
||||
@@ -128,24 +128,28 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
*/
|
||||
protected lazy val (tableAvroSchema: Schema, internalSchema: InternalSchema) = {
|
||||
val schemaResolver = new TableSchemaResolver(metaClient)
|
||||
val avroSchema = Try(schemaResolver.getTableAvroSchema) match {
|
||||
case Success(schema) => schema
|
||||
case Failure(e) =>
|
||||
logWarning("Failed to fetch schema from the table", e)
|
||||
// If there is no commit in the table, we can't get the schema
|
||||
// t/h [[TableSchemaResolver]], fallback to the provided [[userSchema]] instead.
|
||||
userSchema match {
|
||||
case Some(s) => convertToAvroSchema(s)
|
||||
case _ => throw new IllegalArgumentException("User-provided schema is required in case the table is empty")
|
||||
}
|
||||
val avroSchema: Schema = schemaSpec.map(convertToAvroSchema).getOrElse {
|
||||
Try(schemaResolver.getTableAvroSchema) match {
|
||||
case Success(schema) => schema
|
||||
case Failure(e) =>
|
||||
logError("Failed to fetch schema from the table", e)
|
||||
throw new HoodieSchemaException("Failed to fetch schema from the table")
|
||||
}
|
||||
}
|
||||
// try to find internalSchema
|
||||
val internalSchemaFromMeta = try {
|
||||
schemaResolver.getTableInternalSchemaFromCommitMetadata.orElse(InternalSchema.getEmptyInternalSchema)
|
||||
} catch {
|
||||
case _: Exception => InternalSchema.getEmptyInternalSchema
|
||||
|
||||
val internalSchema: InternalSchema = if (!isSchemaEvolutionEnabled) {
|
||||
InternalSchema.getEmptyInternalSchema
|
||||
} else {
|
||||
Try(schemaResolver.getTableInternalSchemaFromCommitMetadata) match {
|
||||
case Success(internalSchemaOpt) =>
|
||||
toScalaOption(internalSchemaOpt).getOrElse(InternalSchema.getEmptyInternalSchema)
|
||||
case Failure(e) =>
|
||||
logWarning("Failed to fetch internal-schema from the table", e)
|
||||
InternalSchema.getEmptyInternalSchema
|
||||
}
|
||||
}
|
||||
(avroSchema, internalSchemaFromMeta)
|
||||
|
||||
(avroSchema, internalSchema)
|
||||
}
|
||||
|
||||
protected lazy val tableStructSchema: StructType = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
|
||||
@@ -503,6 +507,15 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
|
||||
private def prunePartitionColumns(dataStructSchema: StructType): StructType =
|
||||
StructType(dataStructSchema.filterNot(f => partitionColumns.contains(f.name)))
|
||||
|
||||
private def isSchemaEvolutionEnabled = {
|
||||
// NOTE: Schema evolution could be configured both t/h optional parameters vehicle as well as
|
||||
// t/h Spark Session configuration (for ex, for Spark SQL)
|
||||
optParams.getOrElse(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key,
|
||||
DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean ||
|
||||
sparkSession.conf.get(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key,
|
||||
DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean
|
||||
}
|
||||
}
|
||||
|
||||
object HoodieBaseRelation extends SparkAdapterSupport {
|
||||
|
||||
@@ -26,7 +26,7 @@ import org.apache.hudi.HoodieConversionUtils.toProperties
|
||||
import org.apache.hudi.HoodieWriterUtils._
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient}
|
||||
import org.apache.hudi.common.config.{HoodieConfig, HoodieMetadataConfig, TypedProperties}
|
||||
import org.apache.hudi.common.config.{HoodieCommonConfig, HoodieConfig, HoodieMetadataConfig, TypedProperties}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model._
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
@@ -338,7 +338,7 @@ object HoodieSparkSqlWriter {
|
||||
def addSchemaEvolutionParameters(parameters: Map[String, String], internalSchemaOpt: Option[InternalSchema]): Map[String, String] = {
|
||||
val schemaEvolutionEnable = if (internalSchemaOpt.isDefined) "true" else "false"
|
||||
parameters ++ Map(HoodieWriteConfig.INTERNAL_SCHEMA_STRING.key() -> SerDeHelper.toJson(internalSchemaOpt.getOrElse(null)),
|
||||
HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key() -> schemaEvolutionEnable)
|
||||
HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key() -> schemaEvolutionEnable)
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -18,11 +18,10 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.hudi.DataSourceOptionsHelper.allAlternatives
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE
|
||||
import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieConfig, TypedProperties}
|
||||
import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieCommonConfig, HoodieConfig, TypedProperties}
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
@@ -163,9 +162,9 @@ object HoodieWriterUtils {
|
||||
// Check schema evolution for bootstrap table.
|
||||
// now we do not support bootstrap table.
|
||||
if (params.get(OPERATION.key).contains(BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
&& params.getOrElse(HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key(), "false").toBoolean) {
|
||||
&& params.getOrElse(HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key(), "false").toBoolean) {
|
||||
throw new HoodieException(String
|
||||
.format("now schema evolution cannot support bootstrap table, pls set %s to false", HoodieWriteConfig.SCHEMA_EVOLUTION_ENABLE.key()))
|
||||
.format("now schema evolution cannot support bootstrap table, pls set %s to false", HoodieCommonConfig.SCHEMA_EVOLUTION_ENABLE.key()))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -253,8 +253,11 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport {
|
||||
.filterKeys(_.startsWith("hoodie."))
|
||||
}
|
||||
|
||||
def isEnableHive(sparkSession: SparkSession): Boolean =
|
||||
"hive" == sparkSession.sessionState.conf.getConf(StaticSQLConf.CATALOG_IMPLEMENTATION)
|
||||
/**
|
||||
* Checks whether Spark is using Hive as Session's Catalog
|
||||
*/
|
||||
def isUsingHiveCatalog(sparkSession: SparkSession): Boolean =
|
||||
sparkSession.sessionState.conf.getConf(StaticSQLConf.CATALOG_IMPLEMENTATION) == "hive"
|
||||
|
||||
/**
|
||||
* Convert different query instant time format to the commit time format.
|
||||
|
||||
@@ -33,7 +33,7 @@ import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
|
||||
import org.apache.spark.sql.hive.HiveExternalCatalog
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{isEnableHive, withSparkConf}
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.{isUsingHiveCatalog, withSparkConf}
|
||||
import org.apache.spark.sql.hudi.command.{SqlKeyGenerator, ValidateDuplicateKeyPayload}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.StructType
|
||||
@@ -57,7 +57,7 @@ trait ProvidesHoodieConfig extends Logging {
|
||||
|
||||
require(hoodieCatalogTable.primaryKeys.nonEmpty,
|
||||
s"There are no primary key in table ${hoodieCatalogTable.table.identifier}, cannot execute update operator")
|
||||
val enableHive = isEnableHive(sparkSession)
|
||||
val enableHive = isUsingHiveCatalog(sparkSession)
|
||||
|
||||
val hoodieProps = getHoodieProps(catalogProperties, tableConfig, sparkSession.sqlContext.conf)
|
||||
|
||||
@@ -174,7 +174,7 @@ trait ProvidesHoodieConfig extends Logging {
|
||||
|
||||
logInfo(s"Insert statement use write operation type: $operation, payloadClass: $payloadClassName")
|
||||
|
||||
val enableHive = isEnableHive(sparkSession)
|
||||
val enableHive = isUsingHiveCatalog(sparkSession)
|
||||
|
||||
withSparkConf(sparkSession, catalogProperties) {
|
||||
Map(
|
||||
@@ -213,7 +213,7 @@ trait ProvidesHoodieConfig extends Logging {
|
||||
hoodieCatalogTable: HoodieCatalogTable,
|
||||
partitionsToDrop: String): Map[String, String] = {
|
||||
val partitionFields = hoodieCatalogTable.partitionFields.mkString(",")
|
||||
val enableHive = isEnableHive(sparkSession)
|
||||
val enableHive = isUsingHiveCatalog(sparkSession)
|
||||
val catalogProperties = hoodieCatalogTable.catalogProperties
|
||||
val tableConfig = hoodieCatalogTable.tableConfig
|
||||
|
||||
@@ -259,7 +259,7 @@ trait ProvidesHoodieConfig extends Logging {
|
||||
val hiveSyncConfig = buildHiveSyncConfig(hoodieProps, hoodieCatalogTable)
|
||||
|
||||
val options = hoodieCatalogTable.catalogProperties
|
||||
val enableHive = isEnableHive(sparkSession)
|
||||
val enableHive = isUsingHiveCatalog(sparkSession)
|
||||
|
||||
withSparkConf(sparkSession, options) {
|
||||
Map(
|
||||
|
||||
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.catalog._
|
||||
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps
|
||||
import org.apache.spark.sql.hive.HiveClientUtils
|
||||
import org.apache.spark.sql.hive.HiveExternalCatalog._
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isEnableHive
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isUsingHiveCatalog
|
||||
import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlCommonUtils}
|
||||
import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
|
||||
import org.apache.spark.sql.types.StructType
|
||||
@@ -144,7 +144,7 @@ object CreateHoodieTableCommand {
|
||||
)
|
||||
|
||||
// Create table in the catalog
|
||||
val enableHive = isEnableHive(sparkSession)
|
||||
val enableHive = isUsingHiveCatalog(sparkSession)
|
||||
if (enableHive) {
|
||||
createHiveDataSourceTable(sparkSession, newTable, ignoreIfExists)
|
||||
} else {
|
||||
|
||||
Reference in New Issue
Block a user