Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -53,13 +53,15 @@ object HoodieSparkUtils extends SparkAdapterSupport {

def isSpark3_1: Boolean = SPARK_VERSION.startsWith("3.1")

def gteqSpark3_1: Boolean = SPARK_VERSION > "3.1"

def gteqSpark3_1_3: Boolean = SPARK_VERSION >= "3.1.3"

def isSpark3_2: Boolean = SPARK_VERSION.startsWith("3.2")

def gteqSpark3_2: Boolean = SPARK_VERSION > "3.2"

def gteqSpark3_1: Boolean = SPARK_VERSION > "3.1"

def gteqSpark3_1_3: Boolean = SPARK_VERSION >= "3.1.3"
def gteqSpark3_2_1: Boolean = SPARK_VERSION >= "3.2.1"

def getMetaSchema: StructType = {
StructType(HoodieRecord.HOODIE_META_COLUMNS.asScala.map(col => {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -55,6 +55,6 @@ class Spark3_1Adapter extends BaseSpark3Adapter {
}

override def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat] = {
Some(new Spark312HoodieParquetFileFormat(appendPartitionValues))
Some(new Spark31HoodieParquetFileFormat(appendPartitionValues))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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._
Expand All @@ -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,
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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)
Expand All @@ -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 = {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution.datasources.parquet

import org.apache.spark.sql.SPARK_VERSION_METADATA_KEY
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy
import org.apache.spark.util.Utils

object Spark32DataSourceUtils {

/**
* NOTE: This method was copied from Spark 3.2.0, and is required to maintain runtime
* compatibility against Spark 3.2.0
*/
// scalastyle:off
def int96RebaseMode(lookupFileMeta: String => String,
modeByConfig: String): LegacyBehaviorPolicy.Value = {
if (Utils.isTesting && SQLConf.get.getConfString("spark.test.forceNoRebase", "") == "true") {
return LegacyBehaviorPolicy.CORRECTED
}
// If there is no version, we return the mode specified by the config.
Option(lookupFileMeta(SPARK_VERSION_METADATA_KEY)).map { version =>
// Files written by Spark 3.0 and earlier follow the legacy hybrid calendar and we need to
// rebase the INT96 timestamp values.
// Files written by Spark 3.1 and latter may also need the rebase if they were written with
// the "LEGACY" rebase mode.
if (version < "3.1.0" || lookupFileMeta("org.apache.spark.legacyINT96") != null) {
LegacyBehaviorPolicy.LEGACY
} else {
LegacyBehaviorPolicy.CORRECTED
}
}.getOrElse(LegacyBehaviorPolicy.withName(modeByConfig))
}
// scalastyle:on

/**
* NOTE: This method was copied from Spark 3.2.0, and is required to maintain runtime
* compatibility against Spark 3.2.0
*/
// scalastyle:off
def datetimeRebaseMode(lookupFileMeta: String => String,
modeByConfig: String): LegacyBehaviorPolicy.Value = {
if (Utils.isTesting && SQLConf.get.getConfString("spark.test.forceNoRebase", "") == "true") {
return LegacyBehaviorPolicy.CORRECTED
}
// If there is no version, we return the mode specified by the config.
Option(lookupFileMeta(SPARK_VERSION_METADATA_KEY)).map { version =>
// Files written by Spark 2.4 and earlier follow the legacy hybrid calendar and we need to
// rebase the datetime values.
// Files written by Spark 3.0 and latter may also need the rebase if they were written with
// the "LEGACY" rebase mode.
if (version < "3.0.0" || lookupFileMeta("org.apache.spark.legacyDateTime") != null) {
LegacyBehaviorPolicy.LEGACY
} else {
LegacyBehaviorPolicy.CORRECTED
}
}.getOrElse(LegacyBehaviorPolicy.withName(modeByConfig))
}
// scalastyle:on

}
Loading