1
0

[HUDI-3934] Fix Spark32HoodieParquetFileFormat not being compatible w/ Spark 3.2.0 (#5378)

- Due to the fact that Spark 3.2.1 is non-BWC w/ 3.2.0, we have to handle all these incompatibilities in Spark32HoodieParquetFileFormat. This PR is addressing that.

Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
Alexey Kudinkin
2022-04-21 18:00:38 -07:00
committed by GitHub
parent c4bc2deea0
commit c05a4e7b6f
5 changed files with 229 additions and 48 deletions

View File

@@ -23,7 +23,7 @@ import org.apache.spark.SPARK_VERSION
import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer, HoodieSpark3_1AvroDeserializer, HoodieSpark3_1AvroSerializer}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark312HoodieParquetFileFormat}
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark31HoodieParquetFileFormat}
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieSpark3_1CatalystExpressionUtils, SparkSession}
@@ -55,6 +55,6 @@ class Spark3_1Adapter extends BaseSpark3Adapter {
}
override def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = {
Some(new Spark312HoodieParquetFileFormat(appendPartitionValues))
Some(new Spark31HoodieParquetFileFormat(appendPartitionValues))
}
}

View File

@@ -25,7 +25,7 @@ import org.apache.hudi.HoodieSparkUtils
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.util.StringUtils.isNullOrEmpty
import org.apache.hudi.common.util.{InternalSchemaCache, StringUtils}
import org.apache.hudi.common.util.{InternalSchemaCache, ReflectionUtils, StringUtils}
import org.apache.hudi.common.util.collection.Pair
import org.apache.hudi.internal.schema.InternalSchema
import org.apache.hudi.internal.schema.action.InternalSchemaMerger
@@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.datasources.parquet.Spark312HoodieParquetFileFormat.{createParquetFilters, pruneInternalSchema, rebuildFilterFromParquet}
import org.apache.spark.sql.execution.datasources.parquet.Spark31HoodieParquetFileFormat.{createParquetFilters, pruneInternalSchema, rebuildFilterFromParquet}
import org.apache.spark.sql.execution.datasources.{DataSourceUtils, PartitionedFile, RecordReaderIterator}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources._
@@ -61,7 +61,7 @@ import java.net.URI
* <li>Schema on-read</li>
* </ol>
*/
class Spark312HoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat {
class Spark31HoodieParquetFileFormat(private val shouldAppendPartitionValues: Boolean) extends ParquetFileFormat {
override def buildReaderWithPartitionValues(sparkSession: SparkSession,
dataSchema: StructType,
@@ -154,8 +154,8 @@ class Spark312HoodieParquetFileFormat(private val shouldAppendPartitionValues: B
val shouldUseInternalSchema = !isNullOrEmpty(internalSchemaStr) && querySchemaOption.isPresent
val tablePath = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH)
val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong;
val fileSchema = if (shouldUseInternalSchema) {
val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong;
val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST)
InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, sharedConf, if (validCommits == null) "" else validCommits)
} else {
@@ -223,13 +223,17 @@ class Spark312HoodieParquetFileFormat(private val shouldAppendPartitionValues: B
// Clone new conf
val hadoopAttemptConf = new Configuration(broadcastedHadoopConf.value.value)
var typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = new java.util.HashMap()
if (shouldUseInternalSchema) {
var typeChangeInfos: java.util.Map[Integer, Pair[DataType, DataType]] = if (shouldUseInternalSchema) {
val mergedInternalSchema = new InternalSchemaMerger(fileSchema, querySchemaOption.get(), true, true).mergeSchema()
val mergedSchema = SparkInternalSchemaConverter.constructSparkSchemaFromInternalSchema(mergedInternalSchema)
typeChangeInfos = SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema)
hadoopAttemptConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, mergedSchema.json)
SparkInternalSchemaConverter.collectTypeChangedCols(querySchemaOption.get(), mergedInternalSchema)
} else {
new java.util.HashMap()
}
val hadoopAttemptContext =
new TaskAttemptContextImpl(hadoopAttemptConf, attemptId)
@@ -329,9 +333,7 @@ class Spark312HoodieParquetFileFormat(private val shouldAppendPartitionValues: B
}
}
object Spark312HoodieParquetFileFormat {
val PARQUET_FILTERS_CLASS_NAME = "org.apache.spark.sql.execution.datasources.parquet.ParquetFilters"
object Spark31HoodieParquetFileFormat {
def pruneInternalSchema(internalSchemaStr: String, requiredSchema: StructType): String = {
val querySchemaOption = SerDeHelper.fromJson(internalSchemaStr)
@@ -343,10 +345,11 @@ object Spark312HoodieParquetFileFormat {
}
}
private def createParquetFilters(arg: Any*): ParquetFilters = {
val clazz = Class.forName(PARQUET_FILTERS_CLASS_NAME, true, Thread.currentThread().getContextClassLoader)
val ctor = clazz.getConstructors.head
ctor.newInstance(arg.map(_.asInstanceOf[AnyRef]): _*).asInstanceOf[ParquetFilters]
private def createParquetFilters(args: Any*): ParquetFilters = {
// ParquetFilters bears a single ctor (in Spark 3.1)
val ctor = classOf[ParquetFilters].getConstructors.head
ctor.newInstance(args.map(_.asInstanceOf[AnyRef]): _*)
.asInstanceOf[ParquetFilters]
}
private def rebuildFilterFromParquet(oldFilter: Filter, fileSchema: InternalSchema, querySchema: InternalSchema): Filter = {