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 2023/01/09 08:02:07 UTC

[GitHub] [hudi] boneanxs opened a new pull request, #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

boneanxs opened a new pull request, #7627:
URL: https://github.com/apache/hudi/pull/7627

   ### Change Logs
   
   _Describe context and summary for this change. Highlight if any code was copied._
   1. `HoodieInstant` add state transition time support
   2. Add some methods relating to state transition time in HoodieTimeline
   3. Add new configure to allow incremental pulling data by state transition time
   4. Spark `HoodieStreamSource` need to use state transition time to incremental pulling data
   
   ### Impact
   
   Hudi timeline can actually miss some instants if we incremental pulling from upstream hudi table, which is written by several writers.
   
   For example, say we have 2 writers writing data to the hudi table, and the last success incremental pulling end timestamp is 001
   
   w1 is writing 002, w2 is writing 003, if w2 is finished earlier than the w1, then the incremental pulling end timestamp will be updated to 003, and actually w1's commit: 002 will be skipped since it's instant time is earlier than the w2's.
   
   We actually needs to use commit end time(state transition time) to filter the commits if using incremental pulling. As w2's state transition time is earlier than the w1's, so w1's data won't be filtered.
   
   This relates to the [HUDI-1623](https://issues.apache.org/jira/browse/HUDI-1623) but not adding end time to the end of each commit, instead use `FileStatus.getModificationTime` to represent the end time.
   
   ### Risk level (write none, low medium or high below)
   
   low
   
   ### Documentation Update
   
   _Describe any necessary documentation update if there is any new feature, config, or user-facing change_
   
   - _The config description must be updated if new configs are added or the default value of the configs are changed_
   - _Any new feature or user-facing change requires updating the Hudi website. Please create a Jira ticket, attach the
     ticket number here and follow the [instruction](https://hudi.apache.org/contribute/developer-setup#website) to make
     changes to the website._
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1432459147

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3d7b6c042316da0f33cb1b343b878b171f18dbdd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187) 
   * f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] xushiyan commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "xushiyan (via GitHub)" <gi...@apache.org>.
xushiyan commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1197622388


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -46,14 +59,36 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
   public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
       .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
 
+  public static final Comparator<HoodieInstant> STATE_TRANSITION_COMPARATOR =
+      Comparator.comparing(HoodieInstant::getStateTransitionTime)
+          .thenComparing(COMPARATOR);
+
+  public static final String EMPTY_FILE_EXTENSION = "";
+
   public static String getComparableAction(String action) {
     return COMPARABLE_ACTIONS.getOrDefault(action, action);
   }
 
+  public static String extractTimestamp(String fileName) throws IllegalArgumentException {
+    Objects.requireNonNull(fileName);

Review Comment:
   arg should be nonnull by convention so this is redundant. we only annotate it if it's nullable



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala:
##########
@@ -51,45 +53,46 @@ class TestStreamingSource extends StreamTest {
     withTempDir { inputDir =>
       val tablePath = s"${inputDir.getCanonicalPath}/test_cow_stream"
       HoodieTableMetaClient.withPropertyBuilder()
-          .setTableType(COPY_ON_WRITE)
-          .setTableName(getTableName(tablePath))
-          .setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue)
+        .setTableType(COPY_ON_WRITE)
+        .setTableName(getTableName(tablePath))
+        .setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue)
         .setPreCombineField("ts")
-          .initTable(spark.sessionState.newHadoopConf(), tablePath)
+        .initTable(spark.sessionState.newHadoopConf(), tablePath)
 
       addData(tablePath, Seq(("1", "a1", "10", "000")))
       val df = spark.readStream
         .format("org.apache.hudi")
+        .option(DataSourceReadOptions.READ_BY_STATE_TRANSITION_TIME.key(), useTransitionTime)

Review Comment:
   this should be parameterized, otherwise, the default commit instant time is not test-covered



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -70,32 +105,38 @@ public enum State {
     NIL
   }
 
-  private State state = State.COMPLETED;
-  private String action;
-  private String timestamp;
+  private final State state;
+  private final String action;
+  private final String timestamp;
+  private final String stateTransitionTime;
 
   /**
    * Load the instant from the meta FileStatus.
    */
   public HoodieInstant(FileStatus fileStatus) {
     // First read the instant timestamp. [==>20170101193025<==].commit
     String fileName = fileStatus.getPath().getName();
-    String fileExtension = getTimelineFileExtension(fileName);
-    timestamp = fileName.replace(fileExtension, "");
-
-    // Next read the action for this marker
-    action = fileExtension.replaceFirst(".", "");
-    if (action.equals("inflight")) {
-      // This is to support backwards compatibility on how in-flight commit files were written
-      // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
-      action = "commit";
-      state = State.INFLIGHT;
-    } else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
-      state = State.INFLIGHT;
-      action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
-    } else if (action.contains(HoodieTimeline.REQUESTED_EXTENSION)) {
-      state = State.REQUESTED;
-      action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, "");
+    Matcher matcher = NAME_FORMAT.matcher(fileName);
+    if (matcher.find()) {
+      timestamp = matcher.group(1);
+      if (matcher.group(2).equals(HoodieTimeline.INFLIGHT_EXTENSION)) {
+        // This is to support backwards compatibility on how in-flight commit files were written
+        // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
+        action = HoodieTimeline.COMMIT_ACTION;
+        state = State.INFLIGHT;
+      } else {
+        action = matcher.group(2).replaceFirst(DELIMITER, StringUtils.EMPTY_STRING);
+        if (matcher.groupCount() == 3 && matcher.group(3) != null) {
+          state = State.valueOf(matcher.group(3).replaceFirst(DELIMITER, StringUtils.EMPTY_STRING).toUpperCase());
+        } else {
+          // Like 20230104152218702.commit
+          state = State.COMPLETED;
+        }
+      }
+      stateTransitionTime =
+          HoodieInstantTimeGenerator.formatDate(new Date(fileStatus.getModificationTime()));
+    } else {
+      throw new IllegalArgumentException(String.format(FILE_NAME_FORMAT_ERROR, fileName));

Review Comment:
   same here. should have a hudi specific exception



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -46,14 +59,36 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
   public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
       .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
 
+  public static final Comparator<HoodieInstant> STATE_TRANSITION_COMPARATOR =
+      Comparator.comparing(HoodieInstant::getStateTransitionTime)
+          .thenComparing(COMPARATOR);
+
+  public static final String EMPTY_FILE_EXTENSION = "";
+
   public static String getComparableAction(String action) {
     return COMPARABLE_ACTIONS.getOrDefault(action, action);
   }
 
+  public static String extractTimestamp(String fileName) throws IllegalArgumentException {
+    Objects.requireNonNull(fileName);
+
+    Matcher matcher = NAME_FORMAT.matcher(fileName);
+    if (matcher.find()) {
+      return matcher.group(1);
+    }
+
+    throw new IllegalArgumentException(String.format(FILE_NAME_FORMAT_ERROR, fileName));

Review Comment:
   should use a hudi specific exception to give more context



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java:
##########
@@ -197,6 +197,13 @@ public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) {
         getInstantsAsStream().filter(s -> HoodieTimeline.isInRange(s.getTimestamp(), startTs, endTs)), details);
   }
 
+  @Override
+  public HoodieDefaultTimeline findInstantsInRangeByStateTransitionTs(String startTs, String endTs) {

Review Comment:
   should just call it `findInstantsInRangeByStateTransitionTime`. let's avoid having acronym in the api



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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1172331658


##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestHoodieInstant extends HoodieCommonTestHarness {
+
+  @Test
+  public void testExtractTimestamp() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+  }
+
+  @Test
+  public void testGetTimelineFileExtension() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals(".inflight", HoodieInstant.getTimelineFileExtension(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals(".commit.request", HoodieInstant.getTimelineFileExtension(fileName));
+  }
+
+  @Test
+  public void testCreateHoodieInstantByFileStatus() throws IOException {
+    try {
+      initMetaClient();
+      HoodieInstant instantRequested =
+          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieInstant instantCommitted =
+          new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+      timeline.createNewInstant(instantRequested);
+      timeline.transitionRequestedToInflight(instantRequested, Option.empty());
+      timeline.saveAsComplete(
+          new HoodieInstant(true, instantRequested.getAction(), instantRequested.getTimestamp()),
+          Option.empty());
+      metaClient.reloadActiveTimeline();

Review Comment:
   changed



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1176198157


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -83,7 +85,7 @@ public static String getTimelineFileExtension(String fileName) {
       return fileName.substring(matcher.group(1).length());
     }
 
-    return "";
+    return StringUtils.EMPTY_STRING;
   }

Review Comment:
   Can you address this comment ?



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1517455625

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e3449eb4ac288eacb5cd60bedf858622f00979bb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494) 
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1432903652

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1198483774


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala:
##########
@@ -51,45 +53,46 @@ class TestStreamingSource extends StreamTest {
     withTempDir { inputDir =>
       val tablePath = s"${inputDir.getCanonicalPath}/test_cow_stream"
       HoodieTableMetaClient.withPropertyBuilder()
-          .setTableType(COPY_ON_WRITE)
-          .setTableName(getTableName(tablePath))
-          .setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue)
+        .setTableType(COPY_ON_WRITE)
+        .setTableName(getTableName(tablePath))
+        .setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue)
         .setPreCombineField("ts")
-          .initTable(spark.sessionState.newHadoopConf(), tablePath)
+        .initTable(spark.sessionState.newHadoopConf(), tablePath)
 
       addData(tablePath, Seq(("1", "a1", "10", "000")))
       val df = spark.readStream
         .format("org.apache.hudi")
+        .option(DataSourceReadOptions.READ_BY_STATE_TRANSITION_TIME.key(), useTransitionTime)

Review Comment:
   By default `useTransitionTime` is false, and this test covers the default commit instant time, while `TestStreamSourceReadByStateTransitionTime` extends this class and override `useTransitionTime` to true.



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


[GitHub] [hudi] boneanxs commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1491187084

   > +1 on this in general. but would this be a format change? This will have backwards compatibility issues.
   
   > there is not storage change (barring archived entry which needs to be discussed), but we add stateTransition time based on fileStatus's last mod time to every HoodieInstant
   
   Yes, there's no format/ storage changes, given the timeline instants are log based, these instants won't change after written, I use `FileStatus.getModificationTime` to represent the state transition time.
   
   This improvement works well inside my company(HDFS based), and I think it still can work under ObjectStorage, since all instants are file based, and Object Stores should show the right timestamp(https://hadoop.apache.org/docs/current//hadoop-project-dist/hadoop-common/FileSystemShell.html#Timestamps)


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1517518528

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e3449eb4ac288eacb5cd60bedf858622f00979bb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494) 
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * ef5fad3612bc910902d52de0296ef4c8852e5319 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1169760159


##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java:
##########
@@ -67,6 +67,13 @@ public class HoodieCommonConfig extends HoodieConfig {
       .defaultValue(true)
       .withDocumentation("Turn on compression for BITCASK disk map used by the External Spillable Map");
 
+  public static final ConfigProperty<Boolean> INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME = ConfigProperty
+      .key("hoodie.incremental.fetch.instant.by.state.transition.time")
+      .defaultValue(false)
+      .sinceVersion("0.13.0")

Review Comment:
   > hoodie.incremental.fetch.instant.by.state.transition.time
   
   hoodie.datasource.read.incremental.complete.time.offset?



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1169751362


##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestHoodieInstant extends HoodieCommonTestHarness {
+
+  @Test
+  public void testExtractTimestamp() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+  }
+
+  @Test
+  public void testGetTimelineFileExtension() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals(".inflight", HoodieInstant.getTimelineFileExtension(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals(".commit.request", HoodieInstant.getTimelineFileExtension(fileName));
+  }
+
+  @Test
+  public void testCreateHoodieInstantByFileStatus() throws IOException {
+    try {
+      initMetaClient();
+      HoodieInstant instantRequested =
+          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieInstant instantCommitted =
+          new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+      timeline.createNewInstant(instantRequested);
+      timeline.transitionRequestedToInflight(instantRequested, Option.empty());
+      timeline.saveAsComplete(
+          new HoodieInstant(true, instantRequested.getAction(), instantRequested.getTimestamp()),
+          Option.empty());
+      metaClient.reloadActiveTimeline();

Review Comment:
   We should not use `stateTransitionTime` as default true, or this could be a breaking change for all the existing jobs.



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1375269362

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3d7b6c042316da0f33cb1b343b878b171f18dbdd UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] xushiyan commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "xushiyan (via GitHub)" <gi...@apache.org>.
xushiyan commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1197614102


##########
hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc:
##########
@@ -128,6 +128,11 @@
             "HoodieIndexCommitMetadata"
          ],
          "default": null
+      },
+      {
+         "name":"stateTransitionTime",
+         "type":["null","string"],
+         "default": null

Review Comment:
   > Actually in our internal version, we change the Call show commits/ archive commits to also show stateTransition time. If this implementation is acceptable, I'll add that here as well.
   
   @boneanxs It's always good to have CLI updated accordingly.



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1516891256

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e3449eb4ac288eacb5cd60bedf858622f00979bb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1174742688


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java:
##########
@@ -179,6 +179,8 @@ public HoodieArchivedTimeline reload() {
   private HoodieInstant readCommit(GenericRecord record, boolean loadDetails) {
     final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString();
     final String action = record.get(ACTION_TYPE_KEY).toString();
+    final String stateTransitionTime = Option.ofNullable(record.get(STATE_TRANSITION_TIME))
+        .map(Object::toString).orElse(null);

Review Comment:
   Can we return `record.get(STATE_TRANSITION_TIME)` directly ?



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


[GitHub] [hudi] danny0405 commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1523156575

   Can you rebase with the latest master and force push again~


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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1153979409


##########
hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc:
##########
@@ -128,6 +128,11 @@
             "HoodieIndexCommitMetadata"
          ],
          "default": null
+      },
+      {
+         "name":"stateTransitionTime",
+         "type":["null","string"],
+         "default": null

Review Comment:
   Saving stateTransition time in archiver can help us to do some troubleshoot work relate to timeline(if we drop modification time after the instant is archived, we can not identify how downstream jobs ingesting these instants).
   
   Actually in our internal version, we change the `Call show commits/ archive commits` to also show stateTransition time. If this implementation is acceptable, I'll add that here as well.



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1153976344


##########
hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc:
##########
@@ -128,6 +128,11 @@
             "HoodieIndexCommitMetadata"
          ],
          "default": null
+      },
+      {
+         "name":"stateTransitionTime",
+         "type":["null","string"],
+         "default": null

Review Comment:
   In most of the cases, there is no need for modification time actually, because we are assured that the archived instant happens before the active timeline instants.
   
   Unless if we want to handle the processing sequence for archived instants, curious what scenario needs such constraint?



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1432794138

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1172399586


##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java:
##########
@@ -72,6 +72,14 @@ public class HoodieCommonConfig extends HoodieConfig {
       .markAdvanced()
       .withDocumentation("Turn on compression for BITCASK disk map used by the External Spillable Map");
 
+  public static final ConfigProperty<Boolean> INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME = ConfigProperty
+      .key("hoodie.incremental.fetch.instant.by.state.transition.time")
+      .defaultValue(false)
+      .sinceVersion("0.14.0")
+      .withDocumentation("For incremental mode, whether to enable to pull commits in range by stateTransitionTime "
+          + "instead of commit timestamp. Please be aware that enabling this will result in"
+          + "`begin.instanttime` and `end.instanttime` using `stateTransitionTime` instead of the instant's commit time.");

Review Comment:
   ```java
   "For incremental mode, whether to enable to pulling commits in range by state transition time(completion time) "
             + "instead of commit time(start time). Please be aware that enabling this will result in"
             + "`begin.instanttime` and `end.instanttime` using `stateTransitionTime` instead of the instant's commit time."
   ```



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadWithStateTransitionTime.scala:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.functional
+
+import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.model.HoodieTableType
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
+import org.apache.hudi.config.HoodieWriteConfig
+import org.apache.hudi.testutils.HoodieSparkClientTestBase
+import org.apache.spark.sql.{SaveMode, SparkSession}
+import org.junit.jupiter.api.{AfterEach, Assertions, BeforeEach}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.EnumSource
+
+import scala.collection.JavaConversions.asScalaBuffer
+
+class TestIncrementalReadWithStateTransitionTime extends HoodieSparkClientTestBase  {
+
+  var spark: SparkSession = null
+
+  val commonOpts = Map(
+    "hoodie.insert.shuffle.parallelism" -> "4",
+    "hoodie.upsert.shuffle.parallelism" -> "4",
+    DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
+    DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
+    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
+    HoodieWriteConfig.TBL_NAME.key -> "hoodie_test",
+    HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS.key -> "1"
+  )
+
+  @BeforeEach
+  override def setUp(): Unit = {
+    setTableName("hoodie_test")
+    initPath()
+    initSparkContexts()
+    spark = sqlContext.sparkSession
+    initTestDataGenerator()
+    initFileSystem()
+  }
+
+  @AfterEach
+  override def tearDown(): Unit = {
+    cleanupSparkContexts()
+    cleanupTestDataGenerator()
+    cleanupFileSystem()
+  }
+
+  @ParameterizedTest
+  @EnumSource(value = classOf[HoodieTableType])
+  def testReadingWithStateTransitionTime(tableType: HoodieTableType): Unit = {
+    val records = recordsToStrings(dataGen.generateInserts("001", 100)).toList
+    val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
+    inputDF.write.format("org.apache.hudi")
+      .options(commonOpts)
+      .option(DataSourceWriteOptions.TABLE_TYPE.key, tableType.name())
+      .option("hoodie.cleaner.commits.retained", "3")
+      .option("hoodie.keep.min.commits", "4")
+      .option("hoodie.keep.max.commits", "5")
+      .option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
+      .mode(SaveMode.Append)
+      .save(basePath)
+
+    val metaClient = HoodieTableMetaClient.builder()
+      .setConf(spark.sparkContext.hadoopConfiguration)
+      .setBasePath(basePath)
+      .setLoadActiveTimelineOnLoad(true)
+      .build()
+
+    val firstInstant = metaClient.getActiveTimeline.filterCompletedInstants().getInstantsOrderedByStateTransitionTs
+      .findFirst().get()
+
+    val result1 = spark.read.format("org.apache.hudi")
+      .option(DataSourceReadOptions.QUERY_TYPE.key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
+      .option(DataSourceReadOptions.BEGIN_INSTANTTIME.key(), "000")
+      .option(DataSourceReadOptions.INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME.key(), "true")
+      .option(DataSourceReadOptions.END_INSTANTTIME.key(), firstInstant.getTimestamp)
+      .load(basePath)

Review Comment:
   We also need to test the out of order scenarios, where the code would miss data without this change.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -226,6 +275,8 @@ public int compareTo(HoodieInstant o) {
 
   @Override
   public String toString() {
-    return "[" + ((isInflight() || isRequested()) ? "==>" : "") + timestamp + "__" + action + "__" + state + "]";
+    return "[" + ((isInflight() || isRequested()) ? "==>" : "")
+        + timestamp + "__" + action + "__" + state
+        + (StringUtils.isNullOrEmpty(stateTransitionTime) ? "" : ("__" + stateTransitionTime)) + "]";
   }

Review Comment:
   Guess the check `StringUtils.isNullOrEmpty` is unnecessary because `stateTransitionTime` is by defaut an empty string.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -70,32 +100,38 @@ public enum State {
     NIL
   }
 
-  private State state = State.COMPLETED;
-  private String action;
-  private String timestamp;
+  private final State state;
+  private final String action;
+  private final String timestamp;
+  private final String stateTransitionTime;
 
   /**
    * Load the instant from the meta FileStatus.
    */
   public HoodieInstant(FileStatus fileStatus) {
     // First read the instant timestamp. [==>20170101193025<==].commit
     String fileName = fileStatus.getPath().getName();
-    String fileExtension = getTimelineFileExtension(fileName);
-    timestamp = fileName.replace(fileExtension, "");
-
-    // Next read the action for this marker
-    action = fileExtension.replaceFirst(".", "");
-    if (action.equals("inflight")) {
-      // This is to support backwards compatibility on how in-flight commit files were written
-      // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
-      action = "commit";
-      state = State.INFLIGHT;
-    } else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
-      state = State.INFLIGHT;
-      action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
-    } else if (action.contains(HoodieTimeline.REQUESTED_EXTENSION)) {
-      state = State.REQUESTED;
-      action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, "");
+    Matcher matcher = NAME_FORMAT.matcher(fileName);
+    if (matcher.find()) {
+      timestamp = matcher.group(1);
+      if (matcher.group(2).equals(HoodieTimeline.INFLIGHT_EXTENSION)) {
+        // This is to support backwards compatibility on how in-flight commit files were written
+        // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
+        action = "commit";
+        state = State.INFLIGHT;
+      } else {
+        action = matcher.group(2).replaceFirst(".", "");
+        if (matcher.groupCount() == 3 && matcher.group(3) != null) {

Review Comment:
   Can we replace all these hard code string constants with static members? i.e. `"."` and `""`.



##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java:
##########
@@ -72,6 +72,14 @@ public class HoodieCommonConfig extends HoodieConfig {
       .markAdvanced()
       .withDocumentation("Turn on compression for BITCASK disk map used by the External Spillable Map");
 
+  public static final ConfigProperty<Boolean> INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME = ConfigProperty
+      .key("hoodie.incremental.fetch.instant.by.state.transition.time")
+      .defaultValue(false)

Review Comment:
   `hoodie.incremental.fetch.instant.by.state.transition.time` seem not a good name, does
   `hoodie.datasource.read.by.state.transition.time` make sense to you?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java:
##########
@@ -191,7 +192,8 @@ private HoodieInstant readCommit(GenericRecord record, boolean loadDetails) {
         return null;
       });
     }
-    return new HoodieInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action, instantTime);
+    return new HoodieInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action,
+        instantTime, Option.ofNullable(record.get(STATE_TRANSITION_TIME)).orElse("").toString());
   }

Review Comment:
   Use static memner variable instead of hard code string.



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStreamingSource.scala:
##########
@@ -48,159 +48,176 @@ class TestStreamingSource extends StreamTest {
   }
 
   test("test cow stream source") {
-    withTempDir { inputDir =>
-      val tablePath = s"${inputDir.getCanonicalPath}/test_cow_stream"
-      HoodieTableMetaClient.withPropertyBuilder()

Review Comment:
   Can we just override the whole class and make the flag `DataSourceReadOptions.INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME` overridable?



##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestHoodieInstant extends HoodieCommonTestHarness {
+
+  @Test
+  public void testExtractTimestamp() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+  }
+
+  @Test
+  public void testGetTimelineFileExtension() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals(".inflight", HoodieInstant.getTimelineFileExtension(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals(".commit.request", HoodieInstant.getTimelineFileExtension(fileName));
+  }
+
+  @Test
+  public void testCreateHoodieInstantByFileStatus() throws IOException {
+    try {
+      initMetaClient();
+      HoodieInstant instantRequested =
+          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieInstant instantCommitted =
+          new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+      timeline.createNewInstant(instantRequested);
+      timeline.transitionRequestedToInflight(instantRequested, Option.empty());
+      timeline.saveAsComplete(
+          new HoodieInstant(true, instantRequested.getAction(), instantRequested.getTimestamp()),
+          Option.empty());
+      metaClient.reloadActiveTimeline();

Review Comment:
   Can we also add a test when the file is moved, the state transition time also changes.



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1522936046

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522638580",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * bfb9fbbed9a2423ba1781962cea8ccc277a84880 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1184488167


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala:
##########
@@ -163,10 +178,7 @@ class HoodieStreamSource(
     startOffset match {
       case INIT_OFFSET => startOffset.commitTime
       case HoodieSourceOffset(commitTime) =>
-        val time = HoodieActiveTimeline.parseDateFromInstantTime(commitTime).getTime

Review Comment:
   I think this is unnecessary, since we filter all instants in `IncrementalRelation` and `MergeOnReadIncrementalRelation` by methods `findInstantsInRangeByStateTransitionTs` and `findInstantsInRange`, these two methods filter instants after startTs and before or on endTs.
   
   ```java
   /**
      * Create a new Timeline with instants after startTs and before or on endTs.
      */
     HoodieTimeline findInstantsInRange(String startTs, String endTs);
   
     /**`
      * Create a new Timeline with instants after startTs and before or on endTs
      * by state transition timestamp of actions.
      */
     HoodieTimeline findInstantsInRangeByStateTransitionTs(String startTs, String endTs);
   ```



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


[GitHub] [hudi] nsivabalan commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1182021390


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSource.scala:
##########
@@ -163,10 +178,7 @@ class HoodieStreamSource(
     startOffset match {
       case INIT_OFFSET => startOffset.commitTime
       case HoodieSourceOffset(commitTime) =>
-        val time = HoodieActiveTimeline.parseDateFromInstantTime(commitTime).getTime

Review Comment:
   why this was removed? was there any bug? 



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala:
##########
@@ -205,6 +218,9 @@ class IncrementalRelation(val sqlContext: SQLContext,
       val endInstantArchived = commitTimeline.isBeforeTimelineStarts(endInstantTime)
 
       val scanDf = if (fallbackToFullTableScan && (startInstantArchived || endInstantArchived)) {
+        if (useStateTransitionTime) {
+          throw new HoodieException("Cannot use stateTransitionTime while enables full table scan")

Review Comment:
   I am not sure if we can throw and move on. 
   I see this similar to how someone is doing an incremental query as of now. 
   what we are handling here is: 
   if downstream is lagging /down for sometime and then it resumes to consume, and if cleaner has cleaned up the instants to consume, we do a fallback here. 
   
   when migrating to stateTransitionTime, we should find a way to support the same. 



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


[GitHub] [hudi] xushiyan commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "xushiyan (via GitHub)" <gi...@apache.org>.
xushiyan commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1552868836

   in summary there are a few follow-up problems
   - support meta sync properly in the same mult-writer scenario wrt using transition time
   - verify the approach with time travel query when using state transition time
   - identify issue wrt losing precision in the OS last mod time


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


[GitHub] [hudi] nsivabalan commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1153275366


##########
hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc:
##########
@@ -128,6 +128,11 @@
             "HoodieIndexCommitMetadata"
          ],
          "default": null
+      },
+      {
+         "name":"stateTransitionTime",
+         "type":["null","string"],
+         "default": null

Review Comment:
   its a new filed and default is null. so, it is backwards compatible. But curious to know why do we need this in archival entry. what benefit we get by adding this entry to archived entry? can we just ignore it.



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1521388077

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * ef5fad3612bc910902d52de0296ef4c8852e5319 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523) 
   * 8d611417db0e3eb818bec1fd63a37dec5aa4b3ad UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1522667692

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522638580",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * bfb9fbbed9a2423ba1781962cea8ccc277a84880 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1174743866


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala:
##########
@@ -185,7 +185,11 @@ trait HoodieIncrementalRelationTrait extends HoodieBaseRelation {
   protected lazy val incrementalSpanRecordFilters: Seq[Filter] = {
     val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
 
-    val largerThanFilter = GreaterThan(HoodieRecord.COMMIT_TIME_METADATA_FIELD, startTimestamp)
+    val largerThanFilter = if (startInstantArchived) {
+      GreaterThan(HoodieRecord.COMMIT_TIME_METADATA_FIELD, startTimestamp)

Review Comment:
   Is this change related to transition time?



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


[GitHub] [hudi] danny0405 commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1539304963

   Nice findings, we already format the instant time to milli-seconds, so yeah, when the precision is lost, the sequence may break.


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1510790494

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 693fe2d347bf62a4340ad5d397d29caf092a5cb9 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267) 
   * d5f7e0f51ce768b34aa01ff970da9c187c4f8c16 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1169762787


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -70,32 +100,38 @@ public enum State {
     NIL
   }
 
-  private State state = State.COMPLETED;
-  private String action;
-  private String timestamp;
+  private final State state;
+  private final String action;
+  private final String timestamp;
+  private final String stateTransitionTime;

Review Comment:
   rename `stateTransitionTime` to `complete` time ?



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


[GitHub] [hudi] boneanxs commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1552878943

   @xushiyan Will address comments in a follow up pr.


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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1172546206


##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestHoodieInstant extends HoodieCommonTestHarness {
+
+  @Test
+  public void testExtractTimestamp() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+  }
+
+  @Test
+  public void testGetTimelineFileExtension() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals(".inflight", HoodieInstant.getTimelineFileExtension(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals(".commit.request", HoodieInstant.getTimelineFileExtension(fileName));
+  }
+
+  @Test
+  public void testCreateHoodieInstantByFileStatus() throws IOException {
+    try {
+      initMetaClient();
+      HoodieInstant instantRequested =
+          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieInstant instantCommitted =
+          new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+      timeline.createNewInstant(instantRequested);
+      timeline.transitionRequestedToInflight(instantRequested, Option.empty());
+      timeline.saveAsComplete(
+          new HoodieInstant(true, instantRequested.getAction(), instantRequested.getTimestamp()),
+          Option.empty());
+      metaClient.reloadActiveTimeline();

Review Comment:
   Do you mean the instant file is moved?



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


[GitHub] [hudi] danny0405 commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1525213139

   The test has passed: https://dev.azure.com/apache-hudi-ci-org/apache-hudi-ci/_build/results?buildId=16697&view=results


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1524604127

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522638580",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16680",
       "triggerID" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "90a156a656fae44517c297c70662d6723979d3ea",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16697",
       "triggerID" : "90a156a656fae44517c297c70662d6723979d3ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * c926ead8318c18cf0ead9122c9292dd22036d05c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16680) 
   * 90a156a656fae44517c297c70662d6723979d3ea Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16697) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1107973837


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -46,14 +55,35 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
   public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
       .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
 
+  public static final Comparator<HoodieInstant> STATE_TRANSITION_COMPARATOR =
+      Comparator.comparing(HoodieInstant::getStateTransitionTime)
+          .thenComparing(HoodieInstant::getTimestamp)
+          .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
+
   public static String getComparableAction(String action) {
     return COMPARABLE_ACTIONS.getOrDefault(action, action);
   }
 
+  public static String extractTimestamp(String fileName) throws IllegalArgumentException {
+    Objects.requireNonNull(fileName);
+
+    Matcher matcher = NAME_FORMAT.matcher(fileName);
+    if (matcher.find()) {
+      return matcher.group(1);
+    }
+
+    throw new IllegalArgumentException(String.format(FILE_NAME_FORMAT_ERROR, fileName));
+  }
+
   public static String getTimelineFileExtension(String fileName) {
     Objects.requireNonNull(fileName);
-    int dotIndex = fileName.indexOf('.');
-    return dotIndex == -1 ? "" : fileName.substring(dotIndex);
+
+    Matcher matcher = NAME_FORMAT.matcher(fileName);

Review Comment:
   Here I just want to make all extraction logic follow the `NAME_FORMAT`, sry I didn't thought too much about the performance at the first time. Looks it could take more time than before.
   
   I run this method 1000 times, it will take 7-10 ms with the new codes, while for the old it takes 5 ms
   



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


[GitHub] [hudi] boneanxs commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1432896765

   @hudi-bot run azure


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1434933369

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 693fe2d347bf62a4340ad5d397d29caf092a5cb9 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1434058610

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241) 
   * 693fe2d347bf62a4340ad5d397d29caf092a5cb9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by GitBox <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1065348765


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -46,14 +55,35 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
   public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
       .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
 
+  public static final Comparator<HoodieInstant> STATE_TRANSITION_COMPARATOR =
+      Comparator.comparing(HoodieInstant::getStateTransitionTime)
+          .thenComparing(HoodieInstant::getTimestamp)
+          .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
+
   public static String getComparableAction(String action) {
     return COMPARABLE_ACTIONS.getOrDefault(action, action);
   }
 
+  public static String extractTimestamp(String fileName) throws IllegalArgumentException {
+    Objects.requireNonNull(fileName);
+
+    Matcher matcher = NAME_FORMAT.matcher(fileName);
+    if (matcher.find()) {
+      return matcher.group(1);
+    }
+
+    throw new IllegalArgumentException(String.format(FILE_NAME_FORMAT_ERROR, fileName));
+  }
+
   public static String getTimelineFileExtension(String fileName) {
     Objects.requireNonNull(fileName);
-    int dotIndex = fileName.indexOf('.');
-    return dotIndex == -1 ? "" : fileName.substring(dotIndex);
+
+    Matcher matcher = NAME_FORMAT.matcher(fileName);

Review Comment:
   Is the regex expr match efficient enough here ?



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1151331104


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -46,14 +55,35 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
   public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
       .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
 
+  public static final Comparator<HoodieInstant> STATE_TRANSITION_COMPARATOR =
+      Comparator.comparing(HoodieInstant::getStateTransitionTime)

Review Comment:
   We can just use `COMPARATOR` here.



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


[GitHub] [hudi] codope commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "codope (via GitHub)" <gi...@apache.org>.
codope commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1166270869


##########
hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc:
##########
@@ -128,6 +128,11 @@
             "HoodieIndexCommitMetadata"
          ],
          "default": null
+      },
+      {
+         "name":"stateTransitionTime",
+         "type":["null","string"],
+         "default": null

Review Comment:
   +1 we should save it in the archived metadata. I can see other potential use cases when there can be holes in the timeline after we allow archival beyond savepoint. 



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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1168236489


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala:
##########
@@ -205,6 +218,9 @@ class IncrementalRelation(val sqlContext: SQLContext,
       val endInstantArchived = commitTimeline.isBeforeTimelineStarts(endInstantTime)
 
       val scanDf = if (fallbackToFullTableScan && (startInstantArchived || endInstantArchived)) {
+        if (useStateTransitionTime) {
+          throw new HoodieException("Cannot use stateTransitionTime while enables full table scan")

Review Comment:
   If enables `useStateTransitionTime`, means they provides state transition time for `startInstantTime` and `endInstantTime`, and they will be used to compare with `_hoodie_commit_time`, then the result is not accurate. So here we better then error out, or force setting `useStateTransitionTime` back to false?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -46,14 +55,35 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
   public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
       .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
 
+  public static final Comparator<HoodieInstant> STATE_TRANSITION_COMPARATOR =
+      Comparator.comparing(HoodieInstant::getStateTransitionTime)

Review Comment:
   We need to use `STATE_TRANSITION_COMPARATOR` in `Timeline.getInstantsOrderedByStateTransitionTs`, which should sort `stateTransitionTime` first. So here cannot use `COMPARATOR` directly



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala:
##########
@@ -82,9 +87,17 @@ class IncrementalRelation(val sqlContext: SQLContext,
 
   private val lastInstant = commitTimeline.lastInstant().get()
 
-  private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
-    optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key),
-    optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key(), lastInstant.getTimestamp))
+  private val commitsTimelineToReturn = {
+    if (useStateTransitionTime) {

Review Comment:
   If I understand correctly, we only use begin and end instant time(which is `stateTransitionTime` if enabled) to filter instants, and still use instants' commit time to compare with `_hoodie_commit_time`



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -70,32 +100,38 @@ public enum State {
     NIL
   }
 
-  private State state = State.COMPLETED;
-  private String action;
-  private String timestamp;
+  private final State state;
+  private final String action;
+  private final String timestamp;
+  private final String stateTransitionTime;
 
   /**
    * Load the instant from the meta FileStatus.
    */
   public HoodieInstant(FileStatus fileStatus) {
     // First read the instant timestamp. [==>20170101193025<==].commit
     String fileName = fileStatus.getPath().getName();
-    String fileExtension = getTimelineFileExtension(fileName);
-    timestamp = fileName.replace(fileExtension, "");
-
-    // Next read the action for this marker
-    action = fileExtension.replaceFirst(".", "");
-    if (action.equals("inflight")) {
-      // This is to support backwards compatibility on how in-flight commit files were written
-      // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
-      action = "commit";
-      state = State.INFLIGHT;
-    } else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
-      state = State.INFLIGHT;
-      action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
-    } else if (action.contains(HoodieTimeline.REQUESTED_EXTENSION)) {
-      state = State.REQUESTED;
-      action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, "");
+    Matcher matcher = NAME_FORMAT.matcher(fileName);
+    if (matcher.find()) {
+      timestamp = matcher.group(1);
+      if (matcher.group(2).equals(HoodieTimeline.INFLIGHT_EXTENSION)) {
+        // This is to support backwards compatibility on how in-flight commit files were written
+        // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
+        action = "commit";
+        state = State.INFLIGHT;
+      } else {
+        action = matcher.group(2).replaceFirst(".", "");
+        if (matcher.groupCount() == 3 && matcher.group(3) != null) {
+          state = State.valueOf(matcher.group(3).replaceFirst(".", "").toUpperCase());
+        } else {
+          // Like 20230104152218702.commit
+          state = State.COMPLETED;
+        }
+      }
+      stateTransitionTime =

Review Comment:
   Thank you, @vinothchandar , for bringing up this important question. We apologize for not considering this aspect earlier. Since the timeline of the new migrated table could be in a random order if still based on `modificationTime`, it can be challenging for downstream pipelines to recover from a new specified time if it's from the old location but not read yet.
   
   So to migrate a table (let's call it A), I think we can follow these steps:
   
   1. Stop ingesting new data into table A.
   2. Wait for all downstream jobs to finish their incremental jobs from table A.
   3. Migrate table A to a new location.
   4. Delete checkpoint information for all downstream jobs and specify `hoodie.datasource.streaming.startOffset` to the time when the migration is completed. Then restart the jobs.



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1516052481

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d5f7e0f51ce768b34aa01ff970da9c187c4f8c16 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388) 
   * e3449eb4ac288eacb5cd60bedf858622f00979bb UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1516065143

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d5f7e0f51ce768b34aa01ff970da9c187c4f8c16 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388) 
   * e3449eb4ac288eacb5cd60bedf858622f00979bb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1517506427

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e3449eb4ac288eacb5cd60bedf858622f00979bb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494) 
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * ef5fad3612bc910902d52de0296ef4c8852e5319 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] boneanxs commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1522638580

   @hudi-bot run azure


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1523263066

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522638580",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * bfb9fbbed9a2423ba1781962cea8ccc277a84880 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660) 
   * c926ead8318c18cf0ead9122c9292dd22036d05c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1522990780

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522638580",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * bfb9fbbed9a2423ba1781962cea8ccc277a84880 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 merged pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 merged PR #7627:
URL: https://github.com/apache/hudi/pull/7627


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1518412070

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * ef5fad3612bc910902d52de0296ef4c8852e5319 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1153979409


##########
hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc:
##########
@@ -128,6 +128,11 @@
             "HoodieIndexCommitMetadata"
          ],
          "default": null
+      },
+      {
+         "name":"stateTransitionTime",
+         "type":["null","string"],
+         "default": null

Review Comment:
   Saving stateTransition time in archiver can help us 1) consistent with the instants are not archived. 2) to do some troubleshoot work relate to timeline(if we drop modification time after the instant is archived, we can not identify how downstream jobs ingest these instants).
   
   Actually in our internal version, we change the `Call show commits/ archive commits` to also show stateTransition time. If this implementation is acceptable, I'll add that here as well.



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1375274888

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3d7b6c042316da0f33cb1b343b878b171f18dbdd Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1165030487


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -70,32 +100,38 @@ public enum State {
     NIL
   }
 
-  private State state = State.COMPLETED;
-  private String action;
-  private String timestamp;
+  private final State state;
+  private final String action;
+  private final String timestamp;
+  private final String stateTransitionTime;
 
   /**
    * Load the instant from the meta FileStatus.
    */
   public HoodieInstant(FileStatus fileStatus) {
     // First read the instant timestamp. [==>20170101193025<==].commit
     String fileName = fileStatus.getPath().getName();
-    String fileExtension = getTimelineFileExtension(fileName);
-    timestamp = fileName.replace(fileExtension, "");
-
-    // Next read the action for this marker
-    action = fileExtension.replaceFirst(".", "");
-    if (action.equals("inflight")) {
-      // This is to support backwards compatibility on how in-flight commit files were written
-      // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
-      action = "commit";
-      state = State.INFLIGHT;
-    } else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
-      state = State.INFLIGHT;
-      action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
-    } else if (action.contains(HoodieTimeline.REQUESTED_EXTENSION)) {
-      state = State.REQUESTED;
-      action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, "");
+    Matcher matcher = NAME_FORMAT.matcher(fileName);
+    if (matcher.find()) {
+      timestamp = matcher.group(1);
+      if (matcher.group(2).equals(HoodieTimeline.INFLIGHT_EXTENSION)) {
+        // This is to support backwards compatibility on how in-flight commit files were written
+        // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
+        action = "commit";
+        state = State.INFLIGHT;
+      } else {
+        action = matcher.group(2).replaceFirst(".", "");
+        if (matcher.groupCount() == 3 && matcher.group(3) != null) {
+          state = State.valueOf(matcher.group(3).replaceFirst(".", "").toUpperCase());
+        } else {
+          // Like 20230104152218702.commit
+          state = State.COMPLETED;
+        }
+      }
+      stateTransitionTime =

Review Comment:
   It is acceptable, because the table path should have changed, it is a fresh new table, the downstream pipelines should have some awareness.



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1168233922


##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java:
##########
@@ -67,6 +67,13 @@ public class HoodieCommonConfig extends HoodieConfig {
       .defaultValue(true)
       .withDocumentation("Turn on compression for BITCASK disk map used by the External Spillable Map");
 
+  public static final ConfigProperty<Boolean> INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME = ConfigProperty
+      .key("hoodie.incremental.fetch.instant.by.state.transition.time")
+      .defaultValue(false)
+      .sinceVersion("0.13.0")

Review Comment:
   need to fix the since version to 0.14.0



##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestHoodieInstant extends HoodieCommonTestHarness {
+
+  @Test
+  public void testExtractTimestamp() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+  }
+
+  @Test
+  public void testGetTimelineFileExtension() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals(".inflight", HoodieInstant.getTimelineFileExtension(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals(".commit.request", HoodieInstant.getTimelineFileExtension(fileName));
+  }
+
+  @Test
+  public void testCreateHoodieInstantByFileStatus() throws IOException {
+    try {
+      initMetaClient();
+      HoodieInstant instantRequested =
+          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieInstant instantCommitted =
+          new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+      timeline.createNewInstant(instantRequested);
+      timeline.transitionRequestedToInflight(instantRequested, Option.empty());
+      timeline.saveAsComplete(
+          new HoodieInstant(true, instantRequested.getAction(), instantRequested.getTimestamp()),
+          Option.empty());
+      metaClient.reloadActiveTimeline();

Review Comment:
   We also need some tests for incremental data source.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala:
##########
@@ -133,8 +133,16 @@ trait HoodieIncrementalRelationTrait extends HoodieBaseRelation {
   // Validate this Incremental implementation is properly configured
   validate()
 
+  private val useStateTransitionTime = optParams.get(DataSourceReadOptions.INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME.key)
+    .map(_.toBoolean)
+    .getOrElse(DataSourceReadOptions.INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME.defaultValue)
+
   protected def startTimestamp: String = optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key)
-  protected def endTimestamp: String = optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key, super.timeline.lastInstant().get.getTimestamp)
+  protected def endTimestamp: String = if (useStateTransitionTime) {
+    optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key, super.timeline.lastInstant().get.getStateTransitionTime)
+  } else {

Review Comment:
   We need some clarification for the semantics change of the two options when `useStateTransitionTime` is set as true.We should add some clarification for the new sementics of `BEGIN_INSTANTTIME` and `END_INSTANTTIME`.



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala:
##########
@@ -82,9 +87,17 @@ class IncrementalRelation(val sqlContext: SQLContext,
 
   private val lastInstant = commitTimeline.lastInstant().get()
 
-  private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
-    optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key),
-    optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key(), lastInstant.getTimestamp))
+  private val commitsTimelineToReturn = {
+    if (useStateTransitionTime) {

Review Comment:
   Guess this could find out the out of order commits like compaction or clustering, but the assumption for `BEGIN_INSTANTTIME` and `END_INSTANTTIME` should be different, once `useStateTransitionTime` is true, user actually should uses the real transition time instead of the instant time to filter the instants.



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1174745393


##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestHoodieInstant extends HoodieCommonTestHarness {
+
+  @Test
+  public void testExtractTimestamp() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+  }
+
+  @Test
+  public void testGetTimelineFileExtension() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals(".inflight", HoodieInstant.getTimelineFileExtension(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals(".commit.request", HoodieInstant.getTimelineFileExtension(fileName));
+  }
+
+  @Test
+  public void testCreateHoodieInstantByFileStatus() throws IOException {
+    try {
+      initMetaClient();
+      HoodieInstant instantRequested =
+          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieInstant instantCommitted =
+          new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+      timeline.createNewInstant(instantRequested);
+      timeline.transitionRequestedToInflight(instantRequested, Option.empty());
+      timeline.saveAsComplete(
+          new HoodieInstant(true, instantRequested.getAction(), instantRequested.getTimestamp()),
+          Option.empty());
+      metaClient.reloadActiveTimeline();

Review Comment:
   Yes. The test expect to fail in such use case.



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1174743356


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -83,7 +85,7 @@ public static String getTimelineFileExtension(String fileName) {
       return fileName.substring(matcher.group(1).length());
     }
 
-    return "";
+    return StringUtils.EMPTY_STRING;
   }

Review Comment:
   We better define a custom constant here named: `EMPTY_INSTANT` instead of using a general constant.



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


[GitHub] [hudi] nsivabalan commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1530770277

   How does this patch change the "timestamp as of" query? is there any follow up required? 
   when we make this default, we should also make the stateTransition time as the default for "timestamp as of" or time travel query 


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1522155394

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * bfb9fbbed9a2423ba1781962cea8ccc277a84880 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1521558550

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * ef5fad3612bc910902d52de0296ef4c8852e5319 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523) 
   * 8d611417db0e3eb818bec1fd63a37dec5aa4b3ad Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638) 
   * bfb9fbbed9a2423ba1781962cea8ccc277a84880 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1523274852

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522638580",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16680",
       "triggerID" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * bfb9fbbed9a2423ba1781962cea8ccc277a84880 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660) 
   * c926ead8318c18cf0ead9122c9292dd22036d05c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16680) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1524590779

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522638580",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16680",
       "triggerID" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "90a156a656fae44517c297c70662d6723979d3ea",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "90a156a656fae44517c297c70662d6723979d3ea",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * c926ead8318c18cf0ead9122c9292dd22036d05c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16680) 
   * 90a156a656fae44517c297c70662d6723979d3ea UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] vinothchandar commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1489294348

   +1 on this in general. but would this be a format change?


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


[GitHub] [hudi] Zouxxyy commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "Zouxxyy (via GitHub)" <gi...@apache.org>.
Zouxxyy commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1598037979

   > in Hudi release 1.0, we are planning to remove the archival timeline and there is only one timeline which is always active
   
   I noticed that there is an infinite timeline plan in 1.0. Does it mean this? Is there a corresponding RFC now?


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


[GitHub] [hudi] boneanxs commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1445592689

   @xushiyan @danny0405 @yihua @nsivabalan Hi, could you please take a look?


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


[GitHub] [hudi] stream2000 commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1552359857

    > But still, we need to solve the archiving issue.
   
   Yes, I'm trying to leverage state transient time to do the hive sync and am facing the archiving issue. When syncing partitions to hive, it's hard to know whether there are instants with larger state transient time in the archive timeline so we need to load the whole archive timeline to get instants with larger state transient time (which is in fact unacceptable for a large archive timeline) 
   
   We need a fast path to know the max state transient time in the archive timeline (just like a checkpoint idx? ). When `lastCommitTime` synced to hive is greater than the max state transient time in the archive timeline, we don't need to load the archive timeline at all. 
   
   We can implement this by adding a extra flag in some way or just simply comparing the file modified time of the last archived log. Also, to accelerate filtering out instants with greater state transient time in archive timeline we can also compare the modification time of the archived log. 
   
   What do you think? 


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


[GitHub] [hudi] danny0405 commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1598059433

   In 1.0, we plan to migrate a more elegent impl for the timeline instant completion time, there is a PR: https://github.com/apache/hudi/pull/2701, no RFC yet, but I think we should fire a RFC first because we also have other idears to improve the archived timeline.


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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1172539567


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -226,6 +275,8 @@ public int compareTo(HoodieInstant o) {
 
   @Override
   public String toString() {
-    return "[" + ((isInflight() || isRequested()) ? "==>" : "") + timestamp + "__" + action + "__" + state + "]";
+    return "[" + ((isInflight() || isRequested()) ? "==>" : "")
+        + timestamp + "__" + action + "__" + state
+        + (StringUtils.isNullOrEmpty(stateTransitionTime) ? "" : ("__" + stateTransitionTime)) + "]";
   }

Review Comment:
   I changed `stateTransitionTime` default to null to keep the consistent behavior in `HoodieArchivedMetaEntry`



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1169761970


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -46,14 +55,35 @@ public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
   public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
       .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
 
+  public static final Comparator<HoodieInstant> STATE_TRANSITION_COMPARATOR =
+      Comparator.comparing(HoodieInstant::getStateTransitionTime)

Review Comment:
   I mean for `STATE_TRANSITION_COMPARATOR ` construction, `COMPARATOR` can be reused to save the redundant code.



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1510797365

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 693fe2d347bf62a4340ad5d397d29caf092a5cb9 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267) 
   * d5f7e0f51ce768b34aa01ff970da9c187c4f8c16 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] vinothchandar commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1164862966


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala:
##########
@@ -205,6 +218,9 @@ class IncrementalRelation(val sqlContext: SQLContext,
       val endInstantArchived = commitTimeline.isBeforeTimelineStarts(endInstantTime)
 
       val scanDf = if (fallbackToFullTableScan && (startInstantArchived || endInstantArchived)) {
+        if (useStateTransitionTime) {
+          throw new HoodieException("Cannot use stateTransitionTime while enables full table scan")

Review Comment:
   throw error or could we log a warn and move on?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -70,32 +100,38 @@ public enum State {
     NIL
   }
 
-  private State state = State.COMPLETED;
-  private String action;
-  private String timestamp;
+  private final State state;
+  private final String action;
+  private final String timestamp;
+  private final String stateTransitionTime;
 
   /**
    * Load the instant from the meta FileStatus.
    */
   public HoodieInstant(FileStatus fileStatus) {
     // First read the instant timestamp. [==>20170101193025<==].commit
     String fileName = fileStatus.getPath().getName();
-    String fileExtension = getTimelineFileExtension(fileName);
-    timestamp = fileName.replace(fileExtension, "");
-
-    // Next read the action for this marker
-    action = fileExtension.replaceFirst(".", "");
-    if (action.equals("inflight")) {
-      // This is to support backwards compatibility on how in-flight commit files were written
-      // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
-      action = "commit";
-      state = State.INFLIGHT;
-    } else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
-      state = State.INFLIGHT;
-      action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
-    } else if (action.contains(HoodieTimeline.REQUESTED_EXTENSION)) {
-      state = State.REQUESTED;
-      action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, "");
+    Matcher matcher = NAME_FORMAT.matcher(fileName);
+    if (matcher.find()) {
+      timestamp = matcher.group(1);
+      if (matcher.group(2).equals(HoodieTimeline.INFLIGHT_EXTENSION)) {
+        // This is to support backwards compatibility on how in-flight commit files were written
+        // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
+        action = "commit";
+        state = State.INFLIGHT;
+      } else {
+        action = matcher.group(2).replaceFirst(".", "");
+        if (matcher.groupCount() == 3 && matcher.group(3) != null) {
+          state = State.valueOf(matcher.group(3).replaceFirst(".", "").toUpperCase());
+        } else {
+          // Like 20230104152218702.commit
+          state = State.COMPLETED;
+        }
+      }
+      stateTransitionTime =

Review Comment:
   I am worried that the modification time will change when you say copy over the table to a new location .. Then all the incremental queries need to be restarted?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala:
##########
@@ -82,9 +87,17 @@ class IncrementalRelation(val sqlContext: SQLContext,
 
   private val lastInstant = commitTimeline.lastInstant().get()
 
-  private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
-    optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key),
-    optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key(), lastInstant.getTimestamp))
+  private val commitsTimelineToReturn = {
+    if (useStateTransitionTime) {

Review Comment:
   how does this affect the timetravel query? (which basically hits the same code path as inc query with same begin and end instant times)



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1176198903


##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+

Review Comment:
   Fix import sequence checkstyle.



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


[GitHub] [hudi] danny0405 commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1521408752

   [5517.patch.zip](https://github.com/apache/hudi/files/11320540/5517.patch.zip)
   Thanks, I have created a patch to fix the import sequence, overall looks good now ~


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


[GitHub] [hudi] boneanxs commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1522926916

   @hudi-bot run azure


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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1153979409


##########
hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc:
##########
@@ -128,6 +128,11 @@
             "HoodieIndexCommitMetadata"
          ],
          "default": null
+      },
+      {
+         "name":"stateTransitionTime",
+         "type":["null","string"],
+         "default": null

Review Comment:
   Saving stateTransition time in archiver can help us to do some troubleshoot work relate to timeline(if we drop modification time after the instant is archived, we can not identify how downstream jobs ingest these instants).
   
   Actually in our internal version, we change the `Call show commits/ archive commits` to also show stateTransition time. If this implementation is acceptable, I'll add that here as well.



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1524003196

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522638580",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16672",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16660",
       "triggerID" : "1522926916",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16680",
       "triggerID" : "c926ead8318c18cf0ead9122c9292dd22036d05c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * c926ead8318c18cf0ead9122c9292dd22036d05c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16680) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1433686900

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1375598505

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3d7b6c042316da0f33cb1b343b878b171f18dbdd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1521399511

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * ef5fad3612bc910902d52de0296ef4c8852e5319 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523) 
   * 8d611417db0e3eb818bec1fd63a37dec5aa4b3ad Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1176163711


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala:
##########
@@ -185,7 +185,11 @@ trait HoodieIncrementalRelationTrait extends HoodieBaseRelation {
   protected lazy val incrementalSpanRecordFilters: Seq[Filter] = {
     val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
 
-    val largerThanFilter = GreaterThan(HoodieRecord.COMMIT_TIME_METADATA_FIELD, startTimestamp)
+    val largerThanFilter = if (startInstantArchived) {
+      GreaterThan(HoodieRecord.COMMIT_TIME_METADATA_FIELD, startTimestamp)

Review Comment:
   Here I just want to keep the consistent behavior with `IncrementalRelation`, In `IncrementalRelation`, if `fallbackToFullTableScan` enabled, will use `> startTimestamp`, otherwise it will use `>= commitsToReturn.head`



##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestHoodieInstant extends HoodieCommonTestHarness {
+
+  @Test
+  public void testExtractTimestamp() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+  }
+
+  @Test
+  public void testGetTimelineFileExtension() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals(".inflight", HoodieInstant.getTimelineFileExtension(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals(".commit.request", HoodieInstant.getTimelineFileExtension(fileName));
+  }
+
+  @Test
+  public void testCreateHoodieInstantByFileStatus() throws IOException {
+    try {
+      initMetaClient();
+      HoodieInstant instantRequested =
+          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieInstant instantCommitted =
+          new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+      timeline.createNewInstant(instantRequested);
+      timeline.transitionRequestedToInflight(instantRequested, Option.empty());
+      timeline.saveAsComplete(
+          new HoodieInstant(true, instantRequested.getAction(), instantRequested.getTimestamp()),
+          Option.empty());
+      metaClient.reloadActiveTimeline();

Review Comment:
   There're different behaviors for HDFS and object stores, for HDFS, it's modification time won't change if it's a rename operation, but since many object store use `COPY + DELETE` operations to perform `move`, the modification time will be updated in such cases.



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1521618775

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16494",
       "triggerID" : "e3449eb4ac288eacb5cd60bedf858622f00979bb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85b25f5cda4ccd8189a1607259e1732a910c3262",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16523",
       "triggerID" : "ef5fad3612bc910902d52de0296ef4c8852e5319",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638",
       "triggerID" : "8d611417db0e3eb818bec1fd63a37dec5aa4b3ad",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642",
       "triggerID" : "bfb9fbbed9a2423ba1781962cea8ccc277a84880",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 85b25f5cda4ccd8189a1607259e1732a910c3262 UNKNOWN
   * 8d611417db0e3eb818bec1fd63a37dec5aa4b3ad Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16638) 
   * bfb9fbbed9a2423ba1781962cea8ccc277a84880 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16642) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1552739597

   > We can implement this by adding a extra flag in some way or just simply comparing the file modified time of the last archived log. Also, to accelerate filtering out instants with greater state transient time in archive timeline we can also compare the modification time of the archived log.
   
   You are right, we need some facilities to speed up the query for transition time -> completion time mappings, in Hudi release 1.0, we are planning to remove the archival timeline and there is only one timeline which is always active, so in my opition, fixing the hive sync by using the completion time on the active timeline can solve most of the multi-writer use cases, if we fond any instants are archived, we should fallback to a full sync. And we already support such fallback in PR: https://github.com/apache/hudi/pull/8388


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


[GitHub] [hudi] danny0405 commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1530862011

   > we should also make the stateTransition time as the default for "timestamp as of" or time travel query
   
   I'm not sure we should do that, because it is a semantics change, by using the timestamp, we are actually indicating the instant version number(yeah the timestamp is another format of version id), by using stateTransition time, we are using the completion time, that's a huge different conceptially, we should be conservative about that.
   
   The tricky part left that needs to be addressed is like you said, the handing of archivied instants, we do not support effieicnt filtering by state transition time on the archived timeline, and we do not support a state transition time versioning on the files layout(`_hoodie_commit_time` is the txn start time, whichi is a inconsistency).
   
   We need to think through about this part.
   
   So in Flink side, I just drop the support for switching the start/end offset by using state transition time, instead, the state transition time is used as a internal attribute to help find those instants written by multiplr writers that we previously missed to read. But still, we need to solve the archiving issue.


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


[GitHub] [hudi] Zouxxyy commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "Zouxxyy (via GitHub)" <gi...@apache.org>.
Zouxxyy commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1538655216

   @boneanxs @danny0405 
   
   In one of my linux machine,  `fs.listStatus` returns time with second precision. It looks like `stateTransitionTime`'s may depends on the filesystem and machine.
   Problems arise due to loss of precision, e.g. it may be even smaller than the start time.
   Here is the discuss I find https://stackoverflow.com/questions/24804618/get-file-mtime-with-millisecond-resolution-from-java.


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


[GitHub] [hudi] vinothchandar commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "vinothchandar (via GitHub)" <gi...@apache.org>.
vinothchandar commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1167330162


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -70,32 +100,38 @@ public enum State {
     NIL
   }
 
-  private State state = State.COMPLETED;
-  private String action;
-  private String timestamp;
+  private final State state;
+  private final String action;
+  private final String timestamp;
+  private final String stateTransitionTime;
 
   /**
    * Load the instant from the meta FileStatus.
    */
   public HoodieInstant(FileStatus fileStatus) {
     // First read the instant timestamp. [==>20170101193025<==].commit
     String fileName = fileStatus.getPath().getName();
-    String fileExtension = getTimelineFileExtension(fileName);
-    timestamp = fileName.replace(fileExtension, "");
-
-    // Next read the action for this marker
-    action = fileExtension.replaceFirst(".", "");
-    if (action.equals("inflight")) {
-      // This is to support backwards compatibility on how in-flight commit files were written
-      // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
-      action = "commit";
-      state = State.INFLIGHT;
-    } else if (action.contains(HoodieTimeline.INFLIGHT_EXTENSION)) {
-      state = State.INFLIGHT;
-      action = action.replace(HoodieTimeline.INFLIGHT_EXTENSION, "");
-    } else if (action.contains(HoodieTimeline.REQUESTED_EXTENSION)) {
-      state = State.REQUESTED;
-      action = action.replace(HoodieTimeline.REQUESTED_EXTENSION, "");
+    Matcher matcher = NAME_FORMAT.matcher(fileName);
+    if (matcher.find()) {
+      timestamp = matcher.group(1);
+      if (matcher.group(2).equals(HoodieTimeline.INFLIGHT_EXTENSION)) {
+        // This is to support backwards compatibility on how in-flight commit files were written
+        // General rule is inflight extension is .<action>.inflight, but for commit it is .inflight
+        action = "commit";
+        state = State.INFLIGHT;
+      } else {
+        action = matcher.group(2).replaceFirst(".", "");
+        if (matcher.groupCount() == 3 && matcher.group(3) != null) {
+          state = State.valueOf(matcher.group(3).replaceFirst(".", "").toUpperCase());
+        } else {
+          // Like 20230104152218702.commit
+          state = State.COMPLETED;
+        }
+      }
+      stateTransitionTime =

Review Comment:
   but the checkpoints will be lost and the downstream pipelines will have to be recomputed all over again. Have you run into this situation @boneanxs ?



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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1511190609

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388",
       "triggerID" : "d5f7e0f51ce768b34aa01ff970da9c187c4f8c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d5f7e0f51ce768b34aa01ff970da9c187c4f8c16 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=16388) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1432467054

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3d7b6c042316da0f33cb1b343b878b171f18dbdd Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187) 
   * f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] hudi-bot commented on pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "hudi-bot (via GitHub)" <gi...@apache.org>.
hudi-bot commented on PR #7627:
URL: https://github.com/apache/hudi/pull/7627#issuecomment-1434062912

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=14187",
       "triggerID" : "3d7b6c042316da0f33cb1b343b878b171f18dbdd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228",
       "triggerID" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241",
       "triggerID" : "1432896765",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267",
       "triggerID" : "693fe2d347bf62a4340ad5d397d29caf092a5cb9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f7ee43dc4c8d20b3a18fbd47c4da8942d1516bf1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15228) Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15241) 
   * 693fe2d347bf62a4340ad5d397d29caf092a5cb9 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=15267) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1151337720


##########
hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc:
##########
@@ -128,6 +128,11 @@
             "HoodieIndexCommitMetadata"
          ],
          "default": null
+      },
+      {
+         "name":"stateTransitionTime",
+         "type":["null","string"],
+         "default": null

Review Comment:
   This could cause backward compatibility issue, if the table is upgraded, this field could be missing from the old table, we need to address this in release note at least.



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


[GitHub] [hudi] boneanxs commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "boneanxs (via GitHub)" <gi...@apache.org>.
boneanxs commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1169590208


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala:
##########
@@ -133,8 +133,16 @@ trait HoodieIncrementalRelationTrait extends HoodieBaseRelation {
   // Validate this Incremental implementation is properly configured
   validate()
 
+  private val useStateTransitionTime = optParams.get(DataSourceReadOptions.INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME.key)
+    .map(_.toBoolean)
+    .getOrElse(DataSourceReadOptions.INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME.defaultValue)
+
   protected def startTimestamp: String = optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key)
-  protected def endTimestamp: String = optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key, super.timeline.lastInstant().get.getTimestamp)
+  protected def endTimestamp: String = if (useStateTransitionTime) {
+    optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key, super.timeline.lastInstant().get.getStateTransitionTime)
+  } else {

Review Comment:
   sure, will add document for these 2 configs.



##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestHoodieInstant extends HoodieCommonTestHarness {
+
+  @Test
+  public void testExtractTimestamp() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName));
+  }
+
+  @Test
+  public void testGetTimelineFileExtension() {
+    String fileName = "20230104152218702.inflight";
+    assertEquals(".inflight", HoodieInstant.getTimelineFileExtension(fileName));
+
+    fileName = "20230104152218702.commit.request";
+    assertEquals(".commit.request", HoodieInstant.getTimelineFileExtension(fileName));
+  }
+
+  @Test
+  public void testCreateHoodieInstantByFileStatus() throws IOException {
+    try {
+      initMetaClient();
+      HoodieInstant instantRequested =
+          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieInstant instantCommitted =
+          new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
+      HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+      timeline.createNewInstant(instantRequested);
+      timeline.transitionRequestedToInflight(instantRequested, Option.empty());
+      timeline.saveAsComplete(
+          new HoodieInstant(true, instantRequested.getAction(), instantRequested.getTimestamp()),
+          Option.empty());
+      metaClient.reloadActiveTimeline();

Review Comment:
   All tests from `TestStreamingSource` will use `stateTransitionTime` to do the streaming job(This pr enables `hoodie.incremental.fetch.instant.by.state.transition.time` by default for `HoodieStreamSource`)
   
   I'll also add some tests for `IncrementalRelation` and `MergeOnReadIncrementRelation` separately.



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


[GitHub] [hudi] danny0405 commented on a diff in pull request #7627: [HUDI-5517] HoodieTimeline support filter instants by state transition time

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1169765841


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java:
##########
@@ -104,12 +140,21 @@ public HoodieInstant(boolean isInflight, String action, String timestamp) {
     this.state = isInflight ? State.INFLIGHT : State.COMPLETED;
     this.action = action;
     this.timestamp = timestamp;
+    this.stateTransitionTime = "";

Review Comment:
   empty string or just left it as null?



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