You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/09/27 20:13:11 UTC

[GitHub] [hudi] yihua commented on a diff in pull request #6676: [HUDI-4453] Fix schema to include partition columns in bootstrap operation

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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org