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


##########
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)) {

Review Comment:
   Using a list and return here is much cleaner, brilliant



##########
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) {
+          LOG.warn(String.format("Invalid properties file %s: %s", path, 
props));
         }
-      } else {
-        throw ioe;
+      }
+
+      // Failed to read all files so wait before retrying. This can happen in 
cases of parallel updates to the properties.
+      try {
+        Thread.sleep(READ_RETRY_DELAY_MSEC);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting");
       }
     }
+
+    // If we are here than after all retries either no hoodie.properties was 
found or only an invalid file was found.
+    if (found) {
+      throw new IllegalArgumentException("hoodie.properties file seems 
invalid. Please check for left over `.updated` files if any, manually copy it 
to hoodie.properties and retry");
+    } else {
+      throw new HoodieIOException("Failed to read hoodie properties");
+    }

Review Comment:
   If `IllegalArgumentException` can be handled earlier, then we won't have to 
have this logic here as well



##########
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:
   I think we should let `IllegalArgumentException` thrown instead of catching 
it here and throw later. If the config passed in is invalid then Hudi would 
throw it anyway after 5 sec retry



##########
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) {
+          LOG.warn(String.format("Invalid properties file %s: %s", path, 
props));
         }
-      } else {
-        throw ioe;
+      }
+
+      // Failed to read all files so wait before retrying. This can happen in 
cases of parallel updates to the properties.
+      try {
+        Thread.sleep(READ_RETRY_DELAY_MSEC);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted while waiting");
       }
     }
+
+    // If we are here than after all retries either no hoodie.properties was 
found or only an invalid file was found.

Review Comment:
   typo: than -> then



##########
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:
   Just curious: why don't we reload props here anymore



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