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/19 13:11:54 UTC

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

codope commented on code in PR #6676:
URL: https://github.com/apache/hudi/pull/6676#discussion_r974229919


##########
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:
   Should we keep these configs? Does spark...schema() infer string date values as date type?



##########
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:
   now that partition column will be written to the commit metadata, do we still need to append partition column in `HoodieBootstrapRelation`?



-- 
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