[HUDI-4186] Support Hudi with Spark 3.3.0 (#5943)
Co-authored-by: Shawn Chang <yxchang@amazon.com>
This commit is contained in:
@@ -130,6 +130,16 @@ public class HoodieStorageConfig extends HoodieConfig {
|
||||
.defaultValue("TIMESTAMP_MICROS")
|
||||
.withDocumentation("Sets spark.sql.parquet.outputTimestampType. Parquet timestamp type to use when Spark writes data to Parquet files.");
|
||||
|
||||
// SPARK-38094 Spark 3.3 checks if this field is enabled. Hudi has to provide this or there would be NPE thrown
|
||||
// Would ONLY be effective with Spark 3.3+
|
||||
// default value is true which is in accordance with Spark 3.3
|
||||
public static final ConfigProperty<String> PARQUET_FIELD_ID_WRITE_ENABLED = ConfigProperty
|
||||
.key("hoodie.parquet.field_id.write.enabled")
|
||||
.defaultValue("true")
|
||||
.sinceVersion("0.12.0")
|
||||
.withDocumentation("Would only be effective with Spark 3.3+. Sets spark.sql.parquet.fieldId.write.enabled. "
|
||||
+ "If enabled, Spark will write out parquet native field ids that are stored inside StructField's metadata as parquet.field.id to parquet files.");
|
||||
|
||||
public static final ConfigProperty<String> HFILE_COMPRESSION_ALGORITHM_NAME = ConfigProperty
|
||||
.key("hoodie.hfile.compression.algorithm")
|
||||
.defaultValue("GZ")
|
||||
@@ -337,6 +347,11 @@ public class HoodieStorageConfig extends HoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetFieldIdWrite(String parquetFieldIdWrite) {
|
||||
storageConfig.setValue(PARQUET_FIELD_ID_WRITE_ENABLED, parquetFieldIdWrite);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hfileCompressionAlgorithm(String hfileCompressionAlgorithm) {
|
||||
storageConfig.setValue(HFILE_COMPRESSION_ALGORITHM_NAME, hfileCompressionAlgorithm);
|
||||
return this;
|
||||
|
||||
@@ -1681,6 +1681,10 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
return getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE);
|
||||
}
|
||||
|
||||
public String parquetFieldIdWriteEnabled() {
|
||||
return getString(HoodieStorageConfig.PARQUET_FIELD_ID_WRITE_ENABLED);
|
||||
}
|
||||
|
||||
public Option<HoodieLogBlock.HoodieLogBlockType> getLogDataBlockFormat() {
|
||||
return Option.ofNullable(getString(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT))
|
||||
.map(HoodieLogBlock.HoodieLogBlockType::fromId);
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.client.bootstrap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
@@ -71,11 +72,20 @@ public class HoodieSparkBootstrapSchemaProvider extends HoodieBootstrapSchemaPro
|
||||
}
|
||||
|
||||
private static Schema getBootstrapSourceSchemaParquet(HoodieWriteConfig writeConfig, HoodieEngineContext context, Path filePath) {
|
||||
MessageType parquetSchema = new ParquetUtils().readSchema(context.getHadoopConf().get(), filePath);
|
||||
Configuration hadoopConf = context.getHadoopConf().get();
|
||||
MessageType parquetSchema = new ParquetUtils().readSchema(hadoopConf, filePath);
|
||||
|
||||
hadoopConf.set(
|
||||
SQLConf.PARQUET_BINARY_AS_STRING().key(),
|
||||
SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString());
|
||||
hadoopConf.set(
|
||||
SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(),
|
||||
SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString());
|
||||
hadoopConf.set(
|
||||
SQLConf.CASE_SENSITIVE().key(),
|
||||
SQLConf.CASE_SENSITIVE().defaultValueString());
|
||||
ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(hadoopConf);
|
||||
|
||||
ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(
|
||||
Boolean.parseBoolean(SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()),
|
||||
Boolean.parseBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()));
|
||||
StructType sparkSchema = converter.convert(parquetSchema);
|
||||
String tableName = HoodieAvroUtils.sanitizeName(writeConfig.getTableName());
|
||||
String structName = tableName + "_record";
|
||||
|
||||
@@ -50,6 +50,7 @@ public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
|
||||
Configuration hadoopConf = new Configuration(conf);
|
||||
hadoopConf.set("spark.sql.parquet.writeLegacyFormat", writeConfig.parquetWriteLegacyFormatEnabled());
|
||||
hadoopConf.set("spark.sql.parquet.outputTimestampType", writeConfig.parquetOutputTimestampType());
|
||||
hadoopConf.set("spark.sql.parquet.fieldId.write.enabled", writeConfig.parquetFieldIdWriteEnabled());
|
||||
this.hadoopConf = hadoopConf;
|
||||
setSchema(structType, hadoopConf);
|
||||
this.bloomFilter = bloomFilterOpt.orElse(null);
|
||||
|
||||
@@ -52,11 +52,13 @@ private[hudi] trait SparkVersionsSupport {
|
||||
def isSpark3_0: Boolean = getSparkVersion.startsWith("3.0")
|
||||
def isSpark3_1: Boolean = getSparkVersion.startsWith("3.1")
|
||||
def isSpark3_2: Boolean = getSparkVersion.startsWith("3.2")
|
||||
def isSpark3_3: Boolean = getSparkVersion.startsWith("3.3")
|
||||
|
||||
def gteqSpark3_1: Boolean = getSparkVersion >= "3.1"
|
||||
def gteqSpark3_1_3: Boolean = getSparkVersion >= "3.1.3"
|
||||
def gteqSpark3_2: Boolean = getSparkVersion >= "3.2"
|
||||
def gteqSpark3_2_1: Boolean = getSparkVersion >= "3.2.1"
|
||||
def gteqSpark3_3: Boolean = getSparkVersion >= "3.3"
|
||||
}
|
||||
|
||||
object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport {
|
||||
|
||||
@@ -27,7 +27,9 @@ import org.apache.spark.sql.hudi.SparkAdapter
|
||||
trait SparkAdapterSupport {
|
||||
|
||||
lazy val sparkAdapter: SparkAdapter = {
|
||||
val adapterClass = if (HoodieSparkUtils.isSpark3_2) {
|
||||
val adapterClass = if (HoodieSparkUtils.isSpark3_3) {
|
||||
"org.apache.spark.sql.adapter.Spark3_3Adapter"
|
||||
} else if (HoodieSparkUtils.isSpark3_2) {
|
||||
"org.apache.spark.sql.adapter.Spark3_2Adapter"
|
||||
} else if (HoodieSparkUtils.isSpark3_0 || HoodieSparkUtils.isSpark3_1) {
|
||||
"org.apache.spark.sql.adapter.Spark3_1Adapter"
|
||||
|
||||
@@ -24,17 +24,15 @@ import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSchemaConver
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTable
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
|
||||
import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate}
|
||||
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, InterpretedPredicate}
|
||||
import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.JoinType
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, SubqueryAlias}
|
||||
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, SubqueryAlias}
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.execution.datasources.{FilePartition, LogicalRelation, PartitionedFile, SparkParsePartitionUtil}
|
||||
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, LogicalRelation, PartitionedFile, SparkParsePartitionUtil}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.DataType
|
||||
import org.apache.spark.sql.types.{DataType, StructType}
|
||||
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, Row, SparkSession}
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
|
||||
@@ -132,8 +130,8 @@ trait SparkAdapter extends Serializable {
|
||||
}
|
||||
|
||||
/**
|
||||
* Create instance of [[ParquetFileFormat]]
|
||||
*/
|
||||
* Create instance of [[ParquetFileFormat]]
|
||||
*/
|
||||
def createHoodieParquetFileFormat(appendPartitionValues: Boolean): Option[ParquetFileFormat]
|
||||
|
||||
/**
|
||||
@@ -143,6 +141,38 @@ trait SparkAdapter extends Serializable {
|
||||
*/
|
||||
def createInterpretedPredicate(e: Expression): InterpretedPredicate
|
||||
|
||||
/**
|
||||
* Create instance of [[HoodieFileScanRDD]]
|
||||
* SPARK-37273 FileScanRDD constructor changed in SPARK 3.3
|
||||
*/
|
||||
def createHoodieFileScanRDD(sparkSession: SparkSession,
|
||||
readFunction: PartitionedFile => Iterator[InternalRow],
|
||||
filePartitions: Seq[FilePartition],
|
||||
readDataSchema: StructType,
|
||||
metadataColumns: Seq[AttributeReference] = Seq.empty): FileScanRDD
|
||||
|
||||
/**
|
||||
* Resolve [[DeleteFromTable]]
|
||||
* SPARK-38626 condition is no longer Option in Spark 3.3
|
||||
*/
|
||||
def resolveDeleteFromTable(deleteFromTable: Command,
|
||||
resolveExpression: Expression => Expression): LogicalPlan
|
||||
|
||||
/**
|
||||
* Extract condition in [[DeleteFromTable]]
|
||||
* SPARK-38626 condition is no longer Option in Spark 3.3
|
||||
*/
|
||||
def extractCondition(deleteFromTable: Command): Expression
|
||||
|
||||
/**
|
||||
* Get parseQuery from ExtendedSqlParser, only for Spark 3.3+
|
||||
*/
|
||||
def getQueryParserFromExtendedSqlParser(session: SparkSession, delegate: ParserInterface,
|
||||
sqlText: String): LogicalPlan = {
|
||||
// unsupported by default
|
||||
throw new UnsupportedOperationException(s"Unsupported parseQuery method in Spark earlier than Spark 3.3.0")
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts instance of [[StorageLevel]] to a corresponding string
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user