1
0

[HUDI 1308] Harden RFC-15 Implementation based on production testing (#2441)

Addresses leaks, perf degradation observed during testing. These were regressions from the original rfc-15 PoC implementation.

* Pass a single instance of HoodieTableMetadata everywhere
* Fix tests and add config for enabling metrics
 - Removed special casing of assumeDatePartitioning inside FSUtils#getAllPartitionPaths()
 - Consequently, IOException is never thrown and many files had to be adjusted
- More diligent handling of open file handles in metadata table
 - Added config for controlling reuse of connections
 - Added config for turning off fallback to listing, so we can see tests fail
 - Changed all ipf listing code to cache/amortize the open/close for better performance
 - Timelineserver also reuses connections, for better performance
 - Without timelineserver, when metadata table is opened from executors, reuse is not allowed
 - HoodieMetadataConfig passed into HoodieTableMetadata#create as argument.
 -  Fix TestHoodieBackedTableMetadata#testSync
This commit is contained in:
vinoth chandar
2021-01-19 21:20:28 -08:00
committed by GitHub
parent e23967b9e9
commit 5ca0625b27
55 changed files with 767 additions and 570 deletions

View File

@@ -41,6 +41,10 @@ public final class HoodieMetadataConfig extends DefaultHoodieConfig {
public static final boolean DEFAULT_METADATA_VALIDATE = false;
public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false;
// Enable metrics for internal Metadata Table
public static final String METADATA_METRICS_ENABLE_PROP = METADATA_PREFIX + ".metrics.enable";
public static final boolean DEFAULT_METADATA_METRICS_ENABLE = 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;
@@ -63,6 +67,20 @@ public final class HoodieMetadataConfig extends DefaultHoodieConfig {
public static final String CLEANER_COMMITS_RETAINED_PROP = METADATA_PREFIX + ".cleaner.commits.retained";
public static final int DEFAULT_CLEANER_COMMITS_RETAINED = 3;
// Controls whether or no the base file open/log merges are reused per API call
public static final String ENABLE_REUSE_PROP = METADATA_PREFIX + ".reuse.enable";
public static final String DEFAULT_ENABLE_REUSE = "true";
// Controls whether or not, upon failure to fetch from metadata table, should fallback to listing.
public static final String ENABLE_FALLBACK_PROP = METADATA_PREFIX + ".fallback.enable";
public static final String DEFAULT_ENABLE_FALLBACK = "true";
public static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning";
public static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
public static final String FILE_LISTING_PARALLELISM_PROP = "hoodie.file.listing.parallelism";
public static final int DEFAULT_FILE_LISTING_PARALLELISM = 1500;
private HoodieMetadataConfig(Properties props) {
super(props);
}
@@ -71,6 +89,34 @@ public final class HoodieMetadataConfig extends DefaultHoodieConfig {
return new Builder();
}
public int getFileListingParallelism() {
return Math.max(Integer.parseInt(props.getProperty(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_PROP)), 1);
}
public Boolean shouldAssumeDatePartitioning() {
return Boolean.parseBoolean(props.getProperty(HoodieMetadataConfig.HOODIE_ASSUME_DATE_PARTITIONING_PROP));
}
public boolean useFileListingMetadata() {
return Boolean.parseBoolean(props.getProperty(METADATA_ENABLE_PROP));
}
public boolean enableReuse() {
return Boolean.parseBoolean(props.getProperty(ENABLE_REUSE_PROP));
}
public boolean enableFallback() {
return Boolean.parseBoolean(props.getProperty(ENABLE_FALLBACK_PROP));
}
public boolean validateFileListingMetadata() {
return Boolean.parseBoolean(props.getProperty(METADATA_VALIDATE_PROP));
}
public boolean enableMetrics() {
return Boolean.parseBoolean(props.getProperty(METADATA_METRICS_ENABLE_PROP));
}
public static class Builder {
private final Properties props = new Properties();
@@ -92,6 +138,21 @@ public final class HoodieMetadataConfig extends DefaultHoodieConfig {
return this;
}
public Builder enableMetrics(boolean enableMetrics) {
props.setProperty(METADATA_METRICS_ENABLE_PROP, String.valueOf(enableMetrics));
return this;
}
public Builder enableReuse(boolean reuse) {
props.setProperty(ENABLE_REUSE_PROP, String.valueOf(reuse));
return this;
}
public Builder enableFallback(boolean fallback) {
props.setProperty(ENABLE_FALLBACK_PROP, String.valueOf(fallback));
return this;
}
public Builder validate(boolean validate) {
props.setProperty(METADATA_VALIDATE_PROP, String.valueOf(validate));
return this;
@@ -123,10 +184,22 @@ public final class HoodieMetadataConfig extends DefaultHoodieConfig {
return this;
}
public Builder withFileListingParallelism(int parallelism) {
props.setProperty(FILE_LISTING_PARALLELISM_PROP, String.valueOf(parallelism));
return this;
}
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
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_METRICS_ENABLE_PROP), METADATA_METRICS_ENABLE_PROP,
String.valueOf(DEFAULT_METADATA_METRICS_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,
@@ -141,7 +214,14 @@ public final class HoodieMetadataConfig extends DefaultHoodieConfig {
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));
setDefaultOnCondition(props, !props.containsKey(FILE_LISTING_PARALLELISM_PROP), FILE_LISTING_PARALLELISM_PROP,
String.valueOf(DEFAULT_FILE_LISTING_PARALLELISM));
setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
setDefaultOnCondition(props, !props.containsKey(ENABLE_FALLBACK_PROP), ENABLE_FALLBACK_PROP,
DEFAULT_ENABLE_FALLBACK);
setDefaultOnCondition(props, !props.containsKey(ENABLE_REUSE_PROP), ENABLE_REUSE_PROP,
DEFAULT_ENABLE_REUSE);
return config;
}
}

View File

@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieFileFormat;
@@ -34,6 +35,7 @@ import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
@@ -252,15 +254,29 @@ public class FSUtils {
}
}
public static List<String> getAllPartitionPaths(HoodieEngineContext engineContext, FileSystem fs, String basePathStr,
public static List<String> getAllPartitionPaths(HoodieEngineContext engineContext, String basePathStr,
boolean useFileListingFromMetadata, boolean verifyListings,
boolean assumeDatePartitioning) throws IOException {
if (assumeDatePartitioning) {
return getAllPartitionFoldersThreeLevelsDown(fs, basePathStr);
} else {
HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, basePathStr, "/tmp/",
useFileListingFromMetadata, verifyListings, false, false);
boolean assumeDatePartitioning) {
HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder()
.enable(useFileListingFromMetadata)
.validate(verifyListings)
.withAssumeDatePartitioning(assumeDatePartitioning)
.build();
try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
return tableMetadata.getAllPartitionPaths();
} catch (Exception e) {
throw new HoodieException("Error fetching partition paths from metadata table", e);
}
}
public static List<String> getAllPartitionPaths(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
String basePathStr) {
try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
return tableMetadata.getAllPartitionPaths();
} catch (Exception e) {
throw new HoodieException("Error fetching partition paths from metadata table", e);
}
}

View File

@@ -0,0 +1,26 @@
/*
* 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.common.function;
import java.io.Serializable;
@FunctionalInterface
public interface SerializableSupplier<T> extends Serializable {
T get();
}

View File

@@ -18,9 +18,11 @@
package org.apache.hudi.common.metrics;
import java.io.Serializable;
/**
* Interface for Hudi Metric Types.
*/
public interface Metric {
public interface Metric extends Serializable {
Long getValue();
}

View File

@@ -153,6 +153,12 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
return totalTimeTakenToReadAndMergeBlocks;
}
public void close() {
if (records != null) {
records.close();
}
}
/**
* Builder used to build {@code HoodieUnMergedLogRecordScanner}.
*/

View File

@@ -18,12 +18,16 @@
package org.apache.hudi.common.table.view;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.function.SerializableSupplier;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Functions.Function2;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.metadata.HoodieMetadataFileSystemView;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -62,9 +66,9 @@ public class FileSystemViewManager {
// Factory Map to create file-system views
private final Function2<HoodieTableMetaClient, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator;
public FileSystemViewManager(SerializableConfiguration conf, FileSystemViewStorageConfig viewStorageConfig,
private FileSystemViewManager(HoodieEngineContext context, FileSystemViewStorageConfig viewStorageConfig,
Function2<HoodieTableMetaClient, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator) {
this.conf = new SerializableConfiguration(conf);
this.conf = context.getHadoopConf();
this.viewStorageConfig = viewStorageConfig;
this.globalViewMap = new ConcurrentHashMap<>();
this.viewCreator = viewCreator;
@@ -110,8 +114,10 @@ public class FileSystemViewManager {
* Closes all views opened.
*/
public void close() {
this.globalViewMap.values().forEach(SyncableFileSystemView::close);
this.globalViewMap.clear();
if (!this.globalViewMap.isEmpty()) {
this.globalViewMap.values().forEach(SyncableFileSystemView::close);
this.globalViewMap.clear();
}
}
// FACTORY METHODS FOR CREATING FILE-SYSTEM VIEWS
@@ -147,34 +153,32 @@ public class FileSystemViewManager {
/**
* Create an in-memory file System view for a table.
*
* @param conf Hadoop Configuration
* @param viewConf View Storage Configuration
* @param metaClient HoodieTableMetaClient
* @return
*
*/
private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
private static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieMetadataConfig metadataConfig, FileSystemViewStorageConfig viewConf,
HoodieTableMetaClient metaClient, SerializableSupplier<HoodieTableMetadata> metadataSupplier) {
LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath());
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
if (metadataConfig.useFileListingMetadata()) {
ValidationUtils.checkArgument(metadataSupplier != null, "Metadata supplier is null. Cannot instantiate metadata file system view");
return new HoodieMetadataFileSystemView(metaClient, metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(),
metadataSupplier.get());
}
return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled());
}
public static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieEngineContext engineContext,
HoodieTableMetaClient metaClient, boolean useFileListingFromMetadata, boolean verifyListings) {
public static HoodieTableFileSystemView createInMemoryFileSystemView(HoodieEngineContext engineContext, HoodieTableMetaClient metaClient,
HoodieMetadataConfig metadataConfig) {
LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath());
if (useFileListingFromMetadata) {
if (metadataConfig.useFileListingMetadata()) {
return new HoodieMetadataFileSystemView(engineContext, metaClient,
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(),
true,
verifyListings);
metadataConfig);
}
return new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants());
}
/**
* Create a remote file System view for a table.
*
@@ -192,42 +196,56 @@ public class FileSystemViewManager {
metaClient, viewConf.getRemoteTimelineClientTimeoutSecs());
}
public static FileSystemViewManager createViewManager(final HoodieEngineContext context,
final HoodieMetadataConfig metadataConfig,
final FileSystemViewStorageConfig config) {
return createViewManager(context, metadataConfig, config, (SerializableSupplier<HoodieTableMetadata>) null);
}
public static FileSystemViewManager createViewManager(final HoodieEngineContext context,
final HoodieMetadataConfig metadataConfig,
final FileSystemViewStorageConfig config,
final String basePath) {
return createViewManager(context, metadataConfig, config,
() -> HoodieTableMetadata.create(context, metadataConfig, basePath, config.getSpillableDir()));
}
/**
* Main Factory method for building file-system views.
*
* @param conf Hadoop Configuration
* @param config View Storage Configuration
* @return
*
*/
public static FileSystemViewManager createViewManager(final SerializableConfiguration conf,
final FileSystemViewStorageConfig config) {
public static FileSystemViewManager createViewManager(final HoodieEngineContext context,
final HoodieMetadataConfig metadataConfig,
final FileSystemViewStorageConfig config,
final SerializableSupplier<HoodieTableMetadata> metadataSupplier) {
LOG.info("Creating View Manager with storage type :" + config.getStorageType());
final SerializableConfiguration conf = context.getHadoopConf();
switch (config.getStorageType()) {
case EMBEDDED_KV_STORE:
LOG.info("Creating embedded rocks-db based Table View");
return new FileSystemViewManager(conf, config,
return new FileSystemViewManager(context, config,
(metaClient, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, metaClient));
case SPILLABLE_DISK:
LOG.info("Creating Spillable Disk based Table View");
return new FileSystemViewManager(conf, config,
return new FileSystemViewManager(context, config,
(metaClient, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, metaClient));
case MEMORY:
LOG.info("Creating in-memory based Table View");
return new FileSystemViewManager(conf, config,
(metaClient, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, metaClient));
return new FileSystemViewManager(context, config,
(metaClient, viewConfig) -> createInMemoryFileSystemView(metadataConfig, viewConfig, metaClient, metadataSupplier));
case REMOTE_ONLY:
LOG.info("Creating remote only table view");
return new FileSystemViewManager(conf, config, (metaClient, viewConfig) -> createRemoteFileSystemView(conf,
return new FileSystemViewManager(context, config, (metaClient, viewConfig) -> createRemoteFileSystemView(conf,
viewConfig, metaClient));
case REMOTE_FIRST:
LOG.info("Creating remote first table view");
return new FileSystemViewManager(conf, config, (metaClient, viewConfig) -> {
return new FileSystemViewManager(context, config, (metaClient, viewConfig) -> {
RemoteHoodieTableFileSystemView remoteFileSystemView =
createRemoteFileSystemView(conf, viewConfig, metaClient);
SyncableFileSystemView secondaryView;
switch (viewConfig.getSecondaryStorageType()) {
case MEMORY:
secondaryView = createInMemoryFileSystemView(conf, viewConfig, metaClient);
secondaryView = createInMemoryFileSystemView(metadataConfig, viewConfig, metaClient, metadataSupplier);
break;
case EMBEDDED_KV_STORE:
secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, metaClient);

View File

@@ -33,39 +33,49 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
// Property Names
public static final String FILESYSTEM_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.type";
public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
public static final String FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "hoodie.filesystem.view.incr.timeline.sync.enable";
public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "false";
public static final String FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.secondary.type";
public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
public static final String FILESYSTEM_VIEW_REMOTE_HOST = "hoodie.filesystem.view.remote.host";
public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = "localhost";
public static final String FILESYSTEM_VIEW_REMOTE_PORT = "hoodie.filesystem.view.remote.port";
public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = 26754;
public static final String FILESYSTEM_VIEW_SPILLABLE_DIR = "hoodie.filesystem.view.spillable.dir";
public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/";
public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = "hoodie.filesystem.view.spillable.mem";
private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB
public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION =
"hoodie.filesystem.view.spillable.compaction.mem.fraction";
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01;
public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION =
"hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction";
public static final String FILESYSTEM_VIEW_REPLACED_MEM_FRACTION =
"hoodie.filesystem.view.spillable.replaced.mem.fraction";
private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = 0.01;
public static final String FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION =
"hoodie.filesystem.view.spillable.clustering.mem.fraction";
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS = 0.01;
private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path";
public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS =
"hoodie.filesystem.view.remote.timeout.secs";
public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
public static final String DEFAULT_ROCKSDB_BASE_PATH = "/tmp/hoodie_timeline_rocksdb";
public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "false";
public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = "localhost";
public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = 26754;
public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS =
"hoodie.filesystem.view.remote.timeout.secs";
public static final Integer DEFAULT_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = 5 * 60; // 5 min
public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/";
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01;
private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = 0.05;
private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = 0.01;
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS = 0.01;
private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB
/**
* Configs to control whether backup needs to be configured if clients were not able to reach
@@ -135,7 +145,7 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
.longValue();
}
public String getBaseStoreDir() {
public String getSpillableDir() {
return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR);
}

View File

@@ -62,7 +62,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
this.maxMemoryForBootstrapBaseFile = config.getMaxMemoryForBootstrapBaseFile();
this.maxMemoryForReplaceFileGroups = config.getMaxMemoryForReplacedFileGroups();
this.maxMemoryForClusteringFileGroups = config.getMaxMemoryForPendingClusteringFileGroups();
this.baseStoreDir = config.getBaseStoreDir();
this.baseStoreDir = config.getSpillableDir();
init(metaClient, visibleActiveTimeline);
}

View File

@@ -19,29 +19,29 @@
package org.apache.hudi.metadata;
import org.apache.avro.Schema;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieMetadataRecord;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@@ -57,33 +57,33 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
protected final transient HoodieEngineContext engineContext;
protected final SerializableConfiguration hadoopConf;
protected final String datasetBasePath;
protected boolean enabled;
protected final HoodieTableMetaClient datasetMetaClient;
protected final Option<HoodieMetadataMetrics> metrics;
private final boolean validateLookups;
private final boolean assumeDatePartitioning;
protected final HoodieMetadataConfig metadataConfig;
// Directory used for Spillable Map when merging records
protected final String spillableMapDirectory;
private transient HoodieMetadataMergedInstantRecordScanner timelineRecordScanner;
protected BaseTableMetadata(HoodieEngineContext engineContext, String datasetBasePath, String spillableMapDirectory,
boolean enabled, boolean validateLookups, boolean enableMetrics,
boolean assumeDatePartitioning) {
protected boolean enabled;
private TimelineMergedTableMetadata timelineMergedMetadata;
protected BaseTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
String datasetBasePath, String spillableMapDirectory) {
this.engineContext = engineContext;
this.hadoopConf = new SerializableConfiguration(engineContext.getHadoopConf());
this.datasetBasePath = datasetBasePath;
this.datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath);
this.spillableMapDirectory = spillableMapDirectory;
this.metadataConfig = metadataConfig;
this.enabled = enabled;
this.validateLookups = validateLookups;
this.assumeDatePartitioning = assumeDatePartitioning;
if (enableMetrics) {
this.enabled = metadataConfig.useFileListingMetadata();
if (metadataConfig.enableMetrics()) {
this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata")));
} else {
this.metrics = Option.empty();
}
if (enabled) {
openTimelineScanner();
}
}
/**
@@ -101,11 +101,15 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
try {
return fetchAllPartitionPaths();
} catch (Exception e) {
LOG.error("Failed to retrieve list of partition from metadata", e);
if (metadataConfig.enableFallback()) {
LOG.error("Failed to retrieve list of partition from metadata", e);
} else {
throw new HoodieMetadataException("Failed to retrieve list of partition from metadata", e);
}
}
}
return new FileSystemBackedTableMetadata(engineContext, hadoopConf, datasetBasePath,
assumeDatePartitioning).getAllPartitionPaths();
return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, datasetBasePath,
metadataConfig.shouldAssumeDatePartitioning()).getAllPartitionPaths();
}
/**
@@ -125,11 +129,16 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
try {
return fetchAllFilesInPartition(partitionPath);
} catch (Exception e) {
LOG.error("Failed to retrieve files in partition " + partitionPath + " from metadata", e);
if (metadataConfig.enableFallback()) {
LOG.error("Failed to retrieve files in partition " + partitionPath + " from metadata", e);
} else {
throw new HoodieMetadataException("Failed to retrieve files in partition " + partitionPath + " from metadata", e);
}
}
}
return FSUtils.getFs(partitionPath.toString(), hadoopConf.get()).listStatus(partitionPath);
return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, datasetBasePath, metadataConfig.shouldAssumeDatePartitioning())
.getAllFilesInPartition(partitionPath);
}
/**
@@ -155,11 +164,11 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
}
}
if (validateLookups) {
if (metadataConfig.validateFileListingMetadata()) {
// Validate the Metadata Table data by listing the partitions from the file system
timer.startTimer();
FileSystemBackedTableMetadata fileSystemBackedTableMetadata = new FileSystemBackedTableMetadata(engineContext,
hadoopConf, datasetBasePath, assumeDatePartitioning);
FileSystemBackedTableMetadata fileSystemBackedTableMetadata = new FileSystemBackedTableMetadata(getEngineContext(),
hadoopConf, datasetBasePath, metadataConfig.shouldAssumeDatePartitioning());
List<String> actualPartitions = fileSystemBackedTableMetadata.getAllPartitionPaths();
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_PARTITIONS_STR, timer.endTimer()));
@@ -205,19 +214,29 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
statuses = hoodieRecord.get().getData().getFileStatuses(hadoopConf.get(), partitionPath);
}
if (validateLookups) {
if (metadataConfig.validateFileListingMetadata()) {
// Validate the Metadata Table data by listing the partitions from the file system
timer.startTimer();
// Ignore partition metadata file
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath);
FileStatus[] directStatuses = metaClient.getFs().listStatus(partitionPath,
p -> !p.getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE));
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_FILES_STR, timer.endTimer()));
String partitionPathStr = FSUtils.getRelativePartitionPath(new Path(datasetMetaClient.getBasePath()), partitionPath);
String latestDataInstantTime = getLatestDatasetInstantTime();
HoodieTableFileSystemView dataFsView = new HoodieTableFileSystemView(datasetMetaClient, datasetMetaClient.getActiveTimeline());
List<FileStatus> directStatuses = dataFsView.getAllFileSlices(partitionPathStr).flatMap(slice -> {
List<FileStatus> paths = new ArrayList<>();
slice.getBaseFile().ifPresent(baseFile -> {
if (HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, latestDataInstantTime)) {
paths.add(baseFile.getFileStatus());
}
});
//TODO(metadata): this will remain problematic; no way to know the commit time based on log file written
slice.getLogFiles().forEach(logFile -> paths.add(logFile.getFileStatus()));
return paths.stream();
}).collect(Collectors.toList());
List<String> directFilenames = Arrays.stream(directStatuses)
.map(s -> s.getPath().getName()).sorted()
List<String> directFilenames = directStatuses.stream()
.map(fileStatus -> fileStatus.getPath().getName()).sorted()
.collect(Collectors.toList());
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_FILES_STR, timer.endTimer()));
List<String> metadataFilenames = Arrays.stream(statuses)
.map(s -> s.getPath().getName()).sorted()
@@ -232,7 +251,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
}
// Return the direct listing as it should be correct
statuses = directStatuses;
statuses = directStatuses.toArray(new FileStatus[0]);
}
LOG.info("Listed file in partition from metadata: partition=" + partitionName + ", #files=" + statuses.length);
@@ -244,14 +263,11 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
*
* @param key The key of the record
*/
private Option<HoodieRecord<HoodieMetadataPayload>> getMergedRecordByKey(String key) throws IOException {
private Option<HoodieRecord<HoodieMetadataPayload>> getMergedRecordByKey(String key) {
Option<HoodieRecord<HoodieMetadataPayload>> mergedRecord;
openTimelineScanner();
Option<HoodieRecord<HoodieMetadataPayload>> metadataHoodieRecord = getRecordByKeyFromMetadata(key);
// Retrieve record from unsynced timeline instants
Option<HoodieRecord<HoodieMetadataPayload>> timelineHoodieRecord = timelineRecordScanner.getRecordByKey(key);
Option<HoodieRecord<HoodieMetadataPayload>> timelineHoodieRecord = timelineMergedMetadata.getRecordByKey(key);
if (timelineHoodieRecord.isPresent()) {
if (metadataHoodieRecord.isPresent()) {
HoodieRecordPayload mergedPayload = timelineHoodieRecord.get().getData().preCombine(metadataHoodieRecord.get().getData());
@@ -265,37 +281,28 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
return mergedRecord;
}
protected abstract Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKeyFromMetadata(String key) throws IOException;
protected abstract Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKeyFromMetadata(String key);
private void openTimelineScanner() throws IOException {
if (timelineRecordScanner != null) {
// Already opened
return;
private void openTimelineScanner() {
if (timelineMergedMetadata == null) {
List<HoodieInstant> unSyncedInstants = findInstantsToSync();
timelineMergedMetadata =
new TimelineMergedTableMetadata(datasetMetaClient, unSyncedInstants, getSyncedInstantTime(), null);
}
HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath);
List<HoodieInstant> unSyncedInstants = findInstantsToSync(datasetMetaClient);
Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema());
timelineRecordScanner =
new HoodieMetadataMergedInstantRecordScanner(datasetMetaClient, unSyncedInstants, getSyncedInstantTime(), schema, MAX_MEMORY_SIZE_IN_BYTES, spillableMapDirectory, null);
}
protected List<HoodieInstant> findInstantsToSync() {
HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath);
return findInstantsToSync(datasetMetaClient);
}
protected abstract List<HoodieInstant> findInstantsToSync(HoodieTableMetaClient datasetMetaClient);
protected abstract List<HoodieInstant> findInstantsToSync();
public boolean isInSync() {
return enabled && findInstantsToSync().isEmpty();
}
protected void closeReaders() {
timelineRecordScanner = null;
protected HoodieEngineContext getEngineContext() {
return engineContext != null ? engineContext : new HoodieLocalEngineContext(hadoopConf.get());
}
protected HoodieEngineContext getEngineContext() {
return engineContext;
protected String getLatestDatasetInstantTime() {
return datasetMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant()
.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP);
}
}

View File

@@ -114,4 +114,9 @@ public class FileSystemBackedTableMetadata implements HoodieTableMetadata {
public boolean isInSync() {
return true;
}
@Override
public void close() throws Exception {
// no-op
}
}

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.metadata;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieMetadataRecord;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
@@ -32,12 +33,13 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
@@ -50,6 +52,8 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -65,32 +69,41 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadata.class);
private final String metadataBasePath;
private String metadataBasePath;
// Metadata table's timeline and metaclient
private HoodieTableMetaClient metaClient;
private List<FileSlice> latestFileSystemMetadataSlices;
// Readers for the base and log file which store the metadata
private transient HoodieFileReader<GenericRecord> baseFileReader;
private transient HoodieMetadataMergedLogRecordScanner logRecordScanner;
public HoodieBackedTableMetadata(Configuration conf, String datasetBasePath, String spillableMapDirectory, boolean enabled,
boolean validateLookups, boolean assumeDatePartitioning) {
this(new HoodieLocalEngineContext(conf), datasetBasePath, spillableMapDirectory, enabled, validateLookups,
false, assumeDatePartitioning);
public HoodieBackedTableMetadata(Configuration conf, HoodieMetadataConfig metadataConfig,
String datasetBasePath, String spillableMapDirectory) {
this(new HoodieLocalEngineContext(conf), metadataConfig, datasetBasePath, spillableMapDirectory);
}
public HoodieBackedTableMetadata(HoodieEngineContext engineContext, String datasetBasePath, String spillableMapDirectory,
boolean enabled, boolean validateLookups, boolean enableMetrics, boolean assumeDatePartitioning) {
super(engineContext, datasetBasePath, spillableMapDirectory, enabled, validateLookups, enableMetrics, assumeDatePartitioning);
this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetBasePath);
if (enabled) {
public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
String datasetBasePath, String spillableMapDirectory) {
super(engineContext, metadataConfig, datasetBasePath, spillableMapDirectory);
initIfNeeded();
}
private void initIfNeeded() {
if (enabled && this.metaClient == null) {
this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetBasePath);
try {
this.metaClient = new HoodieTableMetaClient(hadoopConf.get(), metadataBasePath);
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline());
latestFileSystemMetadataSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
} catch (TableNotFoundException e) {
LOG.warn("Metadata table was not found at path " + metadataBasePath);
this.enabled = false;
this.metaClient = null;
} catch (Exception e) {
LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e);
this.enabled = false;
this.metaClient = null;
}
} else {
LOG.info("Metadata table is disabled.");
@@ -98,60 +111,67 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
}
@Override
protected Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKeyFromMetadata(String key) throws IOException {
openBaseAndLogFiles();
// Retrieve record from base file
HoodieRecord<HoodieMetadataPayload> hoodieRecord = null;
if (baseFileReader != null) {
protected Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKeyFromMetadata(String key) {
try {
List<Long> timings = new ArrayList<>();
HoodieTimer timer = new HoodieTimer().startTimer();
Option<GenericRecord> baseRecord = baseFileReader.getRecordByKey(key);
if (baseRecord.isPresent()) {
hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(),
metaClient.getTableConfig().getPayloadClass());
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, timer.endTimer()));
}
}
openFileSliceIfNeeded();
timings.add(timer.endTimer());
// Retrieve record from log file
Option<HoodieRecord<HoodieMetadataPayload>> logHoodieRecord = logRecordScanner.getRecordByKey(key);
if (logHoodieRecord.isPresent()) {
if (hoodieRecord != null) {
// Merge the payloads
HoodieRecordPayload mergedPayload = logHoodieRecord.get().getData().preCombine(hoodieRecord.getData());
hoodieRecord = new HoodieRecord(hoodieRecord.getKey(), mergedPayload);
} else {
hoodieRecord = logHoodieRecord.get();
timer.startTimer();
// Retrieve record from base file
HoodieRecord<HoodieMetadataPayload> hoodieRecord = null;
if (baseFileReader != null) {
HoodieTimer readTimer = new HoodieTimer().startTimer();
Option<GenericRecord> baseRecord = baseFileReader.getRecordByKey(key);
if (baseRecord.isPresent()) {
hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(),
metaClient.getTableConfig().getPayloadClass());
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer()));
}
}
}
timings.add(timer.endTimer());
return Option.ofNullable(hoodieRecord);
// Retrieve record from log file
timer.startTimer();
if (logRecordScanner != null) {
Option<HoodieRecord<HoodieMetadataPayload>> logHoodieRecord = logRecordScanner.getRecordByKey(key);
if (logHoodieRecord.isPresent()) {
if (hoodieRecord != null) {
// Merge the payloads
HoodieRecordPayload mergedPayload = logHoodieRecord.get().getData().preCombine(hoodieRecord.getData());
hoodieRecord = new HoodieRecord(hoodieRecord.getKey(), mergedPayload);
} else {
hoodieRecord = logHoodieRecord.get();
}
}
}
timings.add(timer.endTimer());
LOG.info(String.format("Metadata read for key %s took [open, baseFileRead, logMerge] %s ms", key, timings));
return Option.ofNullable(hoodieRecord);
} catch (IOException ioe) {
throw new HoodieIOException("Error merging records from metadata table for key :" + key, ioe);
} finally {
closeIfNeeded();
}
}
/**
* Open readers to the base and log files.
*/
protected synchronized void openBaseAndLogFiles() throws IOException {
if (logRecordScanner != null) {
// Already opened
private synchronized void openFileSliceIfNeeded() throws IOException {
if (metadataConfig.enableReuse() && baseFileReader != null) {
// we will reuse what's open.
return;
}
HoodieTimer timer = new HoodieTimer().startTimer();
// Metadata is in sync till the latest completed instant on the dataset
HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath);
String latestInstantTime = datasetMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant()
.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP);
// Find the latest file slice
HoodieTimeline timeline = metaClient.reloadActiveTimeline();
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline());
List<FileSlice> latestSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
ValidationUtils.checkArgument(latestSlices.size() == 1);
HoodieTimer timer = new HoodieTimer().startTimer();
String latestInstantTime = getLatestDatasetInstantTime();
ValidationUtils.checkArgument(latestFileSystemMetadataSlices.size() == 1, "must be at-least one validata metadata file slice");
// If the base file is present then create a reader
Option<HoodieBaseFile> basefile = latestSlices.get(0).getBaseFile();
Option<HoodieBaseFile> basefile = latestFileSystemMetadataSlices.get(0).getBaseFile();
if (basefile.isPresent()) {
String basefilePath = basefile.get().getPath();
baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath));
@@ -159,18 +179,16 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
}
// Open the log record scanner using the log files from the latest file slice
List<String> logFilePaths = latestSlices.get(0).getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
List<String> logFilePaths = latestFileSystemMetadataSlices.get(0).getLogFiles()
.sorted(HoodieLogFile.getLogFileComparator())
.map(o -> o.getPath().toString())
.collect(Collectors.toList());
Option<HoodieInstant> lastInstant = timeline.filterCompletedInstants().lastInstant();
Option<HoodieInstant> lastInstant = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant();
String latestMetaInstantTimestamp = lastInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP);
// Load the schema
Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema());
logRecordScanner =
new HoodieMetadataMergedLogRecordScanner(metaClient.getFs(), metadataBasePath,
logRecordScanner = new HoodieMetadataMergedLogRecordScanner(metaClient.getFs(), metadataBasePath,
logFilePaths, schema, latestMetaInstantTimestamp, MAX_MEMORY_SIZE_IN_BYTES, BUFFER_SIZE,
spillableMapDirectory, null);
@@ -180,27 +198,42 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, timer.endTimer()));
}
public void closeReaders() {
private void closeIfNeeded() {
try {
if (!metadataConfig.enableReuse()) {
close();
}
} catch (Exception e) {
throw new HoodieException("Error closing resources during metadata table merge", e);
}
}
@Override
public void close() throws Exception {
if (baseFileReader != null) {
baseFileReader.close();
baseFileReader = null;
}
logRecordScanner = null;
if (logRecordScanner != null) {
logRecordScanner.close();
logRecordScanner = null;
}
}
/**
* Return an ordered list of instants which have not been synced to the Metadata Table.
* @param datasetMetaClient {@code HoodieTableMetaClient} for the dataset
*/
protected List<HoodieInstant> findInstantsToSync(HoodieTableMetaClient datasetMetaClient) {
HoodieActiveTimeline metaTimeline = metaClient.reloadActiveTimeline();
protected List<HoodieInstant> findInstantsToSync() {
initIfNeeded();
// if there are no instants yet, return empty list, since there is nothing to sync here.
if (!enabled || !metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().isPresent()) {
return Collections.EMPTY_LIST;
}
// All instants on the data timeline, which are greater than the last instant on metadata timeline
// are candidates for sync.
Option<HoodieInstant> latestMetadataInstant = metaTimeline.filterCompletedInstants().lastInstant();
ValidationUtils.checkArgument(latestMetadataInstant.isPresent(),
"At least one completed instant should exist on the metadata table, before syncing.");
String latestMetadataInstantTime = latestMetadataInstant.get().getTimestamp();
String latestMetadataInstantTime = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get().getTimestamp();
HoodieDefaultTimeline candidateTimeline = datasetMetaClient.getActiveTimeline().findInstantsAfter(latestMetadataInstantTime, Integer.MAX_VALUE);
Option<HoodieInstant> earliestIncompleteInstant = candidateTimeline.filterInflightsAndRequested().firstInstant();
@@ -236,10 +269,6 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
return hadoopConf;
}
public String getDatasetBasePath() {
return datasetBasePath;
}
public HoodieTableMetaClient getMetaClient() {
return metaClient;
}

View File

@@ -23,11 +23,13 @@ import java.io.IOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.exception.HoodieException;
/**
* {@code HoodieTableFileSystemView} implementation that retrieved partition listings from the Metadata Table.
@@ -36,21 +38,20 @@ public class HoodieMetadataFileSystemView extends HoodieTableFileSystemView {
private final HoodieTableMetadata tableMetadata;
public HoodieMetadataFileSystemView(HoodieTableMetaClient metaClient, HoodieTableMetadata tableMetadata,
HoodieTimeline visibleActiveTimeline, boolean enableIncrementalTimelineSync) {
super(metaClient, visibleActiveTimeline, enableIncrementalTimelineSync);
public HoodieMetadataFileSystemView(HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline,
HoodieTableMetadata tableMetadata) {
super(metaClient, visibleActiveTimeline);
this.tableMetadata = tableMetadata;
}
public HoodieMetadataFileSystemView(HoodieEngineContext engineContext,
HoodieTableMetaClient metaClient,
HoodieTimeline visibleActiveTimeline,
boolean useFileListingFromMetadata,
boolean verifyListings) {
HoodieMetadataConfig metadataConfig) {
super(metaClient, visibleActiveTimeline);
this.tableMetadata = HoodieTableMetadata.create(engineContext, metaClient.getBasePath(),
FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR, useFileListingFromMetadata, verifyListings,
false, false);
this.tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath(),
FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR);
}
/**
@@ -63,4 +64,13 @@ public class HoodieMetadataFileSystemView extends HoodieTableFileSystemView {
protected FileStatus[] listPartition(Path partitionPath) throws IOException {
return tableMetadata.getAllFilesInPartition(partitionPath);
}
@Override
public void close() {
try {
tableMetadata.close();
} catch (Exception e) {
throw new HoodieException("Error closing metadata file system view.", e);
}
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.metadata;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -33,7 +34,7 @@ import java.util.List;
/**
* Interface that supports querying various pieces of metadata about a hudi table.
*/
public interface HoodieTableMetadata extends Serializable {
public interface HoodieTableMetadata extends Serializable, AutoCloseable {
// Table name suffix
String METADATA_TABLE_NAME_SUFFIX = "_metadata";
@@ -69,15 +70,13 @@ public interface HoodieTableMetadata extends Serializable {
return basePath.endsWith(METADATA_TABLE_REL_PATH);
}
static HoodieTableMetadata create(HoodieEngineContext engineContext, String datasetBasePath,
String spillableMapPath, boolean useFileListingFromMetadata, boolean verifyListings,
boolean enableMetrics, boolean shouldAssumeDatePartitioning) {
if (useFileListingFromMetadata) {
return new HoodieBackedTableMetadata(engineContext, datasetBasePath, spillableMapPath, useFileListingFromMetadata,
verifyListings, enableMetrics, shouldAssumeDatePartitioning);
static HoodieTableMetadata create(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig, String datasetBasePath,
String spillableMapPath) {
if (metadataConfig.useFileListingMetadata()) {
return new HoodieBackedTableMetadata(engineContext, metadataConfig, datasetBasePath, spillableMapPath);
} else {
return new FileSystemBackedTableMetadata(engineContext, new SerializableConfiguration(engineContext.getHadoopConf()),
datasetBasePath, shouldAssumeDatePartitioning);
datasetBasePath, metadataConfig.shouldAssumeDatePartitioning());
}
}

View File

@@ -18,48 +18,48 @@
package org.apache.hudi.metadata;
import org.apache.avro.Schema;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.exception.HoodieException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.io.Serializable;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
/**
* Provides functionality to convert timeline instants to table metadata records and then merge by key. Specify
* a filter to limit keys that are merged and stored in memory.
*/
public class HoodieMetadataMergedInstantRecordScanner {
public class TimelineMergedTableMetadata implements Serializable {
private static final Logger LOG = LogManager.getLogger(HoodieMetadataMergedInstantRecordScanner.class);
private static final Logger LOG = LogManager.getLogger(TimelineMergedTableMetadata.class);
HoodieTableMetaClient metaClient;
private List<HoodieInstant> instants;
private Option<String> lastSyncTs;
private Set<String> mergeKeyFilter;
protected final ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records;
public HoodieMetadataMergedInstantRecordScanner(HoodieTableMetaClient metaClient, List<HoodieInstant> instants,
Option<String> lastSyncTs, Schema readerSchema, Long maxMemorySizeInBytes,
String spillableMapBasePath, Set<String> mergeKeyFilter) throws IOException {
// keep it a simple hash map, so it can be easily passed onto the executors, once merged.
protected final Map<String, HoodieRecord<? extends HoodieRecordPayload>> timelineMergedRecords;
public TimelineMergedTableMetadata(HoodieTableMetaClient metaClient, List<HoodieInstant> instants,
Option<String> lastSyncTs, Set<String> mergeKeyFilter) {
this.metaClient = metaClient;
this.instants = instants;
this.lastSyncTs = lastSyncTs;
this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet();
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(),
new HoodieRecordSizeEstimator(readerSchema));
this.timelineMergedRecords = new HashMap<>();
scan();
}
@@ -92,13 +92,13 @@ public class HoodieMetadataMergedInstantRecordScanner {
private void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) {
String key = hoodieRecord.getRecordKey();
if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(key)) {
if (records.containsKey(key)) {
if (timelineMergedRecords.containsKey(key)) {
// Merge and store the merged record
HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(records.get(key).getData());
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue));
HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(timelineMergedRecords.get(key).getData(), new Properties());
timelineMergedRecords.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue));
} else {
// Put the record as is
records.put(key, hoodieRecord);
timelineMergedRecords.put(key, hoodieRecord);
}
}
}
@@ -110,6 +110,6 @@ public class HoodieMetadataMergedInstantRecordScanner {
* @return {@code HoodieRecord} if key was found else {@code Option.empty()}
*/
public Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKey(String key) {
return Option.ofNullable((HoodieRecord) records.get(key));
return Option.ofNullable((HoodieRecord) timelineMergedRecords.get(key));
}
}