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/11/28 02:41:01 UTC

[GitHub] [hudi] lsyldliu opened a new pull request #4141: [HUDI-2815] Support partial update for streaming change logs

lsyldliu opened a new pull request #4141:
URL: https://github.com/apache/hudi/pull/4141


   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before opening a pull request.*
   
   ## What is the purpose of the pull request
   
   Support partial update for streaming change logs
   
   ## Brief change log
   
     - *Support partial update for streaming change logs*
   
   ## Verify this pull request
   
   This change added tests and can be verified as follows:
     - *Added TestPartialUpdateWithLatestAvroPayload to verify the change.*
   
   ## 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] hudi-bot removed a comment on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] stayrascal commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record which value is not null to existing record instead of all fields.
+ *
+ * <p> Assuming a {@link GenericRecord} has three fields: a int , b int, c int. The first record value: 1, 2, 3.
+ * The second record value is: 4, 5, null, the field c value is null. After call the combineAndGetUpdateValue method,
+ * we will get final record value: 4, 5, 3, field c value will not be overwritten because its value is null in latest record.
+ */
+public class PartialUpdateWithLatestAvroPayload extends DefaultHoodieRecordPayload {
+

Review comment:
       Hi @danny0405 , if I didn't understand wrongly, the `preCombine` method will be called during deduplicate record by `flushBucket` or `flushRemaining` in `SteamWriteFucntion`, which will only deduplicate the records(new created/updated record) in the buffer. 
   
   If we only overwrite `preCombine`, which will only update/merge the records in the buffer, but if there is a record with same recordKey existed in base/log file, the record will be overwrote by the  new merged record from buffer, right?
   
   For example, in COW mode, we might still need to overwrite `combineAndGetUpdateValue` method, because it will be called by `HoodieMergeHandle.write(GenericRecord oldRecord)`, this method will merge the new merged record with old records.
   ```
   public void write(GenericRecord oldRecord) {
       String key = KeyGenUtils.getRecordKeyFromGenericRecord(oldRecord, keyGeneratorOpt);
       boolean copyOldRecord = true;
       if (keyToNewRecords.containsKey(key)) {
         // If we have duplicate records that we are updating, then the hoodie record will be deflated after
         // writing the first record. So make a copy of the record to be merged
         HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
         try {
           Option<IndexedRecord> combinedAvroRecord =
               hoodieRecord.getData().combineAndGetUpdateValue(oldRecord,
                 useWriterSchema ? tableSchemaWithMetaFields : tableSchema,
                   config.getPayloadConfig().getProps());
   
           if (combinedAvroRecord.isPresent() && combinedAvroRecord.get().equals(IGNORE_RECORD)) {
             // If it is an IGNORE_RECORD, just copy the old record, and do not update the new record.
             copyOldRecord = true;
           } else if (writeUpdateRecord(hoodieRecord, oldRecord, combinedAvroRecord)) {
             /*
              * ONLY WHEN 1) we have an update for this key AND 2) We are able to successfully
              * write the the combined new
              * value
              *
              * We no longer need to copy the old record over.
              */
             copyOldRecord = false;
           }
           writtenRecordKeys.add(key);
         } catch (Exception e) {
           throw new HoodieUpsertException("Failed to combine/merge new record with old value in storage, for new record {"
               + keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
         }
       }
   ```




-- 
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 removed a comment on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] lsyldliu commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record to old record instead of all fields.
+ */
+public class PartialUpdateWithLatestAvroPayload extends DefaultHoodieRecordPayload {
+
+  public PartialUpdateWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
+    super(record, orderingVal);
+  }
+
+  @Override
+  public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException {
+    if (recordBytes.length == 0) {
+      return Option.of(currentValue);
+    }
+
+    GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
+
+    // Null check is needed here to support schema evolution. The record in storage may be from old schema where
+    // the new ordering column might not be present and hence returns null.
+    if (!needUpdatingPersistedRecord(currentValue, incomingRecord, properties)) {
+      return Option.of(currentValue);
+    }
+
+    if (isDeleteRecord(incomingRecord)) {
+      return Option.empty();
+    }
+
+    GenericRecord currentRecord = (GenericRecord) currentValue;
+    // The field num in updated record may be less than old record, so only update these partial fields to old record.
+    List<Schema.Field> fields = schema.getFields();
+    fields.forEach(field -> {
+      Object value = incomingRecord.get(field.name());
+      if (Objects.nonNull(value)) {
+        currentRecord.put(field.name(), value);
+      }

Review comment:
       The user story is some fields may miss value in upstream, they don't want to override the missing value to null. I have discussed with @danny0405 offline, here exists two case: 
   1. the field value is really null in `GenericRecord`;
   2. the field value is missing in `GenericRecord` , however, we will get the null value.
   
   Because we don't identify the field value is really null or missing, so here overrides the field which has value uniformly. 




-- 
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 #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] stayrascal commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record which value is not null to existing record instead of all fields.
+ *
+ * <p> Assuming a {@link GenericRecord} has three fields: a int , b int, c int. The first record value: 1, 2, 3.
+ * The second record value is: 4, 5, null, the field c value is null. After call the combineAndGetUpdateValue method,
+ * we will get final record value: 4, 5, 3, field c value will not be overwritten because its value is null in latest record.
+ */
+public class PartialUpdateWithLatestAvroPayload extends DefaultHoodieRecordPayload {
+

Review comment:
       Seems that the partial update cannot support the case that the partition path is changed.
   
   Let's assume 
   - a record exists in base file (a=1, b=2,c=null,dt=2022-01-15), 
   - and an incoming record (a=1, b=null, c=3, dt=2022-01-16). 
    
   The `BucketAssignFunction` will generate a deleted record with partition path `2022-01-15` and the incoming record with partition path `2022-01-16`.
   The original record in base file will be removed while running `HoodieMergeHandle.write(GenericRecord oldRecord)`, but incoming record cannot find the relevant record under `2022-01-16`, so the result will only keep the info from incoming record(a=1, b=null, c=3, dt=2022-01-16), the original(b=2) will missed.
   
   Any thoughts here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] danny0405 commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record which value is not null to existing record instead of all fields.
+ *
+ * <p> Assuming a {@link GenericRecord} has three fields: a int , b int, c int. The first record value: 1, 2, 3.
+ * The second record value is: 4, 5, null, the field c value is null. After call the combineAndGetUpdateValue method,
+ * we will get final record value: 4, 5, 3, field c value will not be overwritten because its value is null in latest record.
+ */

Review comment:
       ```java
   /**
    * The only difference with {@link OverwriteNonDefaultsWithLatestAvroPayload} is that it supports
    * merging the latest non-null partial fields with the old record instead of replacing the whole record.
    *
    * <p> Assuming a {@link GenericRecord} has row schema: (f0 int , f1 int, f2 int).
    * The first record value is: (1, 2, 3), the second record value is: (4, 5, null) with the field c value as null.
    * Calling the #combineAndGetUpdateValue method of the two records returns record: (4, 5, 3).
    * Note that field c value is ignored because it is null.
    */
   ```

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record which value is not null to existing record instead of all fields.
+ *
+ * <p> Assuming a {@link GenericRecord} has three fields: a int , b int, c int. The first record value: 1, 2, 3.
+ * The second record value is: 4, 5, null, the field c value is null. After call the combineAndGetUpdateValue method,
+ * we will get final record value: 4, 5, 3, field c value will not be overwritten because its value is null in latest record.
+ */
+public class PartialUpdateWithLatestAvroPayload extends DefaultHoodieRecordPayload {
+

Review comment:
       We can extend from `OverwriteNonDefaultsWithLatestAvroPayload` instead of `DefaultHoodieRecordPayload `, and we should override `#preCombine` instead of `combineAndGetUpdateValue `.




-- 
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 #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yihua commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record to old record instead of all fields.

Review comment:
       nit: do you want to give a concrete example here to illustrate the operation of `combineAndGetUpdateValue()`?




-- 
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] lsyldliu commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record to old record instead of all fields.
+ */
+public class PartialUpdateWithLatestAvroPayload extends DefaultHoodieRecordPayload {
+
+  public PartialUpdateWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
+    super(record, orderingVal);
+  }
+
+  @Override
+  public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException {
+    if (recordBytes.length == 0) {
+      return Option.of(currentValue);
+    }
+
+    GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
+
+    // Null check is needed here to support schema evolution. The record in storage may be from old schema where
+    // the new ordering column might not be present and hence returns null.
+    if (!needUpdatingPersistedRecord(currentValue, incomingRecord, properties)) {
+      return Option.of(currentValue);
+    }
+
+    if (isDeleteRecord(incomingRecord)) {
+      return Option.empty();
+    }
+
+    GenericRecord currentRecord = (GenericRecord) currentValue;
+    // The field num in updated record may be less than old record, so only update these partial fields to old record.
+    List<Schema.Field> fields = schema.getFields();
+    fields.forEach(field -> {
+      Object value = incomingRecord.get(field.name());
+      if (Objects.nonNull(value)) {
+        currentRecord.put(field.name(), value);
+      }

Review comment:
       The user story is some fields may miss value in upstream, they don't want to override the missing value to null. I have discussed with @danny0405 offline, here exists two case: 
   1. the field value is really null in `GenericRecord`;
   2. the field value is missing in `GenericRecord` , however, we will get the null value.
   
   Because we can't identify the field value is really null or missing, so here overrides the field which has value uniformly. 




-- 
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] lsyldliu commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record to old record instead of all fields.
+ */
+public class PartialUpdateWithLatestAvroPayload extends DefaultHoodieRecordPayload {
+
+  public PartialUpdateWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
+    super(record, orderingVal);
+  }
+
+  @Override
+  public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException {
+    if (recordBytes.length == 0) {
+      return Option.of(currentValue);
+    }
+
+    GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
+
+    // Null check is needed here to support schema evolution. The record in storage may be from old schema where
+    // the new ordering column might not be present and hence returns null.
+    if (!needUpdatingPersistedRecord(currentValue, incomingRecord, properties)) {
+      return Option.of(currentValue);
+    }
+
+    if (isDeleteRecord(incomingRecord)) {
+      return Option.empty();
+    }
+
+    GenericRecord currentRecord = (GenericRecord) currentValue;
+    // The field num in updated record may be less than old record, so only update these partial fields to old record.
+    List<Schema.Field> fields = schema.getFields();
+    fields.forEach(field -> {
+      Object value = incomingRecord.get(field.name());
+      if (Objects.nonNull(value)) {
+        currentRecord.put(field.name(), value);
+      }

Review comment:
       The user story is some fields may miss value in upstream, they don't want to override the missing value to null. I have discussed with @danny0405 offline, here exists two case: 
   1. the field value is really null in `GenericRecord`;
   2. the field value is missing in `GenericRecord` , however, we will get the null value.
   Because we don't identify the field value is really null or missing, so here overrides the field which has value uniformly. 




-- 
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 #4141: [HUDI-2815] Support partial update for streaming change logs

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






-- 
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] lsyldliu commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record to old record instead of all fields.

Review comment:
       I will provide a concrete example here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] stayrascal commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record which value is not null to existing record instead of all fields.
+ *
+ * <p> Assuming a {@link GenericRecord} has three fields: a int , b int, c int. The first record value: 1, 2, 3.
+ * The second record value is: 4, 5, null, the field c value is null. After call the combineAndGetUpdateValue method,
+ * we will get final record value: 4, 5, 3, field c value will not be overwritten because its value is null in latest record.
+ */
+public class PartialUpdateWithLatestAvroPayload extends DefaultHoodieRecordPayload {
+

Review comment:
       Seems that the partial update cannot support the case that the partition path is changed.
   
   Let's assume 
   - a record exists in base file (a=1, b=2,c=null,dt=2022-01-15), 
   - and an incoming record (a=1, b=null, c=3, dt=2022-01-16). 
    
   The `BucketAssignFunction` will generate a deleted record with partition path `2022-01-15` and the incoming record with partition path `2022-01-16`.
   The original record in base file will be removed while running `HoodieMergeHandle.write(GenericRecord oldRecord)`, but incoming record cannot find the relevant record under `2022-01-16`, so the result will only keep the info from incoming record(a=1, b=null, c=3, dt=2022-01-16), the original(b=2) will missed.
   
   Any thoughts here?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] yihua commented on a change in pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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



##########
File path: hudi-common/src/main/java/org/apache/hudi/common/model/PartialUpdateWithLatestAvroPayload.java
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.model;
+
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro;
+
+/**
+ * The only difference with {@link DefaultHoodieRecordPayload} is that support update partial fields
+ * in latest record to old record instead of all fields.
+ */
+public class PartialUpdateWithLatestAvroPayload extends DefaultHoodieRecordPayload {
+
+  public PartialUpdateWithLatestAvroPayload(GenericRecord record, Comparable orderingVal) {
+    super(record, orderingVal);
+  }
+
+  @Override
+  public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema, Properties properties) throws IOException {
+    if (recordBytes.length == 0) {
+      return Option.of(currentValue);
+    }
+
+    GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
+
+    // Null check is needed here to support schema evolution. The record in storage may be from old schema where
+    // the new ordering column might not be present and hence returns null.
+    if (!needUpdatingPersistedRecord(currentValue, incomingRecord, properties)) {
+      return Option.of(currentValue);
+    }
+
+    if (isDeleteRecord(incomingRecord)) {
+      return Option.empty();
+    }
+
+    GenericRecord currentRecord = (GenericRecord) currentValue;
+    // The field num in updated record may be less than old record, so only update these partial fields to old record.
+    List<Schema.Field> fields = schema.getFields();
+    fields.forEach(field -> {
+      Object value = incomingRecord.get(field.name());
+      if (Objects.nonNull(value)) {
+        currentRecord.put(field.name(), value);
+      }

Review comment:
       The difference compared to DefaultHoodieRecordPayload is that only if the corresponding field has a value, it overrides the field value in the existing record, instead of overriding it to null.  Is it correct?  The docs above are a bit confusing.




-- 
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 removed a comment on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


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


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] lsyldliu commented on pull request #4141: [HUDI-2815] Support partial update for streaming change logs

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


   @hudi-bot run azure


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org