1
0

[HUDI-3290] Different file formats for the partition metadata file. (#5179)

* [HUDI-3290] Different file formats for the partition metadata file.

Partition metadata files are stored in each partition to help identify the base path of a table. These files are saved in the properties file format. Some query engines do not work when non Parquet/ORC files are found in a partition.

Added a new table config 'hoodie.partition.metafile.use.data.format' which when enabled (default false for backward compatibility) ensures that partition metafiles will be saved in the same format as the base files of a dataset.

For new datasets, the config can be set via hudi-cli. Deltastreamer has a new parameter --partition-metafile-use-data-format which will create a table with this setting.

* Code review comments

- Adding a new command to migrate from text to base file formats for meta file.
- Reimplementing readFromFS() to first read the text format, then base format
- Avoid extra exists() checks in readFromFS()
- Added unit tests, enabled parquet format across hoodie-hadoop-mr
- Code cleanup, restructuring, naming consistency.

* Wiring in all the other Spark code paths to respect this config

 - Turned on parquet meta format for COW data source tests
 - Removed the deltastreamer command line to keep it shorter

* populate HoodiePartitionMetadata#format after readFromFS()

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
Prashant Wason
2022-04-04 08:08:20 -07:00
committed by GitHub
parent 8add740d22
commit b28f0d6ceb
33 changed files with 544 additions and 94 deletions

View File

@@ -83,6 +83,8 @@ public class HoodieTableHeaderFields {
public static final String HEADER_HOODIE_PROPERTY = "Property";
public static final String HEADER_OLD_VALUE = "Old Value";
public static final String HEADER_NEW_VALUE = "New Value";
public static final String HEADER_TEXT_METAFILE_PRESENT = "Text Metafile present ?";
public static final String HEADER_BASE_METAFILE_PRESENT = "Base Metafile present ?";
/**
* Fields of Savepoints.

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableConfig;
@@ -31,11 +32,13 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.AvroRuntimeException;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.util.StringUtils;
import org.apache.log4j.Logger;
import org.apache.spark.launcher.SparkLauncher;
import org.apache.spark.util.Utils;
@@ -133,7 +136,8 @@ public class RepairsCommand implements CommandMarker {
row[1] = "No";
if (!dryRun) {
HoodiePartitionMetadata partitionMetadata =
new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath, partitionPath);
new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath, partitionPath,
client.getTableConfig().getPartitionMetafileFormat());
partitionMetadata.trySave(0);
row[2] = "Repaired";
}
@@ -199,4 +203,64 @@ public class RepairsCommand implements CommandMarker {
}
});
}
@CliCommand(value = "repair migrate-partition-meta", help = "Migrate all partition meta file currently stored in text format "
+ "to be stored in base file format. See HoodieTableConfig#PARTITION_METAFILE_USE_DATA_FORMAT.")
public String migratePartitionMeta(
@CliOption(key = {"dryrun"}, help = "dry run without modifying anything.", unspecifiedDefaultValue = "true") final boolean dryRun)
throws IOException {
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(HoodieCLI.conf);
HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
List<String> partitionPaths = FSUtils.getAllPartitionPaths(engineContext, client.getBasePath(), false, false);
Path basePath = new Path(client.getBasePath());
String[][] rows = new String[partitionPaths.size()][];
int ind = 0;
for (String partitionPath : partitionPaths) {
Path partition = FSUtils.getPartitionPath(client.getBasePath(), partitionPath);
Option<Path> textFormatFile = HoodiePartitionMetadata.textFormatMetaPathIfExists(HoodieCLI.fs, partition);
Option<Path> baseFormatFile = HoodiePartitionMetadata.baseFormatMetaPathIfExists(HoodieCLI.fs, partition);
String latestCommit = client.getActiveTimeline().getCommitTimeline().lastInstant().get().getTimestamp();
String[] row = new String[] {
partitionPath,
String.valueOf(textFormatFile.isPresent()),
String.valueOf(baseFormatFile.isPresent()),
textFormatFile.isPresent() ? "MIGRATE" : "NONE"
};
if (!dryRun) {
if (!baseFormatFile.isPresent()) {
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(HoodieCLI.fs, latestCommit, basePath, partition,
Option.of(client.getTableConfig().getBaseFileFormat()));
partitionMetadata.trySave(0);
}
// delete it, in case we failed midway last time.
textFormatFile.ifPresent(path -> {
try {
HoodieCLI.fs.delete(path, false);
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
row[3] = "MIGRATED";
}
rows[ind++] = row;
}
Properties props = new Properties();
props.setProperty(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key(), "true");
HoodieTableConfig.update(HoodieCLI.fs, new Path(client.getMetaPath()), props);
return HoodiePrintHelper.print(new String[] {
HoodieTableHeaderFields.HEADER_PARTITION_PATH,
HoodieTableHeaderFields.HEADER_TEXT_METAFILE_PRESENT,
HoodieTableHeaderFields.HEADER_BASE_METAFILE_PRESENT,
HoodieTableHeaderFields.HEADER_ACTION
}, rows);
}
}