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

@@ -27,6 +27,7 @@ import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.schema.MessageType;
import java.util.HashMap;
import java.util.Map;
/**
* Wrap AvroWriterSupport for plugging in the bloom filter.
@@ -36,6 +37,7 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport {
private Option<BloomFilter> bloomFilterOpt;
private String minRecordKey;
private String maxRecordKey;
private Map<String, String> footerMetadata = new HashMap<>();
public static final String OLD_HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY = "com.uber.hoodie.bloomfilter";
public static final String HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY = "org.apache.hudi.bloomfilter";
@@ -50,18 +52,17 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport {
@Override
public WriteSupport.FinalizedWriteContext finalizeWrite() {
HashMap<String, String> extraMetaData = new HashMap<>();
if (bloomFilterOpt.isPresent()) {
extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilterOpt.get().serializeToString());
footerMetadata.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilterOpt.get().serializeToString());
if (minRecordKey != null && maxRecordKey != null) {
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey);
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey);
footerMetadata.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey);
footerMetadata.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey);
}
if (bloomFilterOpt.get().getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) {
extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilterOpt.get().getBloomFilterTypeCode().name());
footerMetadata.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilterOpt.get().getBloomFilterTypeCode().name());
}
}
return new WriteSupport.FinalizedWriteContext(extraMetaData);
return new WriteSupport.FinalizedWriteContext(footerMetadata);
}
public void add(String recordKey) {
@@ -80,4 +81,8 @@ public class HoodieAvroWriteSupport extends AvroWriteSupport {
}
}
}
public void addFooterMetadata(String key, String value) {
footerMetadata.put(key, value);
}
}

View File

@@ -246,7 +246,7 @@ public class FSUtils {
final List<String> partitions = new ArrayList<>();
processFiles(fs, basePathStr, (locatedFileStatus) -> {
Path filePath = locatedFileStatus.getPath();
if (filePath.getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) {
if (filePath.getName().startsWith(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX)) {
partitions.add(getRelativePartitionPath(basePath, filePath.getParent()));
}
return true;

View File

@@ -18,27 +18,47 @@
package org.apache.hudi.common.model;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.util.AvroOrcUtils;
import org.apache.hudi.common.util.BaseFileUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.orc.OrcFile;
import org.apache.orc.Writer;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
import org.apache.parquet.schema.Types;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* The metadata that goes into the meta file in each partition.
*/
public class HoodiePartitionMetadata {
public static final String HOODIE_PARTITION_METAFILE = ".hoodie_partition_metadata";
public static final String PARTITION_DEPTH_KEY = "partitionDepth";
public static final String HOODIE_PARTITION_METAFILE_PREFIX = ".hoodie_partition_metadata";
public static final String COMMIT_TIME_KEY = "commitTime";
private static final String PARTITION_DEPTH_KEY = "partitionDepth";
private static final Logger LOG = LogManager.getLogger(HoodiePartitionMetadata.class);
/**
* Contents of the metadata.
@@ -52,7 +72,8 @@ public class HoodiePartitionMetadata {
private final FileSystem fs;
private static final Logger LOG = LogManager.getLogger(HoodiePartitionMetadata.class);
// The format in which to write the partition metadata
private Option<HoodieFileFormat> format;
/**
* Construct metadata from existing partition.
@@ -61,13 +82,15 @@ public class HoodiePartitionMetadata {
this.fs = fs;
this.props = new Properties();
this.partitionPath = partitionPath;
this.format = Option.empty();
}
/**
* Construct metadata object to be written out.
*/
public HoodiePartitionMetadata(FileSystem fs, String instantTime, Path basePath, Path partitionPath) {
public HoodiePartitionMetadata(FileSystem fs, String instantTime, Path basePath, Path partitionPath, Option<HoodieFileFormat> format) {
this(fs, partitionPath);
this.format = format;
props.setProperty(COMMIT_TIME_KEY, instantTime);
props.setProperty(PARTITION_DEPTH_KEY, String.valueOf(partitionPath.depth() - basePath.depth()));
}
@@ -83,21 +106,17 @@ public class HoodiePartitionMetadata {
* Write the metadata safely into partition atomically.
*/
public void trySave(int taskPartitionId) {
String extension = getMetafileExtension();
Path tmpMetaPath =
new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE + "_" + taskPartitionId);
Path metaPath = new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX + "_" + taskPartitionId + extension);
Path metaPath = new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX + extension);
boolean metafileExists = false;
try {
metafileExists = fs.exists(metaPath);
if (!metafileExists) {
// write to temporary file
FSDataOutputStream os = fs.create(tmpMetaPath, true);
props.store(os, "partition metadata");
os.hsync();
os.hflush();
os.close();
writeMetafile(tmpMetaPath);
// move to actual path
fs.rename(tmpMetaPath, metaPath);
}
@@ -118,22 +137,103 @@ public class HoodiePartitionMetadata {
}
}
private String getMetafileExtension() {
// To be backwards compatible, there is no extension to the properties file base partition metafile
return format.isPresent() ? format.get().getFileExtension() : StringUtils.EMPTY_STRING;
}
/**
* Write the partition metadata in the correct format in the given file path.
*
* @param filePath Path of the file to write
* @throws IOException
*/
private void writeMetafile(Path filePath) throws IOException {
if (format.isPresent()) {
Schema schema = HoodieAvroUtils.getRecordKeySchema();
switch (format.get()) {
case PARQUET:
// Since we are only interested in saving metadata to the footer, the schema, blocksizes and other
// parameters are not important.
MessageType type = Types.buildMessage().optional(PrimitiveTypeName.INT64).named("dummyint").named("dummy");
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(type, schema, Option.empty());
try (ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.UNCOMPRESSED, 1024, 1024)) {
for (String key : props.stringPropertyNames()) {
writeSupport.addFooterMetadata(key, props.getProperty(key));
}
}
break;
case ORC:
// Since we are only interested in saving metadata to the footer, the schema, blocksizes and other
// parameters are not important.
OrcFile.WriterOptions writerOptions = OrcFile.writerOptions(fs.getConf()).fileSystem(fs)
.setSchema(AvroOrcUtils.createOrcSchema(schema));
try (Writer writer = OrcFile.createWriter(filePath, writerOptions)) {
for (String key : props.stringPropertyNames()) {
writer.addUserMetadata(key, ByteBuffer.wrap(props.getProperty(key).getBytes()));
}
}
break;
default:
throw new HoodieException("Unsupported format for partition metafiles: " + format.get());
}
} else {
// Backwards compatible properties file format
FSDataOutputStream os = fs.create(filePath, true);
props.store(os, "partition metadata");
os.hsync();
os.hflush();
os.close();
}
}
/**
* Read out the metadata for this partition.
*/
public void readFromFS() throws IOException {
FSDataInputStream is = null;
try {
Path metaFile = new Path(partitionPath, HOODIE_PARTITION_METAFILE);
is = fs.open(metaFile);
// first try reading the text format (legacy, currently widespread)
boolean readFile = readTextFormatMetaFile();
if (!readFile) {
// now try reading the base file formats.
readFile = readBaseFormatMetaFile();
}
// throw exception.
if (!readFile) {
throw new HoodieException("Unable to read any partition meta file to locate the table timeline.");
}
}
private boolean readTextFormatMetaFile() {
// Properties file format
Path metafilePath = textFormatMetaFilePath(partitionPath);
try (FSDataInputStream is = fs.open(metafilePath)) {
props.load(is);
} catch (IOException ioe) {
throw new HoodieException("Error reading Hoodie partition metadata for " + partitionPath, ioe);
} finally {
if (is != null) {
is.close();
format = Option.empty();
return true;
} catch (Throwable t) {
LOG.warn("Unable to read partition meta properties file for partition " + partitionPath, t);
return false;
}
}
private boolean readBaseFormatMetaFile() {
for (Path metafilePath : baseFormatMetaFilePaths(partitionPath)) {
try {
BaseFileUtils reader = BaseFileUtils.getInstance(metafilePath.toString());
// Data file format
Map<String, String> metadata = reader.readFooter(fs.getConf(), true, metafilePath, PARTITION_DEPTH_KEY, COMMIT_TIME_KEY);
props.clear();
props.putAll(metadata);
format = Option.of(reader.getFormat());
return true;
} catch (Throwable t) {
// any error, log, check the next base format
LOG.warn("Unable to read partition metadata " + metafilePath.getName() + " for partition " + partitionPath, t);
}
}
return false;
}
/**
@@ -141,12 +241,10 @@ public class HoodiePartitionMetadata {
*/
public Option<String> readPartitionCreatedCommitTime() {
try {
if (props.containsKey(COMMIT_TIME_KEY)) {
return Option.of(props.getProperty(COMMIT_TIME_KEY));
} else {
if (!props.containsKey(COMMIT_TIME_KEY)) {
readFromFS();
return Option.of(props.getProperty(COMMIT_TIME_KEY));
}
return Option.of(props.getProperty(COMMIT_TIME_KEY));
} catch (IOException ioe) {
LOG.warn("Error fetch Hoodie partition metadata for " + partitionPath, ioe);
return Option.empty();
@@ -156,9 +254,55 @@ public class HoodiePartitionMetadata {
// methods related to partition meta data
public static boolean hasPartitionMetadata(FileSystem fs, Path partitionPath) {
try {
return fs.exists(new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE));
return textFormatMetaPathIfExists(fs, partitionPath).isPresent()
|| baseFormatMetaPathIfExists(fs, partitionPath).isPresent();
} catch (IOException ioe) {
throw new HoodieIOException("Error checking presence of partition meta file for " + partitionPath, ioe);
}
}
/**
* Returns the name of the partition metadata.
*
* @return Name of the partition metafile or empty option
*/
public static Option<Path> getPartitionMetafilePath(FileSystem fs, Path partitionPath) {
// The partition listing is a costly operation so instead we are searching for existence of the files instead.
// This is in expected order as properties file based partition metafiles should be the most common.
try {
Option<Path> textFormatPath = textFormatMetaPathIfExists(fs, partitionPath);
if (textFormatPath.isPresent()) {
return textFormatPath;
} else {
return baseFormatMetaPathIfExists(fs, partitionPath);
}
} catch (IOException ioe) {
throw new HoodieException("Error checking Hoodie partition metadata for " + partitionPath, ioe);
}
}
public static Option<Path> baseFormatMetaPathIfExists(FileSystem fs, Path partitionPath) throws IOException {
// Parquet should be more common than ORC so check it first
for (Path metafilePath : baseFormatMetaFilePaths(partitionPath)) {
if (fs.exists(metafilePath)) {
return Option.of(metafilePath);
}
}
return Option.empty();
}
public static Option<Path> textFormatMetaPathIfExists(FileSystem fs, Path partitionPath) throws IOException {
Path path = textFormatMetaFilePath(partitionPath);
return Option.ofNullable(fs.exists(path) ? path : null);
}
static Path textFormatMetaFilePath(Path partitionPath) {
return new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX);
}
static List<Path> baseFormatMetaFilePaths(Path partitionPath) {
return Stream.of(HoodieFileFormat.PARQUET.getFileExtension(), HoodieFileFormat.ORC.getFileExtension())
.map(ext -> new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX + ext))
.collect(Collectors.toList());
}
}

View File

@@ -190,6 +190,12 @@ public class HoodieTableConfig extends HoodieConfig {
.defaultValue(HoodieTimelineTimeZone.LOCAL)
.withDocumentation("User can set hoodie commit timeline timezone, such as utc, local and so on. local is default");
public static final ConfigProperty<Boolean> PARTITION_METAFILE_USE_BASE_FORMAT = ConfigProperty
.key("hoodie.partition.metafile.use.base.format")
.defaultValue(false)
.withDocumentation("If true, partition metafiles are saved in the same format as basefiles for this dataset (e.g. Parquet / ORC). "
+ "If false (default) partition metafiles are saved as properties files.");
public static final ConfigProperty<String> URL_ENCODE_PARTITIONING = KeyGeneratorOptions.URL_ENCODE_PARTITIONING;
public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_ENABLE = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE;
@@ -608,6 +614,16 @@ public class HoodieTableConfig extends HoodieConfig {
return getStringOrDefault(TABLE_METADATA_PARTITIONS, StringUtils.EMPTY_STRING);
}
/**
* Returns the format to use for partition meta files.
*/
public Option<HoodieFileFormat> getPartitionMetafileFormat() {
if (getBooleanOrDefault(PARTITION_METAFILE_USE_BASE_FORMAT)) {
return Option.of(getBaseFileFormat());
}
return Option.empty();
}
public Map<String, String> propsMap() {
return props.entrySet().stream()
.collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));

View File

@@ -699,6 +699,7 @@ public class HoodieTableMetaClient implements Serializable {
private Boolean hiveStylePartitioningEnable;
private Boolean urlEncodePartitioning;
private HoodieTimelineTimeZone commitTimeZone;
private Boolean partitionMetafileUseBaseFormat;
/**
* Persist the configs that is written at the first time, and should not be changed.
@@ -813,6 +814,11 @@ public class HoodieTableMetaClient implements Serializable {
return this;
}
public PropertyBuilder setPartitionMetafileUseBaseFormat(Boolean useBaseFormat) {
this.partitionMetafileUseBaseFormat = useBaseFormat;
return this;
}
public PropertyBuilder set(String key, Object value) {
if (HoodieTableConfig.PERSISTED_CONFIG_LIST.contains(key)) {
this.others.put(key, value);
@@ -908,6 +914,9 @@ public class HoodieTableMetaClient implements Serializable {
if (hoodieConfig.contains(HoodieTableConfig.URL_ENCODE_PARTITIONING)) {
setUrlEncodePartitioning(hoodieConfig.getBoolean(HoodieTableConfig.URL_ENCODE_PARTITIONING));
}
if (hoodieConfig.contains(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT)) {
setPartitionMetafileUseBaseFormat(hoodieConfig.getBoolean(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT));
}
return this;
}
@@ -986,6 +995,9 @@ public class HoodieTableMetaClient implements Serializable {
if (null != commitTimeZone) {
tableConfig.setValue(HoodieTableConfig.TIMELINE_TIMEZONE, commitTimeZone.toString());
}
if (null != partitionMetafileUseBaseFormat) {
tableConfig.setValue(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT, partitionMetafileUseBaseFormat.toString());
}
return tableConfig.getProps();
}

View File

@@ -18,14 +18,6 @@
package org.apache.hudi.common.util;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.BloomFilterFactory;
@@ -36,6 +28,16 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.keygen.BaseKeyGenerator;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
public abstract class BaseFileUtils {
public static BaseFileUtils getInstance(String path) {
@@ -204,4 +206,9 @@ public abstract class BaseFileUtils {
* @return The Avro schema of the data file
*/
public abstract Schema readAvroSchema(Configuration configuration, Path filePath);
/**
* @return The subclass's {@link HoodieFileFormat}.
*/
public abstract HoodieFileFormat getFormat();
}

View File

@@ -18,6 +18,29 @@
package org.apache.hudi.common.util;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.MetadataNotFoundException;
import org.apache.hudi.keygen.BaseKeyGenerator;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.orc.OrcFile;
import org.apache.orc.OrcProto.UserMetadataItem;
import org.apache.orc.Reader;
import org.apache.orc.Reader.Options;
import org.apache.orc.RecordReader;
import org.apache.orc.TypeDescription;
import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@@ -28,27 +51,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.MetadataNotFoundException;
import org.apache.hudi.keygen.BaseKeyGenerator;
import org.apache.orc.OrcFile;
import org.apache.orc.OrcProto.UserMetadataItem;
import org.apache.orc.Reader;
import org.apache.orc.Reader.Options;
import org.apache.orc.RecordReader;
import org.apache.orc.TypeDescription;
/**
* Utility functions for ORC files.
@@ -248,6 +250,11 @@ public class OrcUtils extends BaseFileUtils {
}
}
@Override
public HoodieFileFormat getFormat() {
return HoodieFileFormat.ORC;
}
@Override
public long getRowCount(Configuration conf, Path orcFilePath) {
try (Reader reader = OrcFile.createReader(orcFilePath, OrcFile.readerOptions(conf))) {

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.common.util;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieColumnRangeMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.exception.HoodieIOException;
@@ -228,6 +229,11 @@ public class ParquetUtils extends BaseFileUtils {
return new AvroSchemaConverter(configuration).convert(parquetSchema);
}
@Override
public HoodieFileFormat getFormat() {
return HoodieFileFormat.PARQUET;
}
/**
* NOTE: This literally reads the entire file contents, thus should be used with caution.
*/

View File

@@ -96,7 +96,7 @@ public class FileSystemBackedTableMetadata implements HoodieTableMetadata {
} else if (!fileStatus.getPath().getName().equals(HoodieTableMetaClient.METAFOLDER_NAME)) {
pathsToList.add(fileStatus.getPath());
}
} else if (fileStatus.getPath().getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) {
} else if (fileStatus.getPath().getName().startsWith(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX)) {
String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetBasePath), fileStatus.getPath().getParent());
partitionPaths.add(partitionName);
}