nsivabalan commented on a change in pull request #3968:
URL: https://github.com/apache/hudi/pull/3968#discussion_r752786890



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
##########
@@ -170,10 +191,15 @@ public void scan(Option<List<String>> keys) {
     HoodieTimeline completedInstantsTimeline = 
commitsTimeline.filterCompletedInstants();
     HoodieTimeline inflightInstantsTimeline = 
commitsTimeline.filterInflights();
     try {
-      // iterate over the paths
+
+      // Get the key field based on populate meta fields config
+      // and the table type
+      final String keyField = getKeyField();

Review comment:
       can we move this to constructor. if scan() is called multiple times 
(inline), we might call this multiple times.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
##########
@@ -120,28 +121,32 @@
   private int totalScannedLogFiles;
   // Progress
   private float progress = 0.0f;
+  // Partition name
+  private Option<String> partitionName;
+  // Populate meta fields for the records
+  private boolean populateMetaFieldsEnabled = true;

Review comment:
       we can name this "populateMetaFields", same as the config name.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java
##########
@@ -76,6 +84,23 @@ protected void processNextDeletedKey(HoodieKey hoodieKey) {
     }
   }
 
+  @Override
+  protected HoodieRecord<?> createHoodieRecord(final IndexedRecord rec, final 
HoodieTableConfig hoodieTableConfig,
+                                               final String payloadClassFQN, 
final String preCombineField,
+                                               final boolean 
withOperationField,
+                                               final Option<Pair<String, 
String>> simpleKeyGenFields,
+                                               final Option<String> 
partitionName) {
+    if (hoodieTableConfig.populateMetaFields()) {
+      return super.createHoodieRecord(rec, hoodieTableConfig, payloadClassFQN, 
preCombineField, withOperationField,
+          simpleKeyGenFields, partitionName);
+    }
+
+    // When meta fields are not available, create the record using the
+    // preset key field and the known partition name
+    return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, 
payloadClassFQN,
+        preCombineField, simpleKeyGenFields.get(), withOperationField, 
partitionName);

Review comment:
       where is the preset keyfield used?

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java
##########
@@ -190,9 +190,13 @@ private void initIfNeeded() {
         Option<GenericRecord> baseRecord = baseFileReader.getRecordByKey(key);
         if (baseRecord.isPresent()) {
           hoodieRecord = metadataTableConfig.populateMetaFields()
-              ? 
SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), 
metadataTableConfig.getPayloadClass(), 
metadataTableConfig.getPreCombineField(), false)
-              : 
SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(), 
metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(),
-              Pair.of(metadataTableConfig.getRecordKeyFieldProp(), 
metadataTableConfig.getPartitionFieldProp()), false);
+              ? 
SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(),

Review comment:
       can we move this to a new private method so that its more readable




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