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 2022/10/23 00:52:14 UTC

[GitHub] [hudi] nsivabalan opened a new pull request, #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   ### Change Logs
   
   As of now, during completion of a write operation, we [unpersist all persisted Rdds](https://github.com/apache/hudi/blob/b78c3441c4e28200abec340eaff852375764cbdb/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java#L584) in the same sparkcontext. This could cause issues w/ async table services, metadata table etc. So, it needs to be fixed. 
   
   Fix:
   We need a way to keep track of all persisted RDD ids for a given write operation and unpersist only those in the end. 
   - Initially thought we can leverage JavaSparkContext.setEngineProperty, but looks like its applicable only for current thread. So multiple threads within the same process may not share the same properties. 
   
   Apart from this, I could not locate any other mutable property that can be accessed from lower layers (action executors) and also by write clients, unless we change the all methods to take in extra argument and return extra argument. 
   
   So, only viable option I can think of is introducing a static hashmap which can be accessed from anywhere. So, lower layer class like action executors will add entries to this hashmap. Key refers to table+commit time and value refers to comma separated list of persisted RDD Ids. 
   
   During release resource call towards completion of a write, we can unpersist only those rdds which are part of the tracked entry in the hashmap. 
   
   Deltastreamer unpersist:
   Since the WriteClient.releaseResources() already unpersist the rdds, unpersist call in DeltaSync is redundant and have removed it.  
   
   WIP:
   Patch is a draft patch for now. Yet to fix tests. 
   
   ### Impact
   
   _Describe any public API or user-facing feature change or any performance impact._
   
   ### Risk level (write none, low medium or high below)
   
   _If medium or high, explain what verification was done to mitigate the risks._
   
   ### 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] xushiyan commented on pull request #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   closing this one. https://github.com/apache/hudi/pull/7914 was landed


-- 
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] alexeykudinkin commented on a diff in pull request #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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


##########
hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java:
##########
@@ -44,6 +46,28 @@ public class CommitUtils {
 
   private static final Logger LOG = LogManager.getLogger(CommitUtils.class);
   private static final String NULL_SCHEMA_STR = Schema.create(Schema.Type.NULL).toString();
+  public static transient ConcurrentHashMap<String, List<Integer>> PERSISTED_RDD_IDS = new ConcurrentHashMap();

Review Comment:
   Strongly agree w/ @xushiyan argument: we should try to avoid any global state at all costs. 
   
   Moreover, provided that WriteClient
    - Is the origination point of any RDD handling
    - Also the termination point for it
   
   We should be totally able to manage RDDs lifecycle entirely enclosed w/in the Write Client itself. For ex, we can do something along these lines: In post-commit operation we traverse whole RDD tree and unpersist only RDDs we've just committed



##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java:
##########
@@ -686,27 +686,28 @@ public void testHandleUpdateWithMultiplePartitions() throws Exception {
 
   @Test
   public void testReleaseResource() throws Exception {
-    HoodieWriteConfig.Builder builder = getConfigBuilder(true);
+    HoodieWriteConfig.Builder builder = getConfigBuilder(false);
     builder.withReleaseResourceEnabled(true);
-    builder.withAutoCommit(false);
 
     setUp(builder.build().getProps());
+    HoodieWriteConfig writeConfig = builder.build();
 
     /**
      * Write 1 (test when RELEASE_RESOURCE_ENABLE is true)
      */
-    try (SparkRDDWriteClient client = getHoodieWriteClient(builder.build())) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(writeConfig)) {
 
       String newCommitTime = "001";
       client.startCommitWithTime(newCommitTime);
-
       List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
       JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
       writeRecords.persist(StorageLevel.MEMORY_AND_DISK());
       List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
       assertNoWriteErrors(statuses);
+
       client.commitStats(newCommitTime, statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList()), Option.empty(), metaClient.getCommitActionType());
-      assertEquals(spark().sparkContext().persistentRdds().size(), 0);
+      // when auto commit is enabled, we can't unpersist the rdd at the end of write operation.
+      assertEquals(spark().sparkContext().persistentRdds().size(), writeConfig.isMetadataTableEnabled() ? 3 : 1);

Review Comment:
   Args order should be inverted



##########
hudi-common/src/main/java/org/apache/hudi/common/data/HoodieListData.java:
##########
@@ -184,4 +184,9 @@ public int getNumPartitions() {
   public List<T> collectAsList() {
     return super.collectAsList();
   }
+
+  @Override
+  public int getId() {
+    return 0;

Review Comment:
   Let's return -1 to clearly signal invalid value



-- 
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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 850b1d6209c2b7aa2fe18a5d73fb4941383e617b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616) 
   * e97e4d557c205c9a83751479577ad9a00bf75c41 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619) 
   * e651e0f4274de642388a44cc97765cbcda805f0f 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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e97e4d557c205c9a83751479577ad9a00bf75c41 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619) 
   * e651e0f4274de642388a44cc97765cbcda805f0f Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767",
       "triggerID" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12911",
       "triggerID" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe186d03969c6d1b4a62d9c506585a8b2ea05dd0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12920",
       "triggerID" : "fe186d03969c6d1b4a62d9c506585a8b2ea05dd0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fe186d03969c6d1b4a62d9c506585a8b2ea05dd0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12920) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767",
       "triggerID" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5ff96812e74f348af76c942f58e67445afbb765e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e97e4d557c205c9a83751479577ad9a00bf75c41 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619) 
   * e651e0f4274de642388a44cc97765cbcda805f0f 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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9ffc15c02a719adff1492b6d623873fc9ac8040b 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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767",
       "triggerID" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12911",
       "triggerID" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe186d03969c6d1b4a62d9c506585a8b2ea05dd0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12920",
       "triggerID" : "fe186d03969c6d1b4a62d9c506585a8b2ea05dd0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dc7c2bdefdb2a84b27b40751714a31604e2931eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12911) 
   * fe186d03969c6d1b4a62d9c506585a8b2ea05dd0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12920) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9ffc15c02a719adff1492b6d623873fc9ac8040b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477) 
   * 850b1d6209c2b7aa2fe18a5d73fb4941383e617b 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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 55813534a98978a94e20aa2f109c0298657d011e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e651e0f4274de642388a44cc97765cbcda805f0f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742) 
   * 55813534a98978a94e20aa2f109c0298657d011e 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 closed pull request #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

Posted by "xushiyan (via GitHub)" <gi...@apache.org>.
xushiyan closed pull request #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation
URL: https://github.com/apache/hudi/pull/7039


-- 
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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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


##########
hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java:
##########
@@ -44,6 +46,28 @@ public class CommitUtils {
 
   private static final Logger LOG = LogManager.getLogger(CommitUtils.class);
   private static final String NULL_SCHEMA_STR = Schema.create(Schema.Type.NULL).toString();
+  public static transient ConcurrentHashMap<String, List<Integer>> PERSISTED_RDD_IDS = new ConcurrentHashMap();

Review Comment:
   this tightly couples with spark specific logic, and interacting with spark-internal logic is at this layer is my major concern. And not in favor of maintaining a global state for distributed processing. Can we try tackling this within spark client itself? from high-level, we basically want to track the persisted RDDs' ids and filter the RDDs by the tracked ids when unpersisting. all these can happen within a client's lifecycle so theoretically this can be encapsulated well. just to throw some ideas, haven't verified it by checking the code myself



-- 
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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767",
       "triggerID" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 55813534a98978a94e20aa2f109c0298657d011e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743) 
   * 5ff96812e74f348af76c942f58e67445afbb765e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 55813534a98978a94e20aa2f109c0298657d011e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743) 
   * 5ff96812e74f348af76c942f58e67445afbb765e 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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 850b1d6209c2b7aa2fe18a5d73fb4941383e617b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616) 
   * e97e4d557c205c9a83751479577ad9a00bf75c41 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 850b1d6209c2b7aa2fe18a5d73fb4941383e617b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9ffc15c02a719adff1492b6d623873fc9ac8040b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477) 
   
   <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] nsivabalan commented on pull request #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   @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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e651e0f4274de642388a44cc97765cbcda805f0f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624) 
   
   <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] nsivabalan commented on a diff in pull request #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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


##########
hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java:
##########
@@ -44,6 +46,28 @@ public class CommitUtils {
 
   private static final Logger LOG = LogManager.getLogger(CommitUtils.class);
   private static final String NULL_SCHEMA_STR = Schema.create(Schema.Type.NULL).toString();
+  public static transient ConcurrentHashMap<String, List<Integer>> PERSISTED_RDD_IDS = new ConcurrentHashMap();

Review Comment:
   sounds fair. I have an idea on how to go about this. but lets jam and have a consensus before I go ahead. 



-- 
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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767",
       "triggerID" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12911",
       "triggerID" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5ff96812e74f348af76c942f58e67445afbb765e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767) 
   * dc7c2bdefdb2a84b27b40751714a31604e2931eb Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12911) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9ffc15c02a719adff1492b6d623873fc9ac8040b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9ffc15c02a719adff1492b6d623873fc9ac8040b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477) 
   * 850b1d6209c2b7aa2fe18a5d73fb4941383e617b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 850b1d6209c2b7aa2fe18a5d73fb4941383e617b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616) 
   * e97e4d557c205c9a83751479577ad9a00bf75c41 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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * e651e0f4274de642388a44cc97765cbcda805f0f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624) Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e651e0f4274de642388a44cc97765cbcda805f0f Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624) Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742) 
   * 55813534a98978a94e20aa2f109c0298657d011e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767",
       "triggerID" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5ff96812e74f348af76c942f58e67445afbb765e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767) 
   * dc7c2bdefdb2a84b27b40751714a31604e2931eb 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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767",
       "triggerID" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12911",
       "triggerID" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dc7c2bdefdb2a84b27b40751714a31604e2931eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12911) 
   
   <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 #7039: [HUDI-5080] Fixing unpersist to consider only rdds pertaining to current write operation

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12477",
       "triggerID" : "9ffc15c02a719adff1492b6d623873fc9ac8040b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12616",
       "triggerID" : "850b1d6209c2b7aa2fe18a5d73fb4941383e617b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12619",
       "triggerID" : "e97e4d557c205c9a83751479577ad9a00bf75c41",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12624",
       "triggerID" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e651e0f4274de642388a44cc97765cbcda805f0f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12742",
       "triggerID" : "1301560475",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "55813534a98978a94e20aa2f109c0298657d011e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12743",
       "triggerID" : "55813534a98978a94e20aa2f109c0298657d011e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12767",
       "triggerID" : "5ff96812e74f348af76c942f58e67445afbb765e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12911",
       "triggerID" : "dc7c2bdefdb2a84b27b40751714a31604e2931eb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe186d03969c6d1b4a62d9c506585a8b2ea05dd0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fe186d03969c6d1b4a62d9c506585a8b2ea05dd0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * dc7c2bdefdb2a84b27b40751714a31604e2931eb Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=12911) 
   * fe186d03969c6d1b4a62d9c506585a8b2ea05dd0 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