nbalajee commented on code in PR #8609:
URL: https://github.com/apache/hudi/pull/8609#discussion_r1185552355


##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -401,6 +427,8 @@ private static void modify(FileSystem fs, Path 
metadataFolder, Properties modify
           throw new HoodieIOException("Checksum property missing or does not 
match.");
         }
       }
+
+      // 5. delete the backup properties file
       fs.delete(backupCfgPath, false);

Review Comment:
   Optionally, we could remove the delete of the backupfile, so that the 
previous version of hoodie.properties would be available as backup.  Helps with 
debugging, on what properties changed recently.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -323,22 +326,43 @@ public HoodieTableConfig() {
     super();
   }
 
-  private void fetchConfigs(FileSystem fs, String metaPath) throws IOException 
{
+  private static TypedProperties fetchConfigs(FileSystem fs, String metaPath) 
throws IOException {
     Path cfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
-    try (FSDataInputStream is = fs.open(cfgPath)) {
-      props.load(is);
-    } catch (IOException ioe) {
-      if (!fs.exists(cfgPath)) {
-        LOG.warn("Run `table recover-configs` if config update/delete failed 
midway. Falling back to backed up configs.");
-        // try the backup. this way no query ever fails if update fails midway.
-        Path backupCfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE_BACKUP);
-        try (FSDataInputStream is = fs.open(backupCfgPath)) {
+    Path backupCfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE_BACKUP);
+    int readRetryCount = 0;
+    boolean found = false;
+
+    TypedProperties props = new TypedProperties();
+    while (readRetryCount++ < MAX_READ_RETRIES) {
+      for (Path path : Arrays.asList(cfgPath, backupCfgPath)) {
+        // Read the properties and validate that it is a valid file
+        try (FSDataInputStream is = fs.open(path)) {
+          props.clear();
           props.load(is);
+          found = true;
+          ValidationUtils.checkArgument(props.containsKey(TYPE) && 
props.containsKey(NAME));
+          return props;
+        } catch (IOException e) {
+          LOG.warn(String.format("Could not read properties from %s: %s", 
path, e));
+        } catch (IllegalArgumentException e) {

Review Comment:
   When one writer is updating the hoodie.properties, another process is 
reading the file we could encounter illegalArgumentException, due to missing 
properties (partial hoodie.properties file).  By catching the exception here, 
we could try reading the backup file (for loop).  
   
   If reading from both hoodie.properties and backup failed, we would throw the 
exception later.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -374,25 +398,27 @@ private static void modify(FileSystem fs, Path 
metadataFolder, Properties modify
       // 0. do any recovery from prior attempts.
       recoverIfNeeded(fs, cfgPath, backupCfgPath);
 
-      // 1. backup the existing properties.
-      try (FSDataInputStream in = fs.open(cfgPath);
-           FSDataOutputStream out = fs.create(backupCfgPath, false)) {
-        FileIOUtils.copy(in, out);
+      // 1. Read the existing config
+      TypedProperties props = fetchConfigs(fs, metadataFolder.toString());
+
+      // 2. backup the existing properties.
+      try (FSDataOutputStream out = fs.create(backupCfgPath, false)) {
+        storeProperties(props, out);
       }
-      /// 2. delete the properties file, reads will go to the backup, until we 
are done.
+
+      // 3. 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.
+
+      // 4. Upsert and save back.
       String checksum;
-      try (FSDataInputStream in = fs.open(backupCfgPath);
-           FSDataOutputStream out = fs.create(cfgPath, true)) {
-        Properties props = new TypedProperties();
-        props.load(in);
+      try (FSDataOutputStream out = fs.create(cfgPath, true)) {

Review Comment:
   To avoid/reduce race windows, when multiple process are involved.



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