[HUDI-2795] Add mechanism to safely update,delete and recover table properties (#4038)
* [HUDI-2795] Add mechanism to safely update,delete and recover table properties - Fail safe mechanism, that lets queries succeed off a backup file - Readers who are not upgraded to this version of code will just fail until recovery is done. - Added unit tests that exercises all these scenarios. - Adding CLI for recovery, updation to table command. - [Pending] Add some hash based verfication to ensure any rare partial writes for HDFS * Fixing upgrade/downgrade infrastructure to use new updation method
This commit is contained in:
@@ -29,6 +29,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
@@ -47,6 +48,8 @@ import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.function.BiConsumer;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
@@ -69,6 +72,7 @@ public class HoodieTableConfig extends HoodieConfig {
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieTableConfig.class);
|
||||
|
||||
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
|
||||
public static final String HOODIE_PROPERTIES_FILE_BACKUP = "hoodie.properties.backup";
|
||||
|
||||
public static final ConfigProperty<String> NAME = ConfigProperty
|
||||
.key("hoodie.table.name")
|
||||
@@ -173,12 +177,11 @@ public class HoodieTableConfig extends HoodieConfig {
|
||||
Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
|
||||
LOG.info("Loading table properties from " + propertyPath);
|
||||
try {
|
||||
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
|
||||
props.load(inputStream);
|
||||
}
|
||||
fetchConfigs(fs, metaPath);
|
||||
if (contains(PAYLOAD_CLASS_NAME) && payloadClassName != null
|
||||
&& !getString(PAYLOAD_CLASS_NAME).equals(payloadClassName)) {
|
||||
setValue(PAYLOAD_CLASS_NAME, payloadClassName);
|
||||
// FIXME(vc): wonder if this can be removed. Need to look into history.
|
||||
try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
|
||||
props.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
}
|
||||
@@ -192,16 +195,103 @@ public class HoodieTableConfig extends HoodieConfig {
|
||||
|
||||
/**
|
||||
* For serializing and de-serializing.
|
||||
*
|
||||
*/
|
||||
public HoodieTableConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
private void 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)) {
|
||||
props.load(is);
|
||||
}
|
||||
} else {
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void recover(FileSystem fs, Path metadataFolder) throws IOException {
|
||||
Path cfgPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
Path backupCfgPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE_BACKUP);
|
||||
recoverIfNeeded(fs, cfgPath, backupCfgPath);
|
||||
}
|
||||
|
||||
static void recoverIfNeeded(FileSystem fs, Path cfgPath, Path backupCfgPath) throws IOException {
|
||||
if (!fs.exists(cfgPath)) {
|
||||
// copy over from backup
|
||||
try (FSDataInputStream in = fs.open(backupCfgPath);
|
||||
FSDataOutputStream out = fs.create(cfgPath, false)) {
|
||||
FileIOUtils.copy(in, out);
|
||||
}
|
||||
}
|
||||
// regardless, we don't need the backup anymore.
|
||||
fs.delete(backupCfgPath, false);
|
||||
}
|
||||
|
||||
private static void upsertProperties(Properties current, Properties updated) {
|
||||
updated.forEach((k, v) -> current.setProperty(k.toString(), v.toString()));
|
||||
}
|
||||
|
||||
private static void deleteProperties(Properties current, Properties deleted) {
|
||||
deleted.forEach((k, v) -> current.remove(k.toString()));
|
||||
}
|
||||
|
||||
private static void modify(FileSystem fs, Path metadataFolder, Properties modifyProps, BiConsumer<Properties, Properties> modifyFn) {
|
||||
Path cfgPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
Path backupCfgPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE_BACKUP);
|
||||
try {
|
||||
// 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);
|
||||
}
|
||||
/// 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.
|
||||
try (FSDataInputStream in = fs.open(backupCfgPath);
|
||||
FSDataOutputStream out = fs.create(cfgPath, true)) {
|
||||
Properties props = new Properties();
|
||||
props.load(in);
|
||||
modifyFn.accept(props, modifyProps);
|
||||
props.store(out, "Updated at " + System.currentTimeMillis());
|
||||
}
|
||||
// 4. verify and remove backup.
|
||||
// FIXME(vc): generate a hash for verification.
|
||||
fs.delete(backupCfgPath, false);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Error updating table configs.", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Upserts the table config with the set of properties passed in. We implement a fail-safe backup protocol
|
||||
* here for safely updating with recovery and also ensuring the table config continues to be readable.
|
||||
*/
|
||||
public static void update(FileSystem fs, Path metadataFolder, Properties updatedProps) {
|
||||
modify(fs, metadataFolder, updatedProps, HoodieTableConfig::upsertProperties);
|
||||
}
|
||||
|
||||
public static void delete(FileSystem fs, Path metadataFolder, Set<String> deletedProps) {
|
||||
Properties props = new Properties();
|
||||
deletedProps.forEach(p -> props.setProperty(p, ""));
|
||||
modify(fs, metadataFolder, props, HoodieTableConfig::deleteProperties);
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties).
|
||||
*/
|
||||
public static void createHoodieProperties(FileSystem fs, Path metadataFolder, Properties properties)
|
||||
public static void create(FileSystem fs, Path metadataFolder, Properties properties)
|
||||
throws IOException {
|
||||
if (!fs.exists(metadataFolder)) {
|
||||
fs.mkdirs(metadataFolder);
|
||||
|
||||
@@ -377,7 +377,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
}
|
||||
|
||||
initializeBootstrapDirsIfNotExists(hadoopConf, basePath, fs);
|
||||
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
|
||||
HoodieTableConfig.create(fs, metaPathDir, props);
|
||||
// We should not use fs.getConf as this might be different from the original configuration
|
||||
// used to create the fs in unit tests
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build();
|
||||
|
||||
Reference in New Issue
Block a user