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 2021/08/06 05:50:23 UTC

[GitHub] [hudi] swuferhong opened a new pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

swuferhong opened a new pull request #3401:
URL: https://github.com/apache/hudi/pull/3401


   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contributing.html before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   While preCombie happen, always choose the latest record for HoodieRecordPayload.
   
   ## Brief change log
   
   *(for example:)*
     - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.


-- 
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 change in pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#discussion_r683820637



##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
##########
@@ -231,6 +229,83 @@ public void testUpsertsForMultipleRecordsInSameFile(ExternalSpillableMap.DiskMap
     }
   }
 
+  @ParameterizedTest
+  @MethodSource("testArguments")
+  public void testHoodieMergeHandlePreCombine(ExternalSpillableMap.DiskMapType diskMapType,
+                                              boolean isCompressionEnabled) throws Exception {
+    // Create records in a single partition
+    String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
+    dataGen = new HoodieTestDataGenerator(new String[] {partitionPath});
+
+    // Build a common config with diff configs
+    Properties properties = new Properties();
+    properties.setProperty(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), diskMapType.name());
+    properties.setProperty(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), String.valueOf(isCompressionEnabled));
+    metaClient.getTableConfig().setValue(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP, "timestamp");
+
+    // Build a write config with insert parallelism set
+    HoodieWriteConfig cfg = getConfigBuilder()
+        .withProperties(properties)
+        .build();
+
+    // test1: combine before insert, small preCombineField value record will be merged and replaced.
+    cfg.setValue(COMBINE_BEFORE_INSERT_PROP, "true");
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
+      String newCommitTime = "001";
+      client.startCommitWithTime(newCommitTime);
+
+      List<HoodieRecord> records = dataGen.generateRecordsWithTimestampInsert(newCommitTime);
+      JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
+      List<WriteStatus> statuses = client.insert(writeRecords, newCommitTime).collect();
+      assertNoWriteErrors(statuses);
+
+      metaClient = HoodieTableMetaClient.reload(metaClient);
+      HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
+      Dataset<Row> dataSet = HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime);
+      Row[] rows = (Row[]) dataSet.collect();
+      for (Row row : rows) {
+        if (row.getAs("_hoodie_record_key").equals("01")) {
+          assertEquals((Long) row.getAs("timestamp"), 4, "Record with greater preCombine field is chosen");
+        } else if (row.getAs("_hoodie_record_key").equals("02")) {
+          assertEquals((Long) row.getAs("timestamp"), 5, "Record with equal preCombine field, New arrival record is chosen");
+        } else if (row.getAs("_hoodie_record_key").equals("03")) {
+          assertEquals((Long) row.getAs("timestamp"), 6, "Record with greater preCombine field is chosen");
+        }
+      }
+      assertEquals(5, dataSet.count(),
+          "Must contain 5 records, because three records be combined");
+    }
+
+    //test2 not combine before insert, and combine will happen on HoodieMergeHandle.
+    cfg.setValue(COMBINE_BEFORE_INSERT_PROP, "false");
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
+      //Combine before insert be set false, and combine will happen on HoodieMergeHandle.
+      String newCommitTime = "002";
+      client.startCommitWithTime(newCommitTime);
+
+      List<HoodieRecord> records = dataGen.generateRecordsWithTimestampInsert(newCommitTime);
+      JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
+      List<WriteStatus> statuses = client.insert(writeRecords, newCommitTime).collect();
+      assertNoWriteErrors(statuses);
+
+      metaClient = HoodieTableMetaClient.reload(metaClient);
+      HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
+      Dataset<Row> dataSet = HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime);
+      Row[] rows = (Row[]) dataSet.collect();
+      for (Row row : rows) {

Review comment:
       can we reuse the asserts from here?

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
##########
@@ -231,6 +229,83 @@ public void testUpsertsForMultipleRecordsInSameFile(ExternalSpillableMap.DiskMap
     }
   }
 
+  @ParameterizedTest
+  @MethodSource("testArguments")
+  public void testHoodieMergeHandlePreCombine(ExternalSpillableMap.DiskMapType diskMapType,
+                                              boolean isCompressionEnabled) throws Exception {
+    // Create records in a single partition
+    String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
+    dataGen = new HoodieTestDataGenerator(new String[] {partitionPath});
+
+    // Build a common config with diff configs
+    Properties properties = new Properties();
+    properties.setProperty(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), diskMapType.name());
+    properties.setProperty(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), String.valueOf(isCompressionEnabled));
+    metaClient.getTableConfig().setValue(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP, "timestamp");
+
+    // Build a write config with insert parallelism set
+    HoodieWriteConfig cfg = getConfigBuilder()
+        .withProperties(properties)
+        .build();
+
+    // test1: combine before insert, small preCombineField value record will be merged and replaced.
+    cfg.setValue(COMBINE_BEFORE_INSERT_PROP, "true");
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
+      String newCommitTime = "001";
+      client.startCommitWithTime(newCommitTime);
+
+      List<HoodieRecord> records = dataGen.generateRecordsWithTimestampInsert(newCommitTime);
+      JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
+      List<WriteStatus> statuses = client.insert(writeRecords, newCommitTime).collect();
+      assertNoWriteErrors(statuses);
+
+      metaClient = HoodieTableMetaClient.reload(metaClient);
+      HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
+      Dataset<Row> dataSet = HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime);
+      Row[] rows = (Row[]) dataSet.collect();
+      for (Row row : rows) {
+        if (row.getAs("_hoodie_record_key").equals("01")) {

Review comment:
       use teh `HoodieRecord.` members?

##########
File path: hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java
##########
@@ -70,6 +70,10 @@
   private int type = 0;
   private Map<String, HoodieMetadataFileInfo> filesystemMetadata = null;
 
+  public HoodieMetadataPayload(GenericRecord record, Comparable<?> orderingVal) {

Review comment:
       would all existing payloads be forced to implement this new constructor?




-- 
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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591",
       "triggerID" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1597",
       "triggerID" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 3f595f89fb7ad727b6170c89a9e737d04ca61865 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591) 
   * f7d063a6400a716c3c62188ac6a98e27c14b2c1c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1597) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 change in pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#discussion_r683938063



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
##########
@@ -239,7 +240,14 @@ protected void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
         record.seal();
       }
       // NOTE: Once Records are added to map (spillable-map), DO NOT change it as they won't persist
-      keyToNewRecords.put(record.getRecordKey(), record);
+      String key = record.getRecordKey();
+      if (keyToNewRecords.containsKey(key)) {

Review comment:
       typically we won't hit this, since the precombine is done on incoming records by default, even before we index. So mostly seems like a fallback handling? 




-- 
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] swuferhong commented on a change in pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
swuferhong commented on a change in pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#discussion_r683900152



##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
##########
@@ -231,6 +229,83 @@ public void testUpsertsForMultipleRecordsInSameFile(ExternalSpillableMap.DiskMap
     }
   }
 
+  @ParameterizedTest
+  @MethodSource("testArguments")
+  public void testHoodieMergeHandlePreCombine(ExternalSpillableMap.DiskMapType diskMapType,
+                                              boolean isCompressionEnabled) throws Exception {
+    // Create records in a single partition
+    String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
+    dataGen = new HoodieTestDataGenerator(new String[] {partitionPath});
+
+    // Build a common config with diff configs
+    Properties properties = new Properties();
+    properties.setProperty(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), diskMapType.name());
+    properties.setProperty(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), String.valueOf(isCompressionEnabled));
+    metaClient.getTableConfig().setValue(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP, "timestamp");
+
+    // Build a write config with insert parallelism set
+    HoodieWriteConfig cfg = getConfigBuilder()
+        .withProperties(properties)
+        .build();
+
+    // test1: combine before insert, small preCombineField value record will be merged and replaced.
+    cfg.setValue(COMBINE_BEFORE_INSERT_PROP, "true");
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
+      String newCommitTime = "001";
+      client.startCommitWithTime(newCommitTime);
+
+      List<HoodieRecord> records = dataGen.generateRecordsWithTimestampInsert(newCommitTime);
+      JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
+      List<WriteStatus> statuses = client.insert(writeRecords, newCommitTime).collect();
+      assertNoWriteErrors(statuses);
+
+      metaClient = HoodieTableMetaClient.reload(metaClient);
+      HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
+      Dataset<Row> dataSet = HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime);
+      Row[] rows = (Row[]) dataSet.collect();
+      for (Row row : rows) {
+        if (row.getAs("_hoodie_record_key").equals("01")) {
+          assertEquals((Long) row.getAs("timestamp"), 4, "Record with greater preCombine field is chosen");
+        } else if (row.getAs("_hoodie_record_key").equals("02")) {
+          assertEquals((Long) row.getAs("timestamp"), 5, "Record with equal preCombine field, New arrival record is chosen");
+        } else if (row.getAs("_hoodie_record_key").equals("03")) {
+          assertEquals((Long) row.getAs("timestamp"), 6, "Record with greater preCombine field is chosen");
+        }
+      }
+      assertEquals(5, dataSet.count(),
+          "Must contain 5 records, because three records be combined");
+    }
+
+    //test2 not combine before insert, and combine will happen on HoodieMergeHandle.
+    cfg.setValue(COMBINE_BEFORE_INSERT_PROP, "false");
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
+      //Combine before insert be set false, and combine will happen on HoodieMergeHandle.
+      String newCommitTime = "002";
+      client.startCommitWithTime(newCommitTime);
+
+      List<HoodieRecord> records = dataGen.generateRecordsWithTimestampInsert(newCommitTime);
+      JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
+      List<WriteStatus> statuses = client.insert(writeRecords, newCommitTime).collect();
+      assertNoWriteErrors(statuses);
+
+      metaClient = HoodieTableMetaClient.reload(metaClient);
+      HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
+      Dataset<Row> dataSet = HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime);
+      Row[] rows = (Row[]) dataSet.collect();
+      for (Row row : rows) {

Review comment:
       > can we reuse the asserts from here?
   
   Ok

##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
##########
@@ -231,6 +229,83 @@ public void testUpsertsForMultipleRecordsInSameFile(ExternalSpillableMap.DiskMap
     }
   }
 
+  @ParameterizedTest
+  @MethodSource("testArguments")
+  public void testHoodieMergeHandlePreCombine(ExternalSpillableMap.DiskMapType diskMapType,
+                                              boolean isCompressionEnabled) throws Exception {
+    // Create records in a single partition
+    String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
+    dataGen = new HoodieTestDataGenerator(new String[] {partitionPath});
+
+    // Build a common config with diff configs
+    Properties properties = new Properties();
+    properties.setProperty(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), diskMapType.name());
+    properties.setProperty(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(), String.valueOf(isCompressionEnabled));
+    metaClient.getTableConfig().setValue(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP, "timestamp");
+
+    // Build a write config with insert parallelism set
+    HoodieWriteConfig cfg = getConfigBuilder()
+        .withProperties(properties)
+        .build();
+
+    // test1: combine before insert, small preCombineField value record will be merged and replaced.
+    cfg.setValue(COMBINE_BEFORE_INSERT_PROP, "true");
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
+      String newCommitTime = "001";
+      client.startCommitWithTime(newCommitTime);
+
+      List<HoodieRecord> records = dataGen.generateRecordsWithTimestampInsert(newCommitTime);
+      JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
+      List<WriteStatus> statuses = client.insert(writeRecords, newCommitTime).collect();
+      assertNoWriteErrors(statuses);
+
+      metaClient = HoodieTableMetaClient.reload(metaClient);
+      HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
+      Dataset<Row> dataSet = HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime);
+      Row[] rows = (Row[]) dataSet.collect();
+      for (Row row : rows) {
+        if (row.getAs("_hoodie_record_key").equals("01")) {

Review comment:
       > use teh `HoodieRecord.` members?
   
   Ok




-- 
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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 829139528e53953e6f39d708d31fb876f0a32cd1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * eb5956b8463eafa8253732698fc26778b15b2829 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358) 
   * 829139528e53953e6f39d708d31fb876f0a32cd1 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591",
       "triggerID" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1597",
       "triggerID" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cf30447b9d400b39a8906323547f13fce695621",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1615",
       "triggerID" : "3cf30447b9d400b39a8906323547f13fce695621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 3cf30447b9d400b39a8906323547f13fce695621 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1615) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * eb5956b8463eafa8253732698fc26778b15b2829 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
danny0405 commented on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892481247


   @nsivabalan Please review if you have 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] hudi-bot edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 0b34d55f238b889fb2fcc2526e4657ea981c431c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397) 
   * d00f0bc28228f3699e29ed30ea60cd11b45d5eae Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 0b34d55f238b889fb2fcc2526e4657ea981c431c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * eb5956b8463eafa8253732698fc26778b15b2829 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 1346c3641de4c019ed289f09148eda4df2767ccc Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-896091938


   @swuferhong please ping me once this is ready to go. 


-- 
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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052






-- 
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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * d00f0bc28228f3699e29ed30ea60cd11b45d5eae Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422) 
   * 1ecdef4992b543784bc0854de59fd7d7d58debd3 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot 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 #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
danny0405 merged pull request #3401:
URL: https://github.com/apache/hudi/pull/3401


   


-- 
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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * eb5956b8463eafa8253732698fc26778b15b2829 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591",
       "triggerID" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1597",
       "triggerID" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cf30447b9d400b39a8906323547f13fce695621",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3cf30447b9d400b39a8906323547f13fce695621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * f7d063a6400a716c3c62188ac6a98e27c14b2c1c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1597) 
   * 3cf30447b9d400b39a8906323547f13fce695621 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * d00f0bc28228f3699e29ed30ea60cd11b45d5eae Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 829139528e53953e6f39d708d31fb876f0a32cd1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392) 
   * 0b34d55f238b889fb2fcc2526e4657ea981c431c Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * eb5956b8463eafa8253732698fc26778b15b2829 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358) 
   * 829139528e53953e6f39d708d31fb876f0a32cd1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591",
       "triggerID" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1597",
       "triggerID" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * f7d063a6400a716c3c62188ac6a98e27c14b2c1c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1597) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 change in pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on a change in pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#discussion_r685189688



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/OverwriteWithLatestAvroPayload.java
##########
@@ -47,10 +47,14 @@ public OverwriteWithLatestAvroPayload(Option<GenericRecord> record) {
   }
 
   @Override
-  public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload another) {
+  public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload oldValue) {
     // pick the payload with greatest ordering value
-    if (another.orderingVal.compareTo(orderingVal) > 0) {
-      return another;
+    if (oldValue.recordBytes.length == 0) {
+      // use natural order for delete record
+      return this;
+    }
+    if (oldValue.orderingVal.compareTo(orderingVal) > 0) {

Review comment:
       nit: can you move the comment "// pick the payload with greatest ordering value" to line 56 ish. 




-- 
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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 1ecdef4992b543784bc0854de59fd7d7d58debd3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556) 
   * 1346c3641de4c019ed289f09148eda4df2767ccc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot 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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 1ecdef4992b543784bc0854de59fd7d7d58debd3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot 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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591",
       "triggerID" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1597",
       "triggerID" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3cf30447b9d400b39a8906323547f13fce695621",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1615",
       "triggerID" : "3cf30447b9d400b39a8906323547f13fce695621",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * f7d063a6400a716c3c62188ac6a98e27c14b2c1c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1597) 
   * 3cf30447b9d400b39a8906323547f13fce695621 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1615) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 1346c3641de4c019ed289f09148eda4df2767ccc Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559) 
   * 3f595f89fb7ad727b6170c89a9e737d04ca61865 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 1ecdef4992b543784bc0854de59fd7d7d58debd3 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556) 
   * 1346c3641de4c019ed289f09148eda4df2767ccc Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot 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] swuferhong commented on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
swuferhong commented on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-893953643


   
   
   
   
   > @swuferhong can you please address these and also verify once that #2977 is also fully handled by this PR
   
   Yes, I checked that this PR fully handle #2977.


-- 
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] swuferhong closed pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
swuferhong closed pull request #3401:
URL: https://github.com/apache/hudi/pull/3401


   


-- 
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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591",
       "triggerID" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 3f595f89fb7ad727b6170c89a9e737d04ca61865 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
danny0405 commented on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-896449020


   @vinothchandar merged.


-- 
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 edited a comment on pull request #3401: [HUDI-2170] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 829139528e53953e6f39d708d31fb876f0a32cd1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392) 
   * 0b34d55f238b889fb2fcc2526e4657ea981c431c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591",
       "triggerID" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f7d063a6400a716c3c62188ac6a98e27c14b2c1c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 3f595f89fb7ad727b6170c89a9e737d04ca61865 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591) 
   * f7d063a6400a716c3c62188ac6a98e27c14b2c1c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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] swuferhong closed pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
swuferhong closed pull request #3401:
URL: https://github.com/apache/hudi/pull/3401


   


-- 
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] swuferhong commented on a change in pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
swuferhong commented on a change in pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#discussion_r683900091



##########
File path: hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java
##########
@@ -70,6 +70,10 @@
   private int type = 0;
   private Map<String, HoodieMetadataFileInfo> filesystemMetadata = null;
 
+  public HoodieMetadataPayload(GenericRecord record, Comparable<?> orderingVal) {

Review comment:
       > would all existing payloads be forced to implement this new constructor?
   
   Yes.




-- 
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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1556",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559",
       "triggerID" : "1346c3641de4c019ed289f09148eda4df2767ccc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591",
       "triggerID" : "3f595f89fb7ad727b6170c89a9e737d04ca61865",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 1346c3641de4c019ed289f09148eda4df2767ccc Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1559) 
   * 3f595f89fb7ad727b6170c89a9e737d04ca61865 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1591) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-893875879


   @swuferhong can you please address these and also verify once that #2977 is also fully handled by this 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] hudi-bot edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1ecdef4992b543784bc0854de59fd7d7d58debd3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * d00f0bc28228f3699e29ed30ea60cd11b45d5eae Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422) 
   * 1ecdef4992b543784bc0854de59fd7d7d58debd3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot 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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * 0b34d55f238b889fb2fcc2526e4657ea981c431c Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397) 
   * d00f0bc28228f3699e29ed30ea60cd11b45d5eae UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run travis` re-run the last Travis build
    - `@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 change in pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#discussion_r683992963



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
##########
@@ -239,7 +240,14 @@ protected void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
         record.seal();
       }
       // NOTE: Once Records are added to map (spillable-map), DO NOT change it as they won't persist
-      keyToNewRecords.put(record.getRecordKey(), record);
+      String key = record.getRecordKey();
+      if (keyToNewRecords.containsKey(key)) {

Review comment:
       Yeah, we can avoid this because the deduplication can be executed before entering this handle, but `precombine` is not by default true, take a look at Spark option `DataSourceOptions.INSERT_DROP_DUPS`




-- 
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 edited a comment on pull request #3401: [HUDI-2170] [HUDI-1763] Always choose the latest record for HoodieRecordPayload

Posted by GitBox <gi...@apache.org>.
hudi-bot edited a comment on pull request #3401:
URL: https://github.com/apache/hudi/pull/3401#issuecomment-892472052


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fe0db6bdda8a2f543d068efa1cbb60682b2ef95",
       "triggerType" : "PUSH"
     }, {
       "hash" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1358",
       "triggerID" : "eb5956b8463eafa8253732698fc26778b15b2829",
       "triggerType" : "PUSH"
     }, {
       "hash" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1392",
       "triggerID" : "829139528e53953e6f39d708d31fb876f0a32cd1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1397",
       "triggerID" : "0b34d55f238b889fb2fcc2526e4657ea981c431c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422",
       "triggerID" : "d00f0bc28228f3699e29ed30ea60cd11b45d5eae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7fe0db6bdda8a2f543d068efa1cbb60682b2ef95 UNKNOWN
   * d00f0bc28228f3699e29ed30ea60cd11b45d5eae Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=1422) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot 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