1
0

[HUDI-4138] Fix the concurrency modification of hoodie table config for flink (#5660)

* Remove the metadata cleaning strategy for flink, that means the multi-modal index may be affected
* Improve the HoodieTable#clearMetadataTablePartitionsConfig to only update table config when necessary
* Remove the modification of read code path in HoodieTableConfig
This commit is contained in:
Danny Chan
2022-05-24 13:07:55 +08:00
committed by GitHub
parent af1128acf9
commit 676d5cefe0
4 changed files with 13 additions and 28 deletions

View File

@@ -272,8 +272,8 @@ public class HoodieTableConfig extends HoodieConfig {
* @throws IOException
*/
private static String storeProperties(Properties props, FSDataOutputStream outputStream) throws IOException {
String checksum;
if (props.containsKey(TABLE_CHECKSUM.key()) && validateChecksum(props)) {
final String checksum;
if (isValidChecksum(props)) {
checksum = props.getProperty(TABLE_CHECKSUM.key());
props.store(outputStream, "Updated at " + Instant.now());
} else {
@@ -285,8 +285,8 @@ public class HoodieTableConfig extends HoodieConfig {
return checksum;
}
private boolean isValidChecksum() {
return contains(TABLE_CHECKSUM) && validateChecksum(props);
private static boolean isValidChecksum(Properties props) {
return props.containsKey(TABLE_CHECKSUM.key()) && validateChecksum(props);
}
/**
@@ -298,20 +298,13 @@ public class HoodieTableConfig extends HoodieConfig {
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);
}
}
} 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)) {
props.load(is);
}
@@ -631,7 +624,7 @@ public class HoodieTableConfig extends HoodieConfig {
CONFIG_VALUES_DELIMITER
);
}
/**
* Returns the format to use for partition meta files.
*/