-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-5835] After performing the update operation, the hoodie table cannot be read normally by spark #8026
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[HUDI-5835] After performing the update operation, the hoodie table cannot be read normally by spark #8026
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.{FileStatus, Path} | |
| import org.apache.hadoop.hbase.io.hfile.CacheConfig | ||
| import org.apache.hadoop.mapred.JobConf | ||
| import org.apache.hudi.HoodieBaseRelation._ | ||
| import org.apache.hudi.AvroConversionUtils.getAvroSchemaWithDefaults | ||
| import org.apache.hudi.HoodieConversionUtils.toScalaOption | ||
| import org.apache.hudi.avro.HoodieAvroUtils | ||
| import org.apache.hudi.client.utils.SparkInternalSchemaConverter | ||
|
|
@@ -67,12 +68,7 @@ import scala.util.{Failure, Success, Try} | |
|
|
||
| trait HoodieFileSplit {} | ||
|
|
||
| case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String, internalSchema: Option[InternalSchema] = None) { | ||
|
|
||
| def this(structTypeSchema: StructType) = | ||
| this(structTypeSchema, convertToAvroSchema(structTypeSchema).toString) | ||
|
|
||
| } | ||
| case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String, internalSchema: Option[InternalSchema] = None) | ||
|
|
||
| case class HoodieTableState(tablePath: String, | ||
| latestCommitTimestamp: Option[String], | ||
|
|
@@ -106,6 +102,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, | |
|
|
||
| protected lazy val resolver: Resolver = sparkSession.sessionState.analyzer.resolver | ||
|
|
||
| protected def tableName: String = metaClient.getTableConfig.getTableName | ||
|
|
||
| protected lazy val conf: Configuration = new Configuration(sqlContext.sparkContext.hadoopConfiguration) | ||
| protected lazy val jobConf = new JobConf(conf) | ||
|
|
||
|
|
@@ -162,12 +160,13 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, | |
| } | ||
| } | ||
|
|
||
| val (name, namespace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName) | ||
| val avroSchema = internalSchemaOpt.map { is => | ||
| AvroInternalSchemaConverter.convert(is, "schema") | ||
| AvroInternalSchemaConverter.convert(is, namespace + "." + name) | ||
| } orElse { | ||
| specifiedQueryTimestamp.map(schemaResolver.getTableAvroSchema) | ||
| } orElse { | ||
| schemaSpec.map(convertToAvroSchema) | ||
| schemaSpec.map(s => convertToAvroSchema(s, tableName)) | ||
| } getOrElse { | ||
| Try(schemaResolver.getTableAvroSchema) match { | ||
| case Success(schema) => schema | ||
|
|
@@ -356,7 +355,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, | |
| // schema conversion, which is lossy in nature (for ex, it doesn't preserve original Avro type-names) and | ||
| // could have an effect on subsequent de-/serializing records in some exotic scenarios (when Avro unions | ||
| // w/ more than 2 types are involved) | ||
| val sourceSchema = prunedDataSchema.map(convertToAvroSchema).getOrElse(tableAvroSchema) | ||
| val sourceSchema = prunedDataSchema.map(s => convertToAvroSchema(s, tableName)).getOrElse(tableAvroSchema) | ||
| val (requiredAvroSchema, requiredStructSchema, requiredInternalSchema) = | ||
| projectSchema(Either.cond(internalSchemaOpt.isDefined, internalSchemaOpt.get, sourceSchema), targetColumns) | ||
|
|
||
|
|
@@ -622,8 +621,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, | |
| val prunedRequiredSchema = prunePartitionColumns(requiredSchema.structTypeSchema) | ||
|
|
||
| (partitionSchema, | ||
| new HoodieTableSchema(prunedDataStructSchema), | ||
| new HoodieTableSchema(prunedRequiredSchema)) | ||
| HoodieTableSchema(prunedDataStructSchema, convertToAvroSchema(prunedDataStructSchema, tableName).toString), | ||
| HoodieTableSchema(prunedRequiredSchema, convertToAvroSchema(prunedRequiredSchema, tableName).toString)) | ||
| } else { | ||
| (StructType(Nil), tableSchema, requiredSchema) | ||
| } | ||
|
|
@@ -650,8 +649,11 @@ object HoodieBaseRelation extends SparkAdapterSupport { | |
| def apply(file: PartitionedFile): Iterator[InternalRow] = read.apply(file) | ||
| } | ||
|
|
||
| def convertToAvroSchema(structSchema: StructType): Schema = | ||
| sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = false, "Record") | ||
| def convertToAvroSchema(structSchema: StructType, tableName: String ): Schema = { | ||
| val (recordName, namespace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName) | ||
| val avroSchema = sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = false, recordName, namespace) | ||
| getAvroSchemaWithDefaults(avroSchema, structSchema) | ||
| } | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @xiarixiaoyao i still don't understand why we need to set defaults in the schema. Can you please elaborate on that one?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, eg: If we add a new column newCol: string to the table, the default value of newCol will be null |
||
|
|
||
| def getPartitionPath(fileStatus: FileStatus): Path = | ||
| fileStatus.getPath.getParent | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.