yihua commented on code in PR #6676:
URL: https://github.com/apache/hudi/pull/6676#discussion_r981669014
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java:
##########
@@ -72,26 +68,15 @@ protected Schema
getBootstrapSourceSchema(HoodieEngineContext context, List<Pair
}
private static Schema getBootstrapSourceSchemaParquet(HoodieWriteConfig
writeConfig, HoodieEngineContext context, Path 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);
-
- StructType sparkSchema = converter.convert(parquetSchema);
+ StructType parquetSchema = ((HoodieSparkEngineContext)
context).getSqlContext().read()
+ .option("basePath", writeConfig.getBootstrapSourceBasePath())
+ .parquet(filePath.toString())
+ .schema();
String tableName =
HoodieAvroUtils.sanitizeName(writeConfig.getTableName());
String structName = tableName + "_record";
String recordNamespace = "hoodie." + tableName;
- return AvroConversionUtils.convertStructTypeToAvroSchema(sparkSchema,
structName, recordNamespace);
+ return AvroConversionUtils.convertStructTypeToAvroSchema(parquetSchema,
structName, recordNamespace);
Review Comment:
Good callout! I was debugging the schema resolution in
`HoodieBootstrapRelation` and didn't put the changes up for review. Now, I
verified that we don't need to append the partition column in
`HoodieBootstrapRelation` as the partition column is already available from the
table/read schema.
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java:
##########
@@ -72,26 +68,15 @@ protected Schema
getBootstrapSourceSchema(HoodieEngineContext context, List<Pair
}
private static Schema getBootstrapSourceSchemaParquet(HoodieWriteConfig
writeConfig, HoodieEngineContext context, Path 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(),
Review Comment:
Given that we're using the SQL context to read the schema of the parquet
table, all the SQLConf defaults are automatically added, so these config
settings are not needed anymore.
In the current bootstrap operation, we always assume the partition column is
String typed, so we have to turn off the type inference of the partition column
to be consistent with the existing behavior for now. I created HUDI-4932 to
support the config knob, so that other types of partition column can be
supported in the future.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]