nsivabalan commented on a change in pull request #4712:
URL: https://github.com/apache/hudi/pull/4712#discussion_r794995087



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
##########
@@ -272,15 +287,29 @@ private static void modify(FileSystem fs, Path 
metadataFolder, Properties modify
       /// 2. delete the properties file, reads will go to the backup, until we 
are done.
       fs.delete(cfgPath, false);
       // 3. read current props, upsert and save back.
+      String checksum;
       try (FSDataInputStream in = fs.open(backupCfgPath);
            FSDataOutputStream out = fs.create(cfgPath, true)) {
-        Properties props = new Properties();
+        Properties props = new TypedProperties();
         props.load(in);
         modifyFn.accept(props, modifyProps);
+        if (props.containsKey(TABLE_CHECKSUM.key()) && 
validateChecksum(props)) {
+          checksum = props.getProperty(TABLE_CHECKSUM.key());
+        } else {
+          checksum = String.valueOf(generateChecksum(props));
+        }
+        props.setProperty(TABLE_CHECKSUM.key(), checksum);

Review comment:
       shouldn't we move this to else block only. 

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
##########
@@ -493,6 +544,13 @@ public String getUrlEncodePartitioning() {
     return getString(URL_ENCODE_PARTITIONING);
   }
 
+  /**
+   * Read the table checksum.
+   */
+  public Long getTableChecksum() {

Review comment:
       why public ? 

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
##########
@@ -272,15 +287,29 @@ private static void modify(FileSystem fs, Path 
metadataFolder, Properties modify
       /// 2. delete the properties file, reads will go to the backup, until we 
are done.
       fs.delete(cfgPath, false);
       // 3. read current props, upsert and save back.
+      String checksum;
       try (FSDataInputStream in = fs.open(backupCfgPath);
            FSDataOutputStream out = fs.create(cfgPath, true)) {
-        Properties props = new Properties();
+        Properties props = new TypedProperties();
         props.load(in);
         modifyFn.accept(props, modifyProps);
+        if (props.containsKey(TABLE_CHECKSUM.key()) && 
validateChecksum(props)) {
+          checksum = props.getProperty(TABLE_CHECKSUM.key());
+        } else {
+          checksum = String.valueOf(generateChecksum(props));
+        }
+        props.setProperty(TABLE_CHECKSUM.key(), checksum);
         props.store(out, "Updated at " + System.currentTimeMillis());
       }
       // 4. verify and remove backup.
       // FIXME(vc): generate a hash for verification.
+      try (FSDataInputStream in = fs.open(cfgPath)) {
+        Properties props = new TypedProperties();
+        props.load(in);
+        if (!props.containsKey(TABLE_CHECKSUM.key()) || 
!props.getProperty(TABLE_CHECKSUM.key()).equals(checksum)) {
+          throw new HoodieIOException("Checksum property missing or does not 
match.");

Review comment:
       is it not possible to regenerate from backup rather than failing here? 

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
##########
@@ -272,15 +287,29 @@ private static void modify(FileSystem fs, Path 
metadataFolder, Properties modify
       /// 2. delete the properties file, reads will go to the backup, until we 
are done.
       fs.delete(cfgPath, false);
       // 3. read current props, upsert and save back.
+      String checksum;
       try (FSDataInputStream in = fs.open(backupCfgPath);
            FSDataOutputStream out = fs.create(cfgPath, true)) {
-        Properties props = new Properties();
+        Properties props = new TypedProperties();
         props.load(in);
         modifyFn.accept(props, modifyProps);
+        if (props.containsKey(TABLE_CHECKSUM.key()) && 
validateChecksum(props)) {
+          checksum = props.getProperty(TABLE_CHECKSUM.key());
+        } else {
+          checksum = String.valueOf(generateChecksum(props));
+        }
+        props.setProperty(TABLE_CHECKSUM.key(), checksum);
         props.store(out, "Updated at " + System.currentTimeMillis());
       }
       // 4. verify and remove backup.
       // FIXME(vc): generate a hash for verification.
+      try (FSDataInputStream in = fs.open(cfgPath)) {

Review comment:
       remove L 305




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to