nsivabalan commented on code in PR #8915:
URL: https://github.com/apache/hudi/pull/8915#discussion_r1224689577


##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -1361,7 +1363,13 @@ public static Set<String> 
getValidInstantTimestamps(HoodieTableMetaClient dataMe
         });
 
     // SOLO_COMMIT_TIMESTAMP is used during bootstrap so it is a valid 
timestamp
-    validInstantTimestamps.add(createIndexInitTimestamp(SOLO_COMMIT_TIMESTAMP, 
PARTITION_INITIALIZATION_TIME_SUFFIX));
+    List<String> metadataInitializationTimestamps = 
metadataMetaClient.getActiveTimeline()

Review Comment:
   I am also considering if this will give us any benefit. 
   validInstantTimestamps are used within LogRecordReader to ignore log blocks 
which was written using commits which are not completed yet. 
   
   Lets consider diff cases: 
   1. for an existing table, we may never use SOLO commit time only since there 
will def be a latest completed commit from data table that we will use. 
   2. For a new table. the base commit time to initialize MDT will be chosen as 
SOLO COMMIT TIME + suffix (one for each partition being initialized). and bulk 
insert will kick in. So, the base instant time will have prefix of SOLO COMMIT 
TIME . but any new log files will be added using new delta commits which will 
have diff commit times. so, I don't see a necessity to add SOLO COMMIT TIME  to 
list of valid instant times only.
   
   let me know if I am missing any flow. Just tryin to avoid going through 
entire active timeline of MDT to filter for SOLO COMMIT TIME if its never going 
to be used. 
   
   
   



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##########
@@ -471,12 +471,9 @@ public Pair<HoodieMetadataLogRecordReader, Long> 
getLogRecordScanner(List<Hoodie
 
     // Only those log files which have a corresponding completed instant on 
the dataset should be read
     // This is because the metadata table is updated before the dataset 
instants are committed.
-    Set<String> validInstantTimestamps = HoodieTableMetadataUtil
-        .getValidInstantTimestamps(dataMetaClient, metadataMetaClient);
-
+    Set<String> validInstantTimestamps = 
HoodieTableMetadataUtil.getValidInstantTimestamps(dataMetaClient, 
metadataMetaClient);
     Option<HoodieInstant> latestMetadataInstant = 
metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant();
-    String latestMetadataInstantTime = 
latestMetadataInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP);
-
+    String latestMetadataInstantTime = 
latestMetadataInstant.map(HoodieInstant::getTimestamp).orElse(HoodieTableMetadataUtil.createIndexInitTimestamp(SOLO_COMMIT_TIMESTAMP,
 0));

Review Comment:
   I don't see much benefit here too. 
   but not too strong.
   
   this code will be invoked only after any partition in MDT will be 
initialized. which means, the latestMetadataInstant should already be valid 
(Option will be non empty). So, what are the chances that we will call 
getRecordsByKey with BaseTableMetadata when any of MDT partitions have been 
initialized. 
   
   
   
   
   



##########
hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java:
##########
@@ -434,6 +434,6 @@ protected Configuration getHadoopConf() {
 
   protected String getLatestDataInstantTime() {
     return 
dataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant()
-        .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP);
+        
.map(HoodieInstant::getTimestamp).orElse(HoodieTableMetadataUtil.createIndexInitTimestamp(SOLO_COMMIT_TIMESTAMP,
 0));

Review Comment:
   not sure if this solves/gives us much. 
   1: if we happened to initialize more than 1 MDT partition, the 
initialization time will be different. its 010 suffix for 1st and 011 for 2nd. 
   2: this api is used only in logging. 
   
   So, may not be worth fixing it. atleast for this (getLatestDataInstantTime). 



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