1
0

[HUDI-1591] Implement Spark's FileIndex for Hudi to support queries via Hudi DataSource using non-globbed table path and partition pruning (#2651)

This commit is contained in:
pengzhiwei
2021-04-02 02:12:28 +08:00
committed by GitHub
parent 9804662bc8
commit 684622c7c9
22 changed files with 1074 additions and 82 deletions

View File

@@ -276,6 +276,16 @@ public class FSUtils {
}
}
public static FileStatus[] getFilesInPartition(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
String basePathStr, Path partitionPath) {
try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext,
metadataConfig, basePathStr, FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
return tableMetadata.getAllFilesInPartition(partitionPath);
} catch (Exception e) {
throw new HoodieException("Error get files in partition: " + partitionPath, e);
}
}
public static String getFileExtension(String fullName) {
Objects.requireNonNull(fullName);
String fileName = new File(fullName).getName();

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.common.table;
import java.util.Arrays;
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
@@ -57,6 +58,7 @@ public class HoodieTableConfig implements Serializable {
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
public static final String HOODIE_TABLE_VERSION_PROP_NAME = "hoodie.table.version";
public static final String HOODIE_TABLE_PRECOMBINE_FIELD = "hoodie.table.precombine.field";
public static final String HOODIE_TABLE_PARTITION_COLUMNS = "hoodie.table.partition.columns";
@Deprecated
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format";
@@ -193,6 +195,14 @@ public class HoodieTableConfig implements Serializable {
return props.getProperty(HOODIE_TABLE_PRECOMBINE_FIELD);
}
public Option<String[]> getPartitionColumns() {
if (props.containsKey(HOODIE_TABLE_PARTITION_COLUMNS)) {
return Option.of(Arrays.stream(props.getProperty(HOODIE_TABLE_PARTITION_COLUMNS).split(","))
.filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[]{}));
}
return Option.empty();
}
/**
* Read the payload class for HoodieRecords from the table properties.
*/

View File

@@ -596,6 +596,7 @@ public class HoodieTableMetaClient implements Serializable {
private Integer timelineLayoutVersion;
private String baseFileFormat;
private String preCombineField;
private String partitionColumns;
private String bootstrapIndexClass;
private String bootstrapBasePath;
@@ -646,6 +647,11 @@ public class HoodieTableMetaClient implements Serializable {
return this;
}
public PropertyBuilder setPartitionColumns(String partitionColumns) {
this.partitionColumns = partitionColumns;
return this;
}
public PropertyBuilder setBootstrapIndexClass(String bootstrapIndexClass) {
this.bootstrapIndexClass = bootstrapIndexClass;
return this;
@@ -696,6 +702,9 @@ public class HoodieTableMetaClient implements Serializable {
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD)) {
setPreCombineField(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD));
}
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS)) {
setPartitionColumns(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS));
}
return this;
}
@@ -738,6 +747,10 @@ public class HoodieTableMetaClient implements Serializable {
if (null != preCombineField) {
properties.put(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD, preCombineField);
}
if (null != partitionColumns) {
properties.put(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS, partitionColumns);
}
return properties;
}