bvaradar commented on a change in pull request #4910:
URL: https://github.com/apache/hudi/pull/4910#discussion_r828027957



##########
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java
##########
@@ -77,14 +89,39 @@ public void runMerge(HoodieTable<T, 
JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>
       readSchema = mergeHandle.getWriterSchemaWithMetaFields();
     }
 
+    Option<InternalSchema> querySchemaOpt = 
SerDeHelper.fromJson(table.getConfig().getInternalSchema());
+    Boolean needToReWriteRecord = false;
+    // to do support bootstrap
+    if (querySchemaOpt.isPresent() && 
!baseFile.getBootstrapBaseFile().isPresent()) {
+      // check implicitly add columns, and position reorder(spark sql may 
change cols order)
+      InternalSchema querySchema = 
AvroSchemaUtil.evolutionSchemaFromNewAvroSchema(readSchema, 
querySchemaOpt.get(), true);
+      long commitTime = 
Long.valueOf(FSUtils.getCommitTime(mergeHandle.getOldFilePath().getName()));
+      InternalSchema writeInternalSchema = 
TableInternalSchemaUtils.searchSchemaAndCache(commitTime, 
table.getMetaClient());
+      List<String> colNamesFromQuerySchema = querySchema.getAllColsFullName();
+      List<String> colNamesFromWriteSchema = 
writeInternalSchema.getAllColsFullName();
+      List<String> diffCols = colNamesFromWriteSchema.stream()

Review comment:
       Should this be named sameCols?

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
##########
@@ -361,15 +372,38 @@ private boolean isNewInstantBlock(HoodieLogBlock 
logBlock) {
    */
   private void processDataBlock(HoodieDataBlock dataBlock, 
Option<List<String>> keys) throws Exception {
     try (ClosableIterator<IndexedRecord> recordItr = 
dataBlock.getRecordItr(keys.orElse(Collections.emptyList()))) {
+      boolean headRecord = true;
+      Option<Schema> schemaOption = Option.empty();
       while (recordItr.hasNext()) {
-        IndexedRecord record = recordItr.next();
+        IndexedRecord currentRecord = recordItr.next();
+        if (headRecord) {
+          schemaOption = getMergedSchema(dataBlock, currentRecord);
+          headRecord = false;
+        }
+        IndexedRecord record = schemaOption.isPresent() ? 
AvroSchemaUtil.rewriteRecord(currentRecord, schemaOption.get()) : currentRecord;

Review comment:
       Looks like we are doing this for reading from Log file. Are we doing 
similar schema rewrite when reading from previous version of parquet file for 
compaction and COW ?




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


Reply via email to