[HUDI-3729][SPARK] fixed the per regression by enable vectorizeReader for parquet file (#5168)
* [MINOR][SPARK] fixed the per regression by enable vectorizeReader for parquet file * address comments * add perf result
This commit is contained in:
@@ -55,6 +55,11 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
|
||||
override lazy val mandatoryColumns: Seq[String] =
|
||||
Seq(recordKeyField)
|
||||
|
||||
override def imbueConfigs(sqlContext: SQLContext): Unit = {
|
||||
super.imbueConfigs(sqlContext)
|
||||
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "true")
|
||||
}
|
||||
|
||||
protected override def composeRDD(fileSplits: Seq[HoodieBaseFileSplit],
|
||||
partitionSchema: StructType,
|
||||
tableSchema: HoodieTableSchema,
|
||||
|
||||
@@ -290,7 +290,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
|
||||
)
|
||||
}
|
||||
|
||||
private def imbueConfigs(sqlContext: SQLContext): Unit = {
|
||||
def imbueConfigs(sqlContext: SQLContext): Unit = {
|
||||
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.filterPushdown", "true")
|
||||
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.recordLevelFilter.enabled", "true")
|
||||
// TODO(HUDI-3639) vectorized reader has to be disabled to make sure MORIncrementalRelation is working properly
|
||||
|
||||
@@ -48,6 +48,11 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
|
||||
|
||||
override type FileSplit = HoodieMergeOnReadFileSplit
|
||||
|
||||
override def imbueConfigs(sqlContext: SQLContext): Unit = {
|
||||
super.imbueConfigs(sqlContext)
|
||||
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "false")
|
||||
}
|
||||
|
||||
override protected def timeline: HoodieTimeline = {
|
||||
val startTimestamp = optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key)
|
||||
val endTimestamp = optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key, super.timeline.lastInstant().get.getTimestamp)
|
||||
|
||||
@@ -56,6 +56,11 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
|
||||
protected val mergeType: String = optParams.getOrElse(DataSourceReadOptions.REALTIME_MERGE.key,
|
||||
DataSourceReadOptions.REALTIME_MERGE.defaultValue)
|
||||
|
||||
override def imbueConfigs(sqlContext: SQLContext): Unit = {
|
||||
super.imbueConfigs(sqlContext)
|
||||
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "true")
|
||||
}
|
||||
|
||||
protected override def composeRDD(fileSplits: Seq[HoodieMergeOnReadFileSplit],
|
||||
partitionSchema: StructType,
|
||||
tableSchema: HoodieTableSchema,
|
||||
|
||||
Reference in New Issue
Block a user