[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:
@@ -117,8 +117,8 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
|
||||
|
||||
// also need to copy over partition metadata
|
||||
Path partitionMetaFile =
|
||||
new Path(FSUtils.getPartitionPath(baseDir, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
|
||||
Path partitionMetaFile = HoodiePartitionMetadata.getPartitionMetafilePath(fs1,
|
||||
FSUtils.getPartitionPath(baseDir, partition)).get();
|
||||
if (fs1.exists(partitionMetaFile)) {
|
||||
filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
|
||||
}
|
||||
|
||||
@@ -206,9 +206,9 @@ public class HoodieSnapshotExporter {
|
||||
Stream<HoodieBaseFile> dataFiles = fsView.getLatestBaseFilesBeforeOrOn(partition, latestCommitTimestamp);
|
||||
dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
|
||||
// also need to copy over partition metadata
|
||||
Path partitionMetaFile =
|
||||
new Path(FSUtils.getPartitionPath(cfg.sourceBasePath, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
|
||||
FileSystem fs = FSUtils.getFs(cfg.sourceBasePath, serConf.newCopy());
|
||||
Path partitionMetaFile = HoodiePartitionMetadata.getPartitionMetafilePath(fs,
|
||||
FSUtils.getPartitionPath(cfg.sourceBasePath, partition)).get();
|
||||
if (fs.exists(partitionMetaFile)) {
|
||||
filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
|
||||
}
|
||||
|
||||
@@ -185,6 +185,7 @@ public class BootstrapExecutor implements Serializable {
|
||||
}
|
||||
}
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.fromProperties(props)
|
||||
.setTableType(cfg.tableType)
|
||||
.setTableName(cfg.targetTableName)
|
||||
.setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue())
|
||||
|
||||
@@ -278,6 +278,8 @@ public class DeltaSync implements Serializable {
|
||||
.setKeyGeneratorClassProp(props.getProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
|
||||
SimpleKeyGenerator.class.getName()))
|
||||
.setPreCombineField(cfg.sourceOrderingField)
|
||||
.setPartitionMetafileUseBaseFormat(props.getBoolean(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key(),
|
||||
HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.defaultValue()))
|
||||
.initTable(new Configuration(jssc.hadoopConfiguration()),
|
||||
cfg.targetBasePath);
|
||||
}
|
||||
@@ -371,6 +373,8 @@ public class DeltaSync implements Serializable {
|
||||
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()))
|
||||
.setKeyGeneratorClassProp(props.getProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
|
||||
SimpleKeyGenerator.class.getName()))
|
||||
.setPartitionMetafileUseBaseFormat(props.getBoolean(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key(),
|
||||
HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.defaultValue()))
|
||||
.initTable(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath);
|
||||
}
|
||||
|
||||
|
||||
@@ -468,7 +468,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
compactSchedulingMinShare, clusterSchedulingMinShare, forceDisableCompaction, checkpoint,
|
||||
initialCheckpointProvider, help);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Config{"
|
||||
|
||||
Reference in New Issue
Block a user