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 2022/05/23 17:21:12 UTC

[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5660: [HUDI-4138] Fix the concurrency modification of hoodie table config f…

alexeykudinkin commented on code in PR #5660:
URL: https://github.com/apache/hudi/pull/5660#discussion_r879700212


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java:
##########
@@ -105,13 +105,9 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con
   public <T extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(String triggeringInstantTimestamp,
                                                                                             Option<T> actionMetadata) {
     if (config.isMetadataTableEnabled()) {
-      // even with metadata enabled, some index could have been disabled
-      // delete metadata partitions corresponding to such indexes
-      deleteMetadataIndexIfNecessary();

Review Comment:
   @codope can you please elaborate on the original intent here?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -285,8 +285,8 @@ private static String storeProperties(Properties props, FSDataOutputStream outpu
     return checksum;
   }
 
-  private boolean isValidChecksum() {
-    return contains(TABLE_CHECKSUM) && validateChecksum(props);
+  private static boolean isValidChecksum(Properties props) {

Review Comment:
   👍 



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java:
##########
@@ -105,13 +105,9 @@ protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext con
   public <T extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(String triggeringInstantTimestamp,
                                                                                             Option<T> actionMetadata) {
     if (config.isMetadataTableEnabled()) {
-      // even with metadata enabled, some index could have been disabled
-      // delete metadata partitions corresponding to such indexes
-      deleteMetadataIndexIfNecessary();
       return Option.of(FlinkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config,
           context, actionMetadata, Option.of(triggeringInstantTimestamp)));
     } else {
-      maybeDeleteMetadataTable();

Review Comment:
   @codope and here



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -298,20 +298,13 @@ public HoodieTableConfig() {
 
   private void fetchConfigs(FileSystem fs, String metaPath) throws IOException {
     Path cfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
-    Path backupCfgPath = new Path(metaPath, HOODIE_PROPERTIES_FILE_BACKUP);
     try (FSDataInputStream is = fs.open(cfgPath)) {
       props.load(is);
-      // validate checksum for latest table version
-      if (getTableVersion().versionCode() >= HoodieTableVersion.FOUR.versionCode() && !isValidChecksum()) {
-        LOG.warn("Checksum validation failed. Falling back to backed up configs.");
-        try (FSDataInputStream fsDataInputStream = fs.open(backupCfgPath)) {
-          props.load(fsDataInputStream);
-        }
-      }

Review Comment:
   @danny0405 but why we'd want to omit the validation?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java:
##########
@@ -885,24 +885,24 @@ private boolean shouldExecuteMetadataTableDeletion() {
     // partitions are ready to use
     return !HoodieTableMetadata.isMetadataTable(metaClient.getBasePath())
         && !config.isMetadataTableEnabled()
-        && (!metaClient.getTableConfig().contains(TABLE_METADATA_PARTITIONS)
-        || !metaClient.getTableConfig().getMetadataPartitions().isEmpty());
+        && !metaClient.getTableConfig().getMetadataPartitions().isEmpty();
   }
 
   /**
    * Clears hoodie.table.metadata.partitions in hoodie.properties
    */
   private void clearMetadataTablePartitionsConfig(Option<MetadataPartitionType> partitionType, boolean clearAll) {
-    if (clearAll) {
+    Set<String> partitions = getCompletedMetadataPartitions(metaClient.getTableConfig());
+    if (clearAll && partitions.size() > 0) {
       LOG.info("Clear hoodie.table.metadata.partitions in hoodie.properties");
       metaClient.getTableConfig().setValue(TABLE_METADATA_PARTITIONS.key(), EMPTY_STRING);
       HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps());
       return;
     }
-    Set<String> completedPartitions = getCompletedMetadataPartitions(metaClient.getTableConfig());
-    completedPartitions.remove(partitionType.get().getPartitionPath());
-    metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions));
-    HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps());
+    if (partitions.remove(partitionType.get().getPartitionPath())) {

Review Comment:
   We can make these conditional `else if` and then abolish return in the one above
   



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