yihua commented on code in PR #11947:
URL: https://github.com/apache/hudi/pull/11947#discussion_r1797472112


##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java:
##########
@@ -175,33 +181,72 @@ public Pair<Option<Dataset<Row>>, String> 
fetchNextBatch(Option<String> lastCkpt
     IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy =
         (containsConfigProperty(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             ? IncrSourceHelper.MissingCheckpointStrategy.valueOf(
-            getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
+                getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             : null;
     if (readLatestOnMissingCkpt) {
       missingCheckpointStrategy = 
IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST;
     }
 
-    // Use begin Instant if set and non-empty
-    Option<String> beginInstant =
-        lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Option.empty() 
: lastCkptStr : Option.empty();
+    HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+        
.setConf(HadoopFSUtils.getStorageConfWithCopy(sparkContext.hadoopConfiguration()))
+        .setBasePath(srcPath)
+        .setLoadActiveTimelineOnLoad(true)
+        .build();
 
+    int numInstantsFromConfig = getIntWithAltKeys(props, 
HoodieIncrSourceConfig.NUM_INSTANTS_PER_FETCH);
+
+    String startTime;

Review Comment:
   ```suggestion
       String completionTimeToResumeFrom;
   ```



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java:
##########
@@ -175,33 +181,72 @@ public Pair<Option<Dataset<Row>>, String> 
fetchNextBatch(Option<String> lastCkpt
     IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy =
         (containsConfigProperty(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             ? IncrSourceHelper.MissingCheckpointStrategy.valueOf(
-            getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
+                getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             : null;
     if (readLatestOnMissingCkpt) {
       missingCheckpointStrategy = 
IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST;
     }
 
-    // Use begin Instant if set and non-empty
-    Option<String> beginInstant =
-        lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Option.empty() 
: lastCkptStr : Option.empty();
+    HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+        
.setConf(HadoopFSUtils.getStorageConfWithCopy(sparkContext.hadoopConfiguration()))
+        .setBasePath(srcPath)
+        .setLoadActiveTimelineOnLoad(true)
+        .build();
 
+    int numInstantsFromConfig = getIntWithAltKeys(props, 
HoodieIncrSourceConfig.NUM_INSTANTS_PER_FETCH);
+
+    String startTime;

Review Comment:
   This can be a `Option<String>` (see my comment below).



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java:
##########
@@ -175,33 +181,72 @@ public Pair<Option<Dataset<Row>>, String> 
fetchNextBatch(Option<String> lastCkpt
     IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy =
         (containsConfigProperty(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             ? IncrSourceHelper.MissingCheckpointStrategy.valueOf(
-            getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
+                getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             : null;
     if (readLatestOnMissingCkpt) {
       missingCheckpointStrategy = 
IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST;
     }
 
-    // Use begin Instant if set and non-empty
-    Option<String> beginInstant =
-        lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Option.empty() 
: lastCkptStr : Option.empty();
+    HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+        
.setConf(HadoopFSUtils.getStorageConfWithCopy(sparkContext.hadoopConfiguration()))
+        .setBasePath(srcPath)
+        .setLoadActiveTimelineOnLoad(true)
+        .build();
 
+    int numInstantsFromConfig = getIntWithAltKeys(props, 
HoodieIncrSourceConfig.NUM_INSTANTS_PER_FETCH);
+
+    String startTime;
+    if (lastCkptStr.isPresent() && !lastCkptStr.get().isEmpty()) {
+      startTime = lastCkptStr.get();
+    } else if (missingCheckpointStrategy != null) {
+      switch (missingCheckpointStrategy) {
+        case READ_UPTO_LATEST_COMMIT:
+          startTime = DEFAULT_BEGIN_TIMESTAMP;

Review Comment:
   To be consistent with the `IncrementalQueryAnalyzer` semantics, it's better 
to make `completionTimeToResumeFrom` `null` or `Option.empty` instead of 
`DEFAULT_BEGIN_TIMESTAMP` (we can remove `DEFAULT_BEGIN_TIMESTAMP` if not used 
anywhere else).  We pass `null`/`Option.empty` to `IncrementalQueryAnalyzer` as 
the begin time.



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java:
##########
@@ -175,33 +181,72 @@ public Pair<Option<Dataset<Row>>, String> 
fetchNextBatch(Option<String> lastCkpt
     IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy =
         (containsConfigProperty(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             ? IncrSourceHelper.MissingCheckpointStrategy.valueOf(
-            getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
+                getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             : null;
     if (readLatestOnMissingCkpt) {
       missingCheckpointStrategy = 
IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST;
     }
 
-    // Use begin Instant if set and non-empty
-    Option<String> beginInstant =
-        lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Option.empty() 
: lastCkptStr : Option.empty();
+    HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+        
.setConf(HadoopFSUtils.getStorageConfWithCopy(sparkContext.hadoopConfiguration()))
+        .setBasePath(srcPath)
+        .setLoadActiveTimelineOnLoad(true)
+        .build();
 
+    int numInstantsFromConfig = getIntWithAltKeys(props, 
HoodieIncrSourceConfig.NUM_INSTANTS_PER_FETCH);
+
+    String startTime;
+    if (lastCkptStr.isPresent() && !lastCkptStr.get().isEmpty()) {
+      startTime = lastCkptStr.get();
+    } else if (missingCheckpointStrategy != null) {
+      switch (missingCheckpointStrategy) {
+        case READ_UPTO_LATEST_COMMIT:
+          startTime = DEFAULT_BEGIN_TIMESTAMP;
+          // disrespect numInstantsFromConfig when reading up to latest
+          numInstantsFromConfig = -1;
+          break;
+        case READ_LATEST:
+          Option<HoodieInstant> lastInstant = metaClient
+              .getCommitsAndCompactionTimeline()
+              .filterCompletedInstants()
+              .lastInstant();
+          startTime = lastInstant
+              .map(hoodieInstant -> 
getStrictlyLowerTimestamp(hoodieInstant.getCompletionTime()))
+              .orElse(DEFAULT_BEGIN_TIMESTAMP);
+          break;
+        default:
+          throw new IllegalArgumentException("Unknown missing checkpoint 
strategy: " + missingCheckpointStrategy);
+      }
+    } else {
+      throw new IllegalArgumentException("Missing begin instant for 
incremental pull. For reading from latest "
+          + "committed instant set 
hoodie.streamer.source.hoodieincr.missing.checkpoint.strategy to a valid 
value");
+    }
+
+    final int numInstantsFromConfigFinal = numInstantsFromConfig;
     // If source profile exists, use the numInstants from source profile.
-    final int numInstantsFromConfig = getIntWithAltKeys(props, 
HoodieIncrSourceConfig.NUM_INSTANTS_PER_FETCH);
     int numInstantsPerFetch = getLatestSourceProfile().map(sourceProfile -> {
       int numInstantsFromSourceProfile = 
sourceProfile.getSourceSpecificContext();
-      LOG.info("Overriding numInstantsPerFetch from source profile 
numInstantsFromSourceProfile {} , numInstantsFromConfig {}", 
numInstantsFromSourceProfile, numInstantsFromConfig);
+      LOG.info("Overriding numInstantsPerFetch from source profile 
numInstantsFromSourceProfile {} , numInstantsFromConfig {}",
+          numInstantsFromSourceProfile, numInstantsFromConfigFinal);
       return numInstantsFromSourceProfile;
     }).orElse(numInstantsFromConfig);
 
-    HollowCommitHandling handlingMode = getHollowCommitHandleMode(props);
-    QueryInfo queryInfo = generateQueryInfo(sparkContext, srcPath,
-        numInstantsPerFetch, beginInstant, missingCheckpointStrategy, 
handlingMode,
-        HoodieRecord.COMMIT_TIME_METADATA_FIELD, 
HoodieRecord.RECORD_KEY_METADATA_FIELD,
-        null, false, Option.empty());
+    IncrementalQueryAnalyzer analyzer = IncrementalQueryAnalyzer.builder()
+        .metaClient(metaClient)
+        .startTime(startTime)
+        .endTime(null)
+        .rangeType(RangeType.OPEN_CLOSED)
+        .limit(numInstantsPerFetch)
+        .build();
+
+    QueryContext queryContext = analyzer.analyze();
+    Option<InstantRange> instantRange = queryContext.getInstantRange();
 
-    if (queryInfo.areStartAndEndInstantsEqual()) {
+    String endTime;

Review Comment:
   ```suggestion
       String completionTimeToStopAt;
   ```



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java:
##########
@@ -175,33 +181,72 @@ public Pair<Option<Dataset<Row>>, String> 
fetchNextBatch(Option<String> lastCkpt
     IncrSourceHelper.MissingCheckpointStrategy missingCheckpointStrategy =
         (containsConfigProperty(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             ? IncrSourceHelper.MissingCheckpointStrategy.valueOf(
-            getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
+                getStringWithAltKeys(props, 
HoodieIncrSourceConfig.MISSING_CHECKPOINT_STRATEGY))
             : null;
     if (readLatestOnMissingCkpt) {
       missingCheckpointStrategy = 
IncrSourceHelper.MissingCheckpointStrategy.READ_LATEST;
     }
 
-    // Use begin Instant if set and non-empty
-    Option<String> beginInstant =
-        lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Option.empty() 
: lastCkptStr : Option.empty();
+    HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+        
.setConf(HadoopFSUtils.getStorageConfWithCopy(sparkContext.hadoopConfiguration()))
+        .setBasePath(srcPath)
+        .setLoadActiveTimelineOnLoad(true)
+        .build();
 
+    int numInstantsFromConfig = getIntWithAltKeys(props, 
HoodieIncrSourceConfig.NUM_INSTANTS_PER_FETCH);
+
+    String startTime;
+    if (lastCkptStr.isPresent() && !lastCkptStr.get().isEmpty()) {
+      startTime = lastCkptStr.get();
+    } else if (missingCheckpointStrategy != null) {
+      switch (missingCheckpointStrategy) {
+        case READ_UPTO_LATEST_COMMIT:
+          startTime = DEFAULT_BEGIN_TIMESTAMP;
+          // disrespect numInstantsFromConfig when reading up to latest
+          numInstantsFromConfig = -1;
+          break;
+        case READ_LATEST:
+          Option<HoodieInstant> lastInstant = metaClient
+              .getCommitsAndCompactionTimeline()
+              .filterCompletedInstants()
+              .lastInstant();
+          startTime = lastInstant
+              .map(hoodieInstant -> 
getStrictlyLowerTimestamp(hoodieInstant.getCompletionTime()))
+              .orElse(DEFAULT_BEGIN_TIMESTAMP);

Review Comment:
   Similar here on using `null` or `Option.empty`.



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