1
0

[HUDI-1450] Use metadata table for listing in HoodieROTablePathFilter (apache#2326)

[HUDI-1394] [RFC-15] Use metadata table (if present) to get all partition paths (apache#2351)
This commit is contained in:
Udit Mehrotra
2020-12-31 01:20:02 -08:00
committed by vinoth chandar
parent 298808baaf
commit 4e64226844
38 changed files with 308 additions and 102 deletions

View File

@@ -1,150 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.config;
import org.apache.hudi.common.config.DefaultHoodieConfig;
import javax.annotation.concurrent.Immutable;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
/**
* Configurations used by the HUDI Metadata Table.
*/
@Immutable
public class HoodieMetadataConfig extends DefaultHoodieConfig {
public static final String METADATA_PREFIX = "hoodie.metadata";
// Enable the internal Metadata Table which saves file listings
public static final String METADATA_ENABLE_PROP = METADATA_PREFIX + ".enable";
public static final boolean DEFAULT_METADATA_ENABLE = false;
// Validate contents of Metadata Table on each access against the actual filesystem
public static final String METADATA_VALIDATE_PROP = METADATA_PREFIX + ".validate";
public static final boolean DEFAULT_METADATA_VALIDATE = false;
// Parallelism for inserts
public static final String METADATA_INSERT_PARALLELISM_PROP = METADATA_PREFIX + ".insert.parallelism";
public static final int DEFAULT_METADATA_INSERT_PARALLELISM = 1;
// Async clean
public static final String METADATA_ASYNC_CLEAN_PROP = METADATA_PREFIX + ".clean.async";
public static final boolean DEFAULT_METADATA_ASYNC_CLEAN = false;
// Maximum delta commits before compaction occurs
public static final String METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = METADATA_PREFIX + ".compact.max.delta.commits";
public static final int DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS = 24;
// Archival settings
public static final String MIN_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.min.commits";
public static final int DEFAULT_MIN_COMMITS_TO_KEEP = 20;
public static final String MAX_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.max.commits";
public static final int DEFAULT_MAX_COMMITS_TO_KEEP = 30;
// Cleaner commits retained
public static final String CLEANER_COMMITS_RETAINED_PROP = METADATA_PREFIX + ".cleaner.commits.retained";
public static final int DEFAULT_CLEANER_COMMITS_RETAINED = 3;
private HoodieMetadataConfig(Properties props) {
super(props);
}
public static HoodieMetadataConfig.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
try (FileReader reader = new FileReader(propertiesFile)) {
this.props.load(reader);
return this;
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder enable(boolean enable) {
props.setProperty(METADATA_ENABLE_PROP, String.valueOf(enable));
return this;
}
public Builder validate(boolean validate) {
props.setProperty(METADATA_VALIDATE_PROP, String.valueOf(validate));
return this;
}
public Builder withInsertParallelism(int parallelism) {
props.setProperty(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism));
return this;
}
public Builder withAsyncClean(boolean asyncClean) {
props.setProperty(METADATA_ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
return this;
}
public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
props.setProperty(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
return this;
}
public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
props.setProperty(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
props.setProperty(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
return this;
}
public Builder retainCommits(int commitsRetained) {
props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
return this;
}
public HoodieMetadataConfig build() {
HoodieMetadataConfig config = new HoodieMetadataConfig(props);
setDefaultOnCondition(props, !props.containsKey(METADATA_ENABLE_PROP), METADATA_ENABLE_PROP,
String.valueOf(DEFAULT_METADATA_ENABLE));
setDefaultOnCondition(props, !props.containsKey(METADATA_VALIDATE_PROP), METADATA_VALIDATE_PROP,
String.valueOf(DEFAULT_METADATA_VALIDATE));
setDefaultOnCondition(props, !props.containsKey(METADATA_INSERT_PARALLELISM_PROP), METADATA_INSERT_PARALLELISM_PROP,
String.valueOf(DEFAULT_METADATA_INSERT_PARALLELISM));
setDefaultOnCondition(props, !props.containsKey(METADATA_ASYNC_CLEAN_PROP), METADATA_ASYNC_CLEAN_PROP,
String.valueOf(DEFAULT_METADATA_ASYNC_CLEAN));
setDefaultOnCondition(props, !props.containsKey(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP),
METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS));
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP,
String.valueOf(DEFAULT_CLEANER_COMMITS_RETAINED));
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP,
String.valueOf(DEFAULT_MAX_COMMITS_TO_KEEP));
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP,
String.valueOf(DEFAULT_MIN_COMMITS_TO_KEEP));
return config;
}
}
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.config;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.bootstrap.BootstrapMode;
import org.apache.hudi.common.config.DefaultHoodieConfig;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.client.common.EngineType;
import org.apache.hudi.common.model.HoodieCleaningPolicy;

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.avro.model.HoodieMetadataRecord;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.client.common.HoodieEngineContext;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.fs.FSUtils;
@@ -46,7 +47,6 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMetadataConfig;
import org.apache.hudi.config.HoodieMetricsConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
@@ -275,14 +275,15 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
// List all partitions in the basePath of the containing dataset
FileSystem fs = datasetMetaClient.getFs();
List<String> partitions = FSUtils.getAllPartitionPaths(fs, datasetWriteConfig.getBasePath(), datasetWriteConfig.shouldAssumeDatePartitioning());
FileSystemBackedTableMetadata fileSystemBackedTableMetadata = new FileSystemBackedTableMetadata(hadoopConf, datasetWriteConfig.getBasePath(),
datasetWriteConfig.shouldAssumeDatePartitioning());
List<String> partitions = fileSystemBackedTableMetadata.getAllPartitionPaths();
LOG.info("Initializing metadata table by using file listings in " + partitions.size() + " partitions");
// List all partitions in parallel and collect the files in them
int parallelism = Math.max(partitions.size(), 1);
List<Pair<String, FileStatus[]>> partitionFileList = engineContext.map(partitions, partition -> {
FileSystem fsys = datasetMetaClient.getFs();
FileStatus[] statuses = FSUtils.getAllDataFilesInPartition(fsys, new Path(datasetWriteConfig.getBasePath(), partition));
FileStatus[] statuses = fileSystemBackedTableMetadata.getAllFilesInPartition(new Path(datasetWriteConfig.getBasePath(), partition));
return Pair.of(partition, statuses);
}, parallelism);

View File

@@ -1,52 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.metadata;
import java.io.IOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.table.HoodieTable;
/**
* {@code HoodieTableFileSystemView} implementation that retrieved partition listings from the Metadata Table.
*/
public class HoodieMetadataFileSystemView extends HoodieTableFileSystemView {
private HoodieTable hoodieTable;
public HoodieMetadataFileSystemView(HoodieTableMetaClient metaClient, HoodieTable table,
HoodieTimeline visibleActiveTimeline, boolean enableIncrementalTimelineSync) {
super(metaClient, visibleActiveTimeline, enableIncrementalTimelineSync);
this.hoodieTable = table;
}
/**
* Return all the files in the partition by reading from the Metadata Table.
*
* @param partitionPath The absolute path of the partition
* @throws IOException
*/
@Override
protected FileStatus[] listPartition(Path partitionPath) throws IOException {
return hoodieTable.metadata().getAllFilesInPartition(partitionPath);
}
}

View File

@@ -277,7 +277,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
private SyncableFileSystemView getFileSystemViewInternal(HoodieTimeline timeline) {
if (config.useFileListingMetadata()) {
FileSystemViewStorageConfig viewConfig = config.getViewStorageConfig();
return new HoodieMetadataFileSystemView(metaClient, this, timeline, viewConfig.isIncrementalTimelineSyncEnabled());
return new HoodieMetadataFileSystemView(metaClient, this.metadata, timeline, viewConfig.isIncrementalTimelineSyncEnabled());
} else {
return getViewManager().getFileSystemView(metaClient);
}

View File

@@ -66,8 +66,10 @@ public abstract class PartitionAwareClusteringPlanStrategy<T extends HoodieRecor
try {
HoodieTableMetaClient metaClient = getHoodieTable().getMetaClient();
LOG.info("Scheduling clustering for " + metaClient.getBasePath());
HoodieWriteConfig config = getWriteConfig();
List<String> partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
getWriteConfig().shouldAssumeDatePartitioning());
config.useFileListingMetadata(), config.getFileListingMetadataVerify(),
config.shouldAssumeDatePartitioning());
// filter the partition paths if needed to reduce list status
partitionPaths = filterPartitionPaths(partitionPaths);

View File

@@ -88,12 +88,13 @@ public class RollbackUtils {
* Generate all rollback requests that needs rolling back this action without actually performing rollback for COW table type.
* @param fs instance of {@link FileSystem} to use.
* @param basePath base path of interest.
* @param shouldAssumeDatePartitioning {@code true} if date partitioning should be assumed. {@code false} otherwise.
* @param config instance of {@link HoodieWriteConfig} to use.
* @return {@link List} of {@link ListingBasedRollbackRequest}s thus collected.
*/
public static List<ListingBasedRollbackRequest> generateRollbackRequestsByListingCOW(FileSystem fs, String basePath, boolean shouldAssumeDatePartitioning) {
public static List<ListingBasedRollbackRequest> generateRollbackRequestsByListingCOW(FileSystem fs, String basePath, HoodieWriteConfig config) {
try {
return FSUtils.getAllPartitionPaths(fs, basePath, shouldAssumeDatePartitioning).stream()
return FSUtils.getAllPartitionPaths(fs, basePath, config.useFileListingMetadata(),
config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning()).stream()
.map(ListingBasedRollbackRequest::createRollbackRequestWithDeleteDataAndLogFilesAction)
.collect(Collectors.toList());
} catch (IOException e) {
@@ -113,7 +114,7 @@ public class RollbackUtils {
String commit = instantToRollback.getTimestamp();
HoodieWriteConfig config = table.getConfig();
List<String> partitions = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning());
config.useFileListingMetadata(), config.getFileListingMetadataVerify(), config.shouldAssumeDatePartitioning());
int sparkPartitions = Math.max(Math.min(partitions.size(), config.getRollbackParallelism()), 1);
context.setJobStatus(RollbackUtils.class.getSimpleName(), "Generate all rollback requests");
return context.flatMap(partitions, partitionPath -> {

View File

@@ -36,6 +36,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieSavepointException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.BaseActionExecutor;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -89,15 +90,18 @@ public class SavepointActionExecutor<T extends HoodieRecordPayload, I, K, O> ext
"Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained);
context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime);
Map<String, List<String>> latestFilesMap = context.mapToPair(FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(),
table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()), partitionPath -> {
// Scan all partitions files with this commit time
LOG.info("Collecting latest files in partition path " + partitionPath);
TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView();
List<String> latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime)
.map(HoodieBaseFile::getFileName).collect(Collectors.toList());
return new ImmutablePair<>(partitionPath, latestFiles);
}, null);
List<String> partitions = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(),
table.getMetaClient().getBasePath(), config.useFileListingMetadata(), config.getFileListingMetadataVerify(),
config.shouldAssumeDatePartitioning()
);
Map<String, List<String>> latestFilesMap = context.mapToPair(partitions, partitionPath -> {
// Scan all partitions files with this commit time
LOG.info("Collecting latest files in partition path " + partitionPath);
TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView();
List<String> latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime)
.map(HoodieBaseFile::getFileName).collect(Collectors.toList());
return new ImmutablePair<>(partitionPath, latestFiles);
}, null);
HoodieSavepointMetadata metadata = TimelineMetadataUtils.convertSavepointMetadata(user, comment, latestFilesMap);
// Nothing to save in the savepoint
table.getActiveTimeline().createNewInstant(