[HUDI-842] Implementation of HUDI RFC-15.
- Introduced an internal metadata table, that stores file listings. - metadata table is kept upto date with - Fixed handling of CleanerPlan. - [HUDI-842] Reduce parallelism to speed up the test. - [HUDI-842] Implementation of CLI commands for metadata operations and lookups. - [HUDI-842] Extend rollback metadata to include the files which have been appended to. - [HUDI-842] Support for rollbacks in MOR Table. - MarkerBasedRollbackStrategy needs to correctly provide the list of files for which rollback blocks were appended. - [HUDI-842] Added unit test for rollback of partial commits (inflight but not completed yet). - [HUDI-842] Handled the error case where metadata update succeeds but dataset commit fails. - [HUDI-842] Schema evolution strategy for Metadata Table. Each type of metadata saved (FilesystemMetadata, ColumnIndexMetadata, etc.) will be a separate field with default null. The type of the record will identify the valid field. This way, we can grow the schema when new type of information is saved within in which still keeping it backward compatible. - [HUDI-842] Fix non-partitioned case and speedup initial creation of metadata table.Choose only 1 partition for jsc as the number of records is low (hundreds to thousands). There is more overhead of creating large number of partitions for JavaRDD and it slows down operations like WorkloadProfile. For the non-partitioned case, use "." as the name of the partition to prevent empty keys in HFile. - [HUDI-842] Reworked metrics pusblishing. - Code has been split into reader and writer side. HoodieMetadata code to be accessed by using HoodieTable.metadata() to get instance of metdata for the table. Code is serializable to allow executors to use the functionality. - [RFC-15] Add metrics to track the time for each file system call. - [RFC-15] Added a distributed metrics registry for spark which can be used to collect metrics from executors. This helps create a stats dashboard which shows the metadata table improvements in real-time for production tables. - [HUDI-1321] Created HoodieMetadataConfig to specify configuration for the metadata table. This is safer than full-fledged properties for the metadata table (like HoodieWriteConfig) as it makes burdensome to tune the metadata. With limited configuration, we can control the performance of the metadata table closely. [HUDI-1319][RFC-15] Adding interfaces for HoodieMetadata, HoodieMetadataWriter (apache#2266) - moved MetadataReader to HoodieBackedTableMetadata, under the HoodieTableMetadata interface - moved MetadataWriter to HoodieBackedTableMetadataWriter, under the HoodieTableMetadataWriter - Pulled all the metrics into HoodieMetadataMetrics - Writer now wraps the metadata, instead of extending it - New enum for MetadataPartitionType - Streamlined code flow inside HoodieBackedTableMetadataWriter w.r.t initializing metadata state - [HUDI-1319] Make async operations work with metadata table (apache#2332) - Changes the syncing model to only move over completed instants on data timeline - Syncing happens postCommit and on writeClient initialization - Latest delta commit on the metadata table is sufficient as the watermark for data timeline archival - Cleaning/Compaction use a suffix to the last instant written to metadata table, such that we keep the 1-1 - .. mapping between data and metadata timelines. - Got rid of a lot of the complexity around checking for valid commits during open of base/log files - Tests now use local FS, to simulate more failure scenarios - Some failure scenarios exposed HUDI-1434, which is needed for MOR to work correctly co-authored by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
committed by
vinoth chandar
parent
c3e9243ea1
commit
298808baaf
56
hudi-common/src/main/avro/HoodieMetadata.avsc
Normal file
56
hudi-common/src/main/avro/HoodieMetadata.avsc
Normal file
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
{
|
||||
"namespace": "org.apache.hudi.avro.model",
|
||||
"type": "record",
|
||||
"name": "HoodieMetadataRecord",
|
||||
"doc": "A record saved within the Metadata Table",
|
||||
"fields": [
|
||||
{
|
||||
"name": "key",
|
||||
"type": "string"
|
||||
},
|
||||
{
|
||||
"name": "type",
|
||||
"doc": "Type of the metadata record",
|
||||
"type": "int"
|
||||
},
|
||||
{ "name": "filesystemMetadata",
|
||||
"doc": "Contains information about partitions and files within the dataset",
|
||||
"type": ["null", {
|
||||
"type": "map",
|
||||
"values": {
|
||||
"type": "record",
|
||||
"name": "HoodieMetadataFileInfo",
|
||||
"fields": [
|
||||
{
|
||||
"name": "size",
|
||||
"type": "long",
|
||||
"doc": "Size of the file"
|
||||
},
|
||||
{
|
||||
"name": "isDeleted",
|
||||
"type": "boolean",
|
||||
"doc": "True if this file has been deleted"
|
||||
}
|
||||
]
|
||||
}
|
||||
}]
|
||||
}
|
||||
]
|
||||
}
|
||||
@@ -30,7 +30,15 @@
|
||||
"fields": [
|
||||
{"name": "partitionPath", "type": "string"},
|
||||
{"name": "successDeleteFiles", "type": {"type": "array", "items": "string"}},
|
||||
{"name": "failedDeleteFiles", "type": {"type": "array", "items": "string"}}
|
||||
{"name": "failedDeleteFiles", "type": {"type": "array", "items": "string"}},
|
||||
{"name": "appendFiles", "type": {
|
||||
"type": "map",
|
||||
"doc": "Files to which append blocks were written",
|
||||
"values": {
|
||||
"type": "long",
|
||||
"doc": "Size of this file in bytes"
|
||||
}
|
||||
}}
|
||||
]
|
||||
}
|
||||
}
|
||||
|
||||
@@ -46,6 +46,8 @@ import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
@@ -193,8 +195,17 @@ public class FSUtils {
|
||||
/**
|
||||
* Obtain all the partition paths, that are present in this table, denoted by presence of
|
||||
* {@link HoodiePartitionMetadata#HOODIE_PARTITION_METAFILE}.
|
||||
*
|
||||
* If the basePathStr is a subdirectory of .hoodie folder then we assume that the partitions of an internal
|
||||
* table (a hoodie table within the .hoodie directory) are to be obtained.
|
||||
*
|
||||
* @param fs FileSystem instance
|
||||
* @param basePathStr base directory
|
||||
*/
|
||||
public static List<String> getAllFoldersWithPartitionMetaFile(FileSystem fs, String basePathStr) throws IOException {
|
||||
// If the basePathStr is a folder within the .hoodie directory then we are listing partitions within an
|
||||
// internal table.
|
||||
final boolean isMetadataTable = basePathStr.contains(HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
final Path basePath = new Path(basePathStr);
|
||||
final List<String> partitions = new ArrayList<>();
|
||||
processFiles(fs, basePathStr, (locatedFileStatus) -> {
|
||||
@@ -203,7 +214,7 @@ public class FSUtils {
|
||||
partitions.add(getRelativePartitionPath(basePath, filePath.getParent()));
|
||||
}
|
||||
return true;
|
||||
}, true);
|
||||
}, !isMetadataTable);
|
||||
return partitions;
|
||||
}
|
||||
|
||||
@@ -384,6 +395,20 @@ public class FSUtils {
|
||||
return matcher.find() && logPath.getName().contains(".log");
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the names of all the base and log files in the given partition path.
|
||||
*/
|
||||
public static FileStatus[] getAllDataFilesInPartition(FileSystem fs, Path partitionPath) throws IOException {
|
||||
final Set<String> validFileExtensions = Arrays.stream(HoodieFileFormat.values())
|
||||
.map(HoodieFileFormat::getFileExtension).collect(Collectors.toCollection(HashSet::new));
|
||||
final String logFileExtension = HoodieFileFormat.HOODIE_LOG.getFileExtension();
|
||||
|
||||
return Arrays.stream(fs.listStatus(partitionPath, path -> {
|
||||
String extension = FSUtils.getFileExtension(path.getName());
|
||||
return validFileExtensions.contains(extension) || path.getName().contains(logFileExtension);
|
||||
})).filter(FileStatus::isFile).toArray(FileStatus[]::new);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the latest log file written from the list of log files passed in.
|
||||
*/
|
||||
|
||||
@@ -19,6 +19,8 @@
|
||||
package org.apache.hudi.common.fs;
|
||||
|
||||
import org.apache.hudi.common.metrics.Registry;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
@@ -65,15 +67,56 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
public static final String HOODIE_SCHEME_PREFIX = "hoodie-";
|
||||
|
||||
private enum MetricName {
|
||||
create, rename, delete, listStatus, mkdirs, getFileStatus, globStatus, listFiles
|
||||
protected enum MetricName {
|
||||
create, rename, delete, listStatus, mkdirs, getFileStatus, globStatus, listFiles, read, write
|
||||
}
|
||||
|
||||
private static Registry METRICS_REGISTRY_DATA;
|
||||
private static Registry METRICS_REGISTRY_META;
|
||||
|
||||
public static void setMetricsRegistry(Registry registry, Registry registryMeta) {
|
||||
METRICS_REGISTRY_DATA = registry;
|
||||
METRICS_REGISTRY_META = registryMeta;
|
||||
}
|
||||
|
||||
|
||||
private ConcurrentMap<String, SizeAwareFSDataOutputStream> openStreams = new ConcurrentHashMap<>();
|
||||
private FileSystem fileSystem;
|
||||
private URI uri;
|
||||
private ConsistencyGuard consistencyGuard = new NoOpConsistencyGuard();
|
||||
private Registry metricsRegistry = Registry.getRegistry(this.getClass().getSimpleName());
|
||||
|
||||
@FunctionalInterface
|
||||
public interface CheckedFunction<R> {
|
||||
R get() throws IOException;
|
||||
}
|
||||
|
||||
private static Registry getMetricRegistryForPath(Path p) {
|
||||
return ((p != null) && (p.toString().contains(HoodieTableMetaClient.METAFOLDER_NAME)))
|
||||
? METRICS_REGISTRY_META : METRICS_REGISTRY_DATA;
|
||||
}
|
||||
|
||||
protected static <R> R executeFuncWithTimeMetrics(String metricName, Path p, CheckedFunction<R> func) throws IOException {
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
R res = func.get();
|
||||
|
||||
Registry registry = getMetricRegistryForPath(p);
|
||||
if (registry != null) {
|
||||
registry.increment(metricName);
|
||||
registry.add(metricName + ".totalDuration", timer.endTimer());
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
protected static <R> R executeFuncWithTimeAndByteMetrics(String metricName, Path p, long byteCount,
|
||||
CheckedFunction<R> func) throws IOException {
|
||||
Registry registry = getMetricRegistryForPath(p);
|
||||
if (registry != null) {
|
||||
registry.add(metricName + ".totalBytes", byteCount);
|
||||
}
|
||||
|
||||
return executeFuncWithTimeMetrics(metricName, p, func);
|
||||
}
|
||||
|
||||
public HoodieWrapperFileSystem() {}
|
||||
|
||||
@@ -140,16 +183,17 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
|
||||
return fileSystem.open(convertToDefaultPath(f), bufferSize);
|
||||
return wrapInputStream(f, fileSystem.open(convertToDefaultPath(f), bufferSize));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize,
|
||||
short replication, long blockSize, Progressable progress) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
final Path translatedPath = convertToDefaultPath(f);
|
||||
return wrapOutputStream(f,
|
||||
fileSystem.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize, progress));
|
||||
short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
final Path translatedPath = convertToDefaultPath(f);
|
||||
return wrapOutputStream(f,
|
||||
fileSystem.create(translatedPath, permission, overwrite, bufferSize, replication, blockSize, progress));
|
||||
});
|
||||
}
|
||||
|
||||
private FSDataOutputStream wrapOutputStream(final Path path, FSDataOutputStream fsDataOutputStream)
|
||||
@@ -164,79 +208,97 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
return os;
|
||||
}
|
||||
|
||||
private FSDataInputStream wrapInputStream(final Path path, FSDataInputStream fsDataInputStream) throws IOException {
|
||||
if (fsDataInputStream instanceof TimedFSDataInputStream) {
|
||||
return fsDataInputStream;
|
||||
}
|
||||
return new TimedFSDataInputStream(path, fsDataInputStream);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite));
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f)));
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f)));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, Progressable progress) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), progress));
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), progress));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, short replication) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication));
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, short replication, Progressable progress) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication, progress));
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), replication, progress));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize));
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, Progressable progress)
|
||||
throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress));
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, progress));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize,
|
||||
Progressable progress) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f,
|
||||
fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress));
|
||||
Progressable progress) throws IOException {
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f,
|
||||
fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize, progress));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags, int bufferSize,
|
||||
short replication, long blockSize, Progressable progress) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f,
|
||||
fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize, progress));
|
||||
short replication, long blockSize, Progressable progress) throws IOException {
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f,
|
||||
fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication, blockSize, progress));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission, EnumSet<CreateFlag> flags, int bufferSize,
|
||||
short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication,
|
||||
blockSize, progress, checksumOpt));
|
||||
short replication, long blockSize, Progressable progress, Options.ChecksumOpt checksumOpt) throws IOException {
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f, fileSystem.create(convertToDefaultPath(f), permission, flags, bufferSize, replication,
|
||||
blockSize, progress, checksumOpt));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize, short replication, long blockSize)
|
||||
throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.create.name());
|
||||
return wrapOutputStream(f,
|
||||
fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize));
|
||||
return executeFuncWithTimeMetrics(MetricName.create.name(), f, () -> {
|
||||
return wrapOutputStream(f,
|
||||
fileSystem.create(convertToDefaultPath(f), overwrite, bufferSize, replication, blockSize));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -246,50 +308,53 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public boolean rename(Path src, Path dst) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.rename.name());
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(src));
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for " + src + " to appear", e);
|
||||
}
|
||||
|
||||
boolean success = fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
|
||||
if (success) {
|
||||
return executeFuncWithTimeMetrics(MetricName.rename.name(), src, () -> {
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(src));
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for " + dst + " to appear", e);
|
||||
throw new HoodieException("Timed out waiting for " + src + " to appear", e);
|
||||
}
|
||||
|
||||
try {
|
||||
consistencyGuard.waitTillFileDisappears(convertToDefaultPath(src));
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for " + src + " to disappear", e);
|
||||
boolean success = fileSystem.rename(convertToDefaultPath(src), convertToDefaultPath(dst));
|
||||
|
||||
if (success) {
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(dst));
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for " + dst + " to appear", e);
|
||||
}
|
||||
|
||||
try {
|
||||
consistencyGuard.waitTillFileDisappears(convertToDefaultPath(src));
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for " + src + " to disappear", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
return success;
|
||||
return success;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean delete(Path f, boolean recursive) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.delete.name());
|
||||
boolean success = fileSystem.delete(convertToDefaultPath(f), recursive);
|
||||
return executeFuncWithTimeMetrics(MetricName.delete.name(), f, () -> {
|
||||
boolean success = fileSystem.delete(convertToDefaultPath(f), recursive);
|
||||
|
||||
if (success) {
|
||||
try {
|
||||
consistencyGuard.waitTillFileDisappears(f);
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for " + f + " to disappear", e);
|
||||
if (success) {
|
||||
try {
|
||||
consistencyGuard.waitTillFileDisappears(f);
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for " + f + " to disappear", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
return success;
|
||||
return success;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path f) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.listStatus.name());
|
||||
return fileSystem.listStatus(convertToDefaultPath(f));
|
||||
return executeFuncWithTimeMetrics(MetricName.listStatus.name(), f, () -> {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -304,27 +369,29 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.mkdirs.name());
|
||||
boolean success = fileSystem.mkdirs(convertToDefaultPath(f), permission);
|
||||
if (success) {
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
|
||||
return executeFuncWithTimeMetrics(MetricName.mkdirs.name(), f, () -> {
|
||||
boolean success = fileSystem.mkdirs(convertToDefaultPath(f), permission);
|
||||
if (success) {
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
return success;
|
||||
return success;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus getFileStatus(Path f) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.getFileStatus.name());
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
|
||||
} catch (TimeoutException e) {
|
||||
// pass
|
||||
}
|
||||
return fileSystem.getFileStatus(convertToDefaultPath(f));
|
||||
return executeFuncWithTimeMetrics(MetricName.getFileStatus.name(), f, () -> {
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
|
||||
} catch (TimeoutException e) {
|
||||
// pass
|
||||
}
|
||||
return fileSystem.getFileStatus(convertToDefaultPath(f));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -389,12 +456,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public FSDataInputStream open(Path f) throws IOException {
|
||||
return fileSystem.open(convertToDefaultPath(f));
|
||||
return wrapInputStream(f, fileSystem.open(convertToDefaultPath(f)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize, short replication,
|
||||
long blockSize, Progressable progress) throws IOException {
|
||||
long blockSize, Progressable progress) throws IOException {
|
||||
Path p = convertToDefaultPath(f);
|
||||
return wrapOutputStream(p,
|
||||
fileSystem.createNonRecursive(p, overwrite, bufferSize, replication, blockSize, progress));
|
||||
@@ -402,7 +469,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite, int bufferSize,
|
||||
short replication, long blockSize, Progressable progress) throws IOException {
|
||||
short replication, long blockSize, Progressable progress) throws IOException {
|
||||
Path p = convertToDefaultPath(f);
|
||||
return wrapOutputStream(p,
|
||||
fileSystem.createNonRecursive(p, permission, overwrite, bufferSize, replication, blockSize, progress));
|
||||
@@ -410,7 +477,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, EnumSet<CreateFlag> flags,
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
|
||||
Path p = convertToDefaultPath(f);
|
||||
return wrapOutputStream(p,
|
||||
fileSystem.createNonRecursive(p, permission, flags, bufferSize, replication, blockSize, progress));
|
||||
@@ -462,8 +529,9 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public boolean delete(Path f) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.delete.name());
|
||||
return delete(f, true);
|
||||
return executeFuncWithTimeMetrics(MetricName.delete.name(), f, () -> {
|
||||
return delete(f, true);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -508,32 +576,37 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path f, PathFilter filter) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.listStatus.name());
|
||||
return fileSystem.listStatus(convertToDefaultPath(f), filter);
|
||||
return executeFuncWithTimeMetrics(MetricName.listStatus.name(), f, () -> {
|
||||
return fileSystem.listStatus(convertToDefaultPath(f), filter);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.listStatus.name());
|
||||
return fileSystem.listStatus(convertDefaults(files));
|
||||
return executeFuncWithTimeMetrics(MetricName.listStatus.name(), files.length > 0 ? files[0] : null, () -> {
|
||||
return fileSystem.listStatus(convertDefaults(files));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path[] files, PathFilter filter) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.listStatus.name());
|
||||
return fileSystem.listStatus(convertDefaults(files), filter);
|
||||
return executeFuncWithTimeMetrics(MetricName.listStatus.name(), files.length > 0 ? files[0] : null, () -> {
|
||||
return fileSystem.listStatus(convertDefaults(files), filter);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] globStatus(Path pathPattern) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.globStatus.name());
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern));
|
||||
return executeFuncWithTimeMetrics(MetricName.globStatus.name(), pathPattern, () -> {
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern));
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus[] globStatus(Path pathPattern, PathFilter filter) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.globStatus.name());
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter);
|
||||
return executeFuncWithTimeMetrics(MetricName.globStatus.name(), pathPattern, () -> {
|
||||
return fileSystem.globStatus(convertToDefaultPath(pathPattern), filter);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -543,8 +616,9 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.listFiles.name());
|
||||
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
|
||||
return executeFuncWithTimeMetrics(MetricName.listFiles.name(), f, () -> {
|
||||
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -554,16 +628,17 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
|
||||
@Override
|
||||
public boolean mkdirs(Path f) throws IOException {
|
||||
this.metricsRegistry.increment(MetricName.mkdirs.name());
|
||||
boolean success = fileSystem.mkdirs(convertToDefaultPath(f));
|
||||
if (success) {
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
|
||||
return executeFuncWithTimeMetrics(MetricName.mkdirs.name(), f, () -> {
|
||||
boolean success = fileSystem.mkdirs(convertToDefaultPath(f));
|
||||
if (success) {
|
||||
try {
|
||||
consistencyGuard.waitTillFileAppears(convertToDefaultPath(f));
|
||||
} catch (TimeoutException e) {
|
||||
throw new HoodieException("Timed out waiting for directory " + f + " to appear", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
return success;
|
||||
return success;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -43,7 +43,7 @@ public class SizeAwareFSDataOutputStream extends FSDataOutputStream {
|
||||
private final ConsistencyGuard consistencyGuard;
|
||||
|
||||
public SizeAwareFSDataOutputStream(Path path, FSDataOutputStream out, ConsistencyGuard consistencyGuard,
|
||||
Runnable closeCallback) throws IOException {
|
||||
Runnable closeCallback) throws IOException {
|
||||
super(out, null);
|
||||
this.path = path;
|
||||
this.closeCallback = closeCallback;
|
||||
@@ -52,14 +52,22 @@ public class SizeAwareFSDataOutputStream extends FSDataOutputStream {
|
||||
|
||||
@Override
|
||||
public synchronized void write(byte[] b, int off, int len) throws IOException {
|
||||
bytesWritten.addAndGet(len);
|
||||
super.write(b, off, len);
|
||||
HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.write.name(), path,
|
||||
len, () -> {
|
||||
bytesWritten.addAndGet(len);
|
||||
super.write(b, off, len);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] b) throws IOException {
|
||||
bytesWritten.addAndGet(b.length);
|
||||
super.write(b);
|
||||
HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.write.name(), path,
|
||||
b.length, () -> {
|
||||
bytesWritten.addAndGet(b.length);
|
||||
super.write(b);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -76,5 +84,4 @@ public class SizeAwareFSDataOutputStream extends FSDataOutputStream {
|
||||
public long getBytesWritten() {
|
||||
return bytesWritten.get();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -0,0 +1,79 @@
|
||||
/*
|
||||
* 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.fs;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.ReadOption;
|
||||
import org.apache.hadoop.io.ByteBufferPool;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.EnumSet;
|
||||
|
||||
/**
|
||||
* Wrapper over <code>FSDataInputStream</code> that also times the operations.
|
||||
*/
|
||||
public class TimedFSDataInputStream extends FSDataInputStream {
|
||||
|
||||
// Path
|
||||
private final Path path;
|
||||
|
||||
public TimedFSDataInputStream(Path path, FSDataInputStream in) {
|
||||
super(in);
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(ByteBuffer buf) throws IOException {
|
||||
return HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(),
|
||||
path, 0, () -> super.read(buf));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(long position, byte[] buffer, int offset, int length) throws IOException {
|
||||
return HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(),
|
||||
path, length, () -> super.read(position, buffer, offset, length));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ByteBuffer read(ByteBufferPool bufferPool, int maxLength, EnumSet<ReadOption> opts)
|
||||
throws IOException, UnsupportedOperationException {
|
||||
return HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(),
|
||||
path, maxLength, () -> super.read(bufferPool, maxLength, opts));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFully(long position, byte[] buffer) throws IOException {
|
||||
HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(),
|
||||
path, buffer.length, () -> {
|
||||
super.readFully(position, buffer);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFully(long position, byte[] buffer, int offset, int length) throws IOException {
|
||||
HoodieWrapperFileSystem.executeFuncWithTimeAndByteMetrics(HoodieWrapperFileSystem.MetricName.read.name(),
|
||||
path, length, () -> {
|
||||
super.readFully(position, buffer, offset, length);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* 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.metrics;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
/**
|
||||
* Registry that tracks metrics local to a single jvm process.
|
||||
*/
|
||||
public class LocalRegistry implements Registry {
|
||||
ConcurrentHashMap<String, Counter> counters = new ConcurrentHashMap<>();
|
||||
private final String name;
|
||||
|
||||
public LocalRegistry(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
counters.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void increment(String name) {
|
||||
getCounter(name).increment();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(String name, long value) {
|
||||
getCounter(name).add(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all Counter type metrics.
|
||||
*/
|
||||
@Override
|
||||
public Map<String, Long> getAllCounts(boolean prefixWithRegistryName) {
|
||||
HashMap<String, Long> countersMap = new HashMap<>();
|
||||
counters.forEach((k, v) -> {
|
||||
String key = prefixWithRegistryName ? name + "." + k : k;
|
||||
countersMap.put(key, v.getValue());
|
||||
});
|
||||
return countersMap;
|
||||
}
|
||||
|
||||
private synchronized Counter getCounter(String name) {
|
||||
if (!counters.containsKey(name)) {
|
||||
counters.put(name, new Counter());
|
||||
}
|
||||
return counters.get(name);
|
||||
}
|
||||
}
|
||||
@@ -18,87 +18,99 @@
|
||||
|
||||
package org.apache.hudi.common.metrics;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
|
||||
|
||||
/**
|
||||
* Lightweight Metrics Registry to track Hudi events.
|
||||
* Interface which defines a lightweight Metrics Registry to track Hudi events.
|
||||
*/
|
||||
public class Registry {
|
||||
ConcurrentHashMap<String, Counter> counters = new ConcurrentHashMap<>();
|
||||
final String name;
|
||||
public interface Registry extends Serializable {
|
||||
|
||||
private static ConcurrentHashMap<String, Registry> registryMap = new ConcurrentHashMap<>();
|
||||
|
||||
private Registry(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
ConcurrentHashMap<String, Registry> REGISTRY_MAP = new ConcurrentHashMap<>();
|
||||
|
||||
/**
|
||||
* Get (or create) the registry for a provided name.
|
||||
*
|
||||
* This function creates a {@code LocalRegistry}.
|
||||
*
|
||||
* @param registryName Name of the registry
|
||||
*/
|
||||
public static synchronized Registry getRegistry(String registryName) {
|
||||
if (!registryMap.containsKey(registryName)) {
|
||||
registryMap.put(registryName, new Registry(registryName));
|
||||
static Registry getRegistry(String registryName) {
|
||||
return getRegistry(registryName, LocalRegistry.class.getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get (or create) the registry for a provided name and given class.
|
||||
*
|
||||
* @param registryName Name of the registry.
|
||||
* @param clazz The fully qualified name of the registry class to create.
|
||||
*/
|
||||
static Registry getRegistry(String registryName, String clazz) {
|
||||
synchronized (Registry.class) {
|
||||
if (!REGISTRY_MAP.containsKey(registryName)) {
|
||||
Registry registry = (Registry)ReflectionUtils.loadClass(clazz, registryName);
|
||||
REGISTRY_MAP.put(registryName, registry);
|
||||
}
|
||||
return REGISTRY_MAP.get(registryName);
|
||||
}
|
||||
return registryMap.get(registryName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all registered metrics.
|
||||
* @param flush clean all metrics as part of this operation.
|
||||
*
|
||||
* @param flush clear all metrics after this operation.
|
||||
* @param prefixWithRegistryName prefix each metric name with the registry name.
|
||||
* @return
|
||||
*/
|
||||
public static synchronized Map<String, Long> getAllMetrics(boolean flush, boolean prefixWithRegistryName) {
|
||||
HashMap<String, Long> allMetrics = new HashMap<>();
|
||||
registryMap.forEach((registryName, registry) -> {
|
||||
allMetrics.putAll(registry.getAllCounts(prefixWithRegistryName));
|
||||
if (flush) {
|
||||
registry.clear();
|
||||
}
|
||||
});
|
||||
return allMetrics;
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
counters.clear();
|
||||
}
|
||||
|
||||
public void increment(String name) {
|
||||
getCounter(name).increment();
|
||||
}
|
||||
|
||||
public void add(String name, long value) {
|
||||
getCounter(name).add(value);
|
||||
}
|
||||
|
||||
private synchronized Counter getCounter(String name) {
|
||||
if (!counters.containsKey(name)) {
|
||||
counters.put(name, new Counter());
|
||||
static Map<String, Long> getAllMetrics(boolean flush, boolean prefixWithRegistryName) {
|
||||
synchronized (Registry.class) {
|
||||
HashMap<String, Long> allMetrics = new HashMap<>();
|
||||
REGISTRY_MAP.forEach((registryName, registry) -> {
|
||||
allMetrics.putAll(registry.getAllCounts(prefixWithRegistryName));
|
||||
if (flush) {
|
||||
registry.clear();
|
||||
}
|
||||
});
|
||||
return allMetrics;
|
||||
}
|
||||
return counters.get(name);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clear all metrics.
|
||||
*/
|
||||
void clear();
|
||||
|
||||
/**
|
||||
* Increment the metric.
|
||||
*
|
||||
* @param name Name of the metric to increment.
|
||||
*/
|
||||
void increment(String name);
|
||||
|
||||
/**
|
||||
* Add value to the metric.
|
||||
*
|
||||
* @param name Name of the metric.
|
||||
* @param value The value to add to the metrics.
|
||||
*/
|
||||
void add(String name, long value);
|
||||
|
||||
/**
|
||||
* Get all Counter type metrics.
|
||||
*/
|
||||
public Map<String, Long> getAllCounts() {
|
||||
default Map<String, Long> getAllCounts() {
|
||||
return getAllCounts(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all Counter type metrics.
|
||||
*
|
||||
* @param prefixWithRegistryName If true, the names of all metrics are prefixed with name of this registry.
|
||||
*/
|
||||
public Map<String, Long> getAllCounts(boolean prefixWithRegistryName) {
|
||||
HashMap<String, Long> countersMap = new HashMap<>();
|
||||
counters.forEach((k, v) -> {
|
||||
String key = prefixWithRegistryName ? name + "." + k : k;
|
||||
countersMap.put(key, v.getValue());
|
||||
});
|
||||
return countersMap;
|
||||
}
|
||||
|
||||
}
|
||||
Map<String, Long> getAllCounts(boolean prefixWithRegistryName);
|
||||
}
|
||||
|
||||
@@ -80,7 +80,7 @@ public abstract class AbstractHoodieLogRecordScanner {
|
||||
// Merge strategy to use when combining records from log
|
||||
private final String payloadClassFQN;
|
||||
// Log File Paths
|
||||
private final List<String> logFilePaths;
|
||||
protected final List<String> logFilePaths;
|
||||
// Read Lazily flag
|
||||
private final boolean readBlocksLazily;
|
||||
// Reverse reader - Not implemented yet (NA -> Why do we need ?)
|
||||
@@ -148,7 +148,8 @@ public abstract class AbstractHoodieLogRecordScanner {
|
||||
switch (r.getBlockType()) {
|
||||
case HFILE_DATA_BLOCK:
|
||||
case AVRO_DATA_BLOCK:
|
||||
LOG.info("Reading a data block from file " + logFile.getPath());
|
||||
LOG.info("Reading a data block from file " + logFile.getPath() + " at instant "
|
||||
+ r.getLogBlockHeader().get(INSTANT_TIME));
|
||||
if (isNewInstantBlock(r) && !readBlocksLazily) {
|
||||
// If this is an avro data block belonging to a different commit/instant,
|
||||
// then merge the last blocks and records into the main result
|
||||
@@ -202,8 +203,7 @@ public abstract class AbstractHoodieLogRecordScanner {
|
||||
LOG.info("Rolling back the last corrupted log block read in " + logFile.getPath());
|
||||
currentInstantLogBlocks.pop();
|
||||
numBlocksRolledBack++;
|
||||
} else if (lastBlock.getBlockType() != CORRUPT_BLOCK
|
||||
&& targetInstantForCommandBlock.contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) {
|
||||
} else if (targetInstantForCommandBlock.contentEquals(lastBlock.getLogBlockHeader().get(INSTANT_TIME))) {
|
||||
// rollback last data block or delete block
|
||||
LOG.info("Rolling back the last log block read in " + logFile.getPath());
|
||||
currentInstantLogBlocks.pop();
|
||||
@@ -278,12 +278,14 @@ public abstract class AbstractHoodieLogRecordScanner {
|
||||
List<IndexedRecord> recs = dataBlock.getRecords();
|
||||
totalLogRecords.addAndGet(recs.size());
|
||||
for (IndexedRecord rec : recs) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord =
|
||||
SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN);
|
||||
processNextRecord(hoodieRecord);
|
||||
processNextRecord(createHoodieRecord(rec));
|
||||
}
|
||||
}
|
||||
|
||||
protected HoodieRecord<?> createHoodieRecord(IndexedRecord rec) {
|
||||
return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN);
|
||||
}
|
||||
|
||||
/**
|
||||
* Process next record.
|
||||
*
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.common.table.log;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.fs.TimedFSDataInputStream;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
|
||||
@@ -73,8 +74,8 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
boolean readBlockLazily, boolean reverseReader) throws IOException {
|
||||
FSDataInputStream fsDataInputStream = fs.open(logFile.getPath(), bufferSize);
|
||||
if (fsDataInputStream.getWrappedStream() instanceof FSInputStream) {
|
||||
this.inputStream = new FSDataInputStream(
|
||||
new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize));
|
||||
this.inputStream = new TimedFSDataInputStream(logFile.getPath(), new FSDataInputStream(
|
||||
new BufferedFSInputStream((FSInputStream) fsDataInputStream.getWrappedStream(), bufferSize)));
|
||||
} else {
|
||||
// fsDataInputStream.getWrappedStream() maybe a BufferedFSInputStream
|
||||
// need to wrap in another BufferedFSInputStream the make bufferSize work?
|
||||
|
||||
@@ -57,39 +57,56 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieMergedLogRecordScanner.class);
|
||||
|
||||
// Final map of compacted/merged records
|
||||
private final ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records;
|
||||
protected final ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records;
|
||||
|
||||
// count of merged records in log
|
||||
private long numMergedRecordsInLog;
|
||||
private long maxMemorySizeInBytes;
|
||||
|
||||
// Stores the total time taken to perform reading and merging of log blocks
|
||||
private final long totalTimeTakenToReadAndMergeBlocks;
|
||||
private long totalTimeTakenToReadAndMergeBlocks;
|
||||
// A timer for calculating elapsed time in millis
|
||||
public final HoodieTimer timer = new HoodieTimer();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema,
|
||||
String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily, boolean reverseReader,
|
||||
int bufferSize, String spillableMapBasePath) {
|
||||
String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily,
|
||||
boolean reverseReader, int bufferSize, String spillableMapBasePath) {
|
||||
this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, readBlocksLazily,
|
||||
reverseReader, bufferSize, spillableMapBasePath, true);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema,
|
||||
String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily,
|
||||
boolean reverseReader, int bufferSize, String spillableMapBasePath, boolean autoScan) {
|
||||
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize);
|
||||
try {
|
||||
// Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
|
||||
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(),
|
||||
new HoodieRecordSizeEstimator(readerSchema));
|
||||
// Do the scan and merge
|
||||
timer.startTimer();
|
||||
scan();
|
||||
this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer();
|
||||
this.numMergedRecordsInLog = records.size();
|
||||
LOG.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes);
|
||||
LOG.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries());
|
||||
LOG.info(
|
||||
"Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize());
|
||||
LOG.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries());
|
||||
LOG.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("IOException when reading log file ", e);
|
||||
throw new HoodieIOException("IOException when creating ExternalSpillableMap at " + spillableMapBasePath, e);
|
||||
}
|
||||
|
||||
if (autoScan) {
|
||||
performScan();
|
||||
}
|
||||
}
|
||||
|
||||
protected void performScan() {
|
||||
// Do the scan and merge
|
||||
timer.startTimer();
|
||||
scan();
|
||||
this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer();
|
||||
this.numMergedRecordsInLog = records.size();
|
||||
LOG.info("Number of log files scanned => " + logFilePaths.size());
|
||||
LOG.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes);
|
||||
LOG.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries());
|
||||
LOG.info(
|
||||
"Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize());
|
||||
LOG.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries());
|
||||
LOG.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -118,6 +119,8 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
recordKey = record.get(keyField.pos()).toString();
|
||||
}
|
||||
byte[] recordBytes = HoodieAvroUtils.indexedRecordToBytes(record);
|
||||
ValidationUtils.checkState(!sortedRecordsMap.containsKey(recordKey),
|
||||
"Writing multiple records with same key not supported for " + this.getClass().getName());
|
||||
sortedRecordsMap.put(recordKey, recordBytes);
|
||||
}
|
||||
|
||||
|
||||
@@ -62,12 +62,16 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
|
||||
|
||||
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList(
|
||||
COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
|
||||
INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION,
|
||||
INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
|
||||
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION,
|
||||
REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION));
|
||||
|
||||
COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION,
|
||||
DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION,
|
||||
SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION,
|
||||
CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
|
||||
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION,
|
||||
INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION,
|
||||
ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION,
|
||||
REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION
|
||||
));
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
private static AtomicReference<String> lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE));
|
||||
|
||||
@@ -27,6 +27,8 @@ import org.apache.avro.io.DatumWriter;
|
||||
import org.apache.avro.specific.SpecificDatumReader;
|
||||
import org.apache.avro.specific.SpecificDatumWriter;
|
||||
import org.apache.avro.specific.SpecificRecordBase;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
@@ -68,8 +70,10 @@ public class TimelineMetadataUtils {
|
||||
Map<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder = new HashMap<>();
|
||||
int totalDeleted = 0;
|
||||
for (HoodieRollbackStat stat : rollbackStats) {
|
||||
Map<String, Long> appendFiles = stat.getCommandBlocksCount().keySet().stream()
|
||||
.collect(Collectors.toMap(f -> f.getPath().toString(), FileStatus::getLen));
|
||||
HoodieRollbackPartitionMetadata metadata = new HoodieRollbackPartitionMetadata(stat.getPartitionPath(),
|
||||
stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles());
|
||||
stat.getSuccessDeleteFiles(), stat.getFailedDeleteFiles(), appendFiles);
|
||||
partitionMetadataBuilder.put(stat.getPartitionPath(), metadata);
|
||||
totalDeleted += stat.getSuccessDeleteFiles().size();
|
||||
}
|
||||
@@ -146,6 +150,10 @@ public class TimelineMetadataUtils {
|
||||
return deserializeAvroMetadata(bytes, HoodieRollbackMetadata.class);
|
||||
}
|
||||
|
||||
public static HoodieRestoreMetadata deserializeHoodieRestoreMetadata(byte[] bytes) throws IOException {
|
||||
return deserializeAvroMetadata(bytes, HoodieRestoreMetadata.class);
|
||||
}
|
||||
|
||||
public static HoodieSavepointMetadata deserializeHoodieSavepointMetadata(byte[] bytes) throws IOException {
|
||||
return deserializeAvroMetadata(bytes, HoodieSavepointMetadata.class);
|
||||
}
|
||||
|
||||
@@ -276,7 +276,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
|
||||
Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPathStr);
|
||||
FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
|
||||
long beginLsTs = System.currentTimeMillis();
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
|
||||
FileStatus[] statuses = listPartition(partitionPath);
|
||||
long endLsTs = System.currentTimeMillis();
|
||||
LOG.info("#files found in partition (" + partitionPathStr + ") =" + statuses.length + ", Time taken ="
|
||||
+ (endLsTs - beginLsTs));
|
||||
@@ -297,6 +297,16 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Return all the files from the partition.
|
||||
*
|
||||
* @param partitionPath The absolute path of the partition
|
||||
* @throws IOException
|
||||
*/
|
||||
protected FileStatus[] listPartition(Path partitionPath) throws IOException {
|
||||
return metaClient.getFs().listStatus(partitionPath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to convert file-status to base-files.
|
||||
*
|
||||
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception thrown for table metadata related failures.
|
||||
* </p>
|
||||
*/
|
||||
public class HoodieMetadataException extends HoodieException {
|
||||
public HoodieMetadataException(String msg, Exception t) {
|
||||
super(msg, t);
|
||||
}
|
||||
|
||||
public HoodieMetadataException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -18,9 +18,11 @@
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
@@ -39,16 +41,14 @@ public class TableNotFoundException extends HoodieException {
|
||||
}
|
||||
|
||||
public static void checkTableValidity(FileSystem fs, Path basePathDir, Path metaPathDir) {
|
||||
// Check if the base path is found
|
||||
// Check if the base and meta paths are found
|
||||
try {
|
||||
if (!fs.exists(basePathDir) || !fs.isDirectory(basePathDir)) {
|
||||
throw new TableNotFoundException(basePathDir.toString());
|
||||
}
|
||||
// Check if the meta path is found
|
||||
if (!fs.exists(metaPathDir) || !fs.isDirectory(metaPathDir)) {
|
||||
// Since metaPath is within the basePath, it is enough to check the metaPath exists
|
||||
FileStatus status = fs.getFileStatus(metaPathDir);
|
||||
if (!status.isDirectory()) {
|
||||
throw new TableNotFoundException(metaPathDir.toString());
|
||||
}
|
||||
} catch (IllegalArgumentException e) {
|
||||
} catch (FileNotFoundException | IllegalArgumentException e) {
|
||||
// if the base path is file:///, then we have a IllegalArgumentException
|
||||
throw new TableNotFoundException(metaPathDir.toString());
|
||||
} catch (IOException e) {
|
||||
|
||||
@@ -0,0 +1,447 @@
|
||||
/*
|
||||
* 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 java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
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.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataRecord;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.metrics.Registry;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
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.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.HoodieMetadataException;
|
||||
import org.apache.hudi.exception.TableNotFoundException;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Table metadata provided by an internal DFS backed Hudi metadata table.
|
||||
*
|
||||
* If the metadata table does not exist, RPC calls are used to retrieve file listings from the file system.
|
||||
* No updates are applied to the table and it is not synced.
|
||||
*/
|
||||
public class HoodieBackedTableMetadata implements HoodieTableMetadata {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadata.class);
|
||||
private static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024;
|
||||
private static final int BUFFER_SIZE = 10 * 1024 * 1024;
|
||||
|
||||
private final SerializableConfiguration hadoopConf;
|
||||
private final String datasetBasePath;
|
||||
private final String metadataBasePath;
|
||||
private final Option<HoodieMetadataMetrics> metrics;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
private boolean enabled;
|
||||
private final boolean validateLookups;
|
||||
private final boolean assumeDatePartitioning;
|
||||
// Directory used for Spillable Map when merging records
|
||||
private final String spillableMapDirectory;
|
||||
|
||||
// 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(conf, datasetBasePath, spillableMapDirectory, enabled, validateLookups, false, assumeDatePartitioning);
|
||||
}
|
||||
|
||||
public HoodieBackedTableMetadata(Configuration conf, String datasetBasePath, String spillableMapDirectory,
|
||||
boolean enabled, boolean validateLookups, boolean enableMetrics,
|
||||
boolean assumeDatePartitioning) {
|
||||
this.hadoopConf = new SerializableConfiguration(conf);
|
||||
this.datasetBasePath = datasetBasePath;
|
||||
this.metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(datasetBasePath);
|
||||
this.validateLookups = validateLookups;
|
||||
this.spillableMapDirectory = spillableMapDirectory;
|
||||
this.enabled = enabled;
|
||||
this.assumeDatePartitioning = assumeDatePartitioning;
|
||||
|
||||
if (enabled) {
|
||||
try {
|
||||
this.metaClient = new HoodieTableMetaClient(hadoopConf.get(), metadataBasePath);
|
||||
} catch (TableNotFoundException e) {
|
||||
LOG.error("Metadata table was not found at path " + metadataBasePath);
|
||||
this.enabled = false;
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to initialize metadata table at path " + metadataBasePath, e);
|
||||
this.enabled = false;
|
||||
}
|
||||
} else {
|
||||
LOG.info("Metadata table is disabled.");
|
||||
}
|
||||
|
||||
if (enableMetrics) {
|
||||
this.metrics = Option.of(new HoodieMetadataMetrics(Registry.getRegistry("HoodieMetadata")));
|
||||
} else {
|
||||
this.metrics = Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the list of partitions in the dataset.
|
||||
*
|
||||
* If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of
|
||||
* partitions is retrieved directly from the underlying {@code FileSystem}.
|
||||
*
|
||||
* On any errors retrieving the listing from the metadata, defaults to using the file system listings.
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
public List<String> getAllPartitionPaths()
|
||||
throws IOException {
|
||||
if (enabled) {
|
||||
try {
|
||||
return fetchAllPartitionPaths();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to retrieve list of partition from metadata", e);
|
||||
}
|
||||
}
|
||||
|
||||
FileSystem fs = FSUtils.getFs(datasetBasePath, hadoopConf.get());
|
||||
return FSUtils.getAllPartitionPaths(fs, datasetBasePath, assumeDatePartitioning);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the list of files in a partition.
|
||||
*
|
||||
* If the Metadata Table is enabled, the listing is retrieved from the stored metadata. Otherwise, the list of
|
||||
* partitions is retrieved directly from the underlying {@code FileSystem}.
|
||||
*
|
||||
* On any errors retrieving the listing from the metadata, defaults to using the file system listings.
|
||||
*
|
||||
* @param partitionPath The absolute path of the partition to list
|
||||
*/
|
||||
@Override
|
||||
public FileStatus[] getAllFilesInPartition(Path partitionPath)
|
||||
throws IOException {
|
||||
if (enabled) {
|
||||
try {
|
||||
return fetchAllFilesInPartition(partitionPath);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Failed to retrive files in partition " + partitionPath + " from metadata", e);
|
||||
}
|
||||
}
|
||||
|
||||
return FSUtils.getFs(partitionPath.toString(), hadoopConf.get()).listStatus(partitionPath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a list of all partitions.
|
||||
*/
|
||||
protected List<String> fetchAllPartitionPaths() throws IOException {
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord = getMergedRecordByKey(RECORDKEY_PARTITION_LIST);
|
||||
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_PARTITIONS_STR, timer.endTimer()));
|
||||
|
||||
List<String> partitions = Collections.emptyList();
|
||||
if (hoodieRecord.isPresent()) {
|
||||
if (!hoodieRecord.get().getData().getDeletions().isEmpty()) {
|
||||
throw new HoodieMetadataException("Metadata partition list record is inconsistent: "
|
||||
+ hoodieRecord.get().getData());
|
||||
}
|
||||
|
||||
partitions = hoodieRecord.get().getData().getFilenames();
|
||||
// Partition-less tables have a single empty partition
|
||||
if (partitions.contains(NON_PARTITIONED_NAME)) {
|
||||
partitions.remove(NON_PARTITIONED_NAME);
|
||||
partitions.add("");
|
||||
}
|
||||
}
|
||||
|
||||
if (validateLookups) {
|
||||
// Validate the Metadata Table data by listing the partitions from the file system
|
||||
timer.startTimer();
|
||||
List<String> actualPartitions = FSUtils.getAllPartitionPaths(metaClient.getFs(), datasetBasePath, false);
|
||||
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_PARTITIONS_STR, timer.endTimer()));
|
||||
|
||||
Collections.sort(actualPartitions);
|
||||
Collections.sort(partitions);
|
||||
if (!actualPartitions.equals(partitions)) {
|
||||
LOG.error("Validation of metadata partition list failed. Lists do not match.");
|
||||
LOG.error("Partitions from metadata: " + Arrays.toString(partitions.toArray()));
|
||||
LOG.error("Partitions from file system: " + Arrays.toString(actualPartitions.toArray()));
|
||||
|
||||
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_ERRORS_STR, 0));
|
||||
}
|
||||
|
||||
// Return the direct listing as it should be correct
|
||||
partitions = actualPartitions;
|
||||
}
|
||||
|
||||
LOG.info("Listed partitions from metadata: #partitions=" + partitions.size());
|
||||
return partitions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return all the files from the partition.
|
||||
*
|
||||
* @param partitionPath The absolute path of the partition
|
||||
*/
|
||||
FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException {
|
||||
String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetBasePath), partitionPath);
|
||||
if (partitionName.isEmpty()) {
|
||||
partitionName = NON_PARTITIONED_NAME;
|
||||
}
|
||||
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
Option<HoodieRecord<HoodieMetadataPayload>> hoodieRecord = getMergedRecordByKey(partitionName);
|
||||
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.LOOKUP_FILES_STR, timer.endTimer()));
|
||||
|
||||
FileStatus[] statuses = {};
|
||||
if (hoodieRecord.isPresent()) {
|
||||
if (!hoodieRecord.get().getData().getDeletions().isEmpty()) {
|
||||
throw new HoodieMetadataException("Metadata record for partition " + partitionName + " is inconsistent: "
|
||||
+ hoodieRecord.get().getData());
|
||||
}
|
||||
statuses = hoodieRecord.get().getData().getFileStatuses(partitionPath);
|
||||
}
|
||||
|
||||
if (validateLookups) {
|
||||
// Validate the Metadata Table data by listing the partitions from the file system
|
||||
timer.startTimer();
|
||||
|
||||
// Ignore partition metadata file
|
||||
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()));
|
||||
|
||||
List<String> directFilenames = Arrays.stream(directStatuses)
|
||||
.map(s -> s.getPath().getName()).sorted()
|
||||
.collect(Collectors.toList());
|
||||
|
||||
List<String> metadataFilenames = Arrays.stream(statuses)
|
||||
.map(s -> s.getPath().getName()).sorted()
|
||||
.collect(Collectors.toList());
|
||||
|
||||
if (!metadataFilenames.equals(directFilenames)) {
|
||||
LOG.error("Validation of metadata file listing for partition " + partitionName + " failed.");
|
||||
LOG.error("File list from metadata: " + Arrays.toString(metadataFilenames.toArray()));
|
||||
LOG.error("File list from direct listing: " + Arrays.toString(directFilenames.toArray()));
|
||||
|
||||
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_ERRORS_STR, 0));
|
||||
}
|
||||
|
||||
// Return the direct listing as it should be correct
|
||||
statuses = directStatuses;
|
||||
}
|
||||
|
||||
LOG.info("Listed file in partition from metadata: partition=" + partitionName + ", #files=" + statuses.length);
|
||||
return statuses;
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve the merged {@code HoodieRecord} mapped to the given key.
|
||||
*
|
||||
* @param key The key of the record
|
||||
*/
|
||||
private Option<HoodieRecord<HoodieMetadataPayload>> getMergedRecordByKey(String key) throws IOException {
|
||||
openBaseAndLogFiles();
|
||||
|
||||
// Retrieve record from base file
|
||||
HoodieRecord<HoodieMetadataPayload> hoodieRecord = null;
|
||||
if (basefileReader != null) {
|
||||
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()));
|
||||
}
|
||||
}
|
||||
|
||||
// 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();
|
||||
}
|
||||
}
|
||||
|
||||
return Option.ofNullable(hoodieRecord);
|
||||
}
|
||||
|
||||
/**
|
||||
* Open readers to the base and log files.
|
||||
*/
|
||||
private synchronized void openBaseAndLogFiles() throws IOException {
|
||||
if (logRecordScanner != null) {
|
||||
// Already opened
|
||||
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);
|
||||
|
||||
// If the base file is present then create a reader
|
||||
Option<HoodieBaseFile> basefile = latestSlices.get(0).getBaseFile();
|
||||
if (basefile.isPresent()) {
|
||||
String basefilePath = basefile.get().getPath();
|
||||
basefileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath));
|
||||
LOG.info("Opened metadata base file from " + basefilePath + " at instant " + basefile.get().getCommitTime());
|
||||
}
|
||||
|
||||
// Open the log record scanner using the log files from the latest file slice
|
||||
List<String> logFilePaths = latestSlices.get(0).getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
|
||||
.map(o -> o.getPath().toString())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
Option<HoodieInstant> lastInstant = timeline.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,
|
||||
logFilePaths, schema, latestMetaInstantTimestamp, MAX_MEMORY_SIZE_IN_BYTES, BUFFER_SIZE,
|
||||
spillableMapDirectory, null);
|
||||
|
||||
LOG.info("Opened metadata log files from " + logFilePaths + " at instant " + latestInstantTime
|
||||
+ "(dataset instant=" + latestInstantTime + ", metadata instant=" + latestMetaInstantTimestamp + ")");
|
||||
|
||||
metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR, timer.endTimer()));
|
||||
}
|
||||
|
||||
public void closeReaders() {
|
||||
if (basefileReader != null) {
|
||||
basefileReader.close();
|
||||
basefileReader = null;
|
||||
}
|
||||
logRecordScanner = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return {@code True} if all Instants from the dataset have been synced with the Metadata Table.
|
||||
*/
|
||||
@Override
|
||||
public boolean isInSync() {
|
||||
return enabled && findInstantsToSync().isEmpty();
|
||||
}
|
||||
|
||||
private List<HoodieInstant> findInstantsToSync() {
|
||||
HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath);
|
||||
return findInstantsToSync(datasetMetaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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();
|
||||
|
||||
// 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();
|
||||
HoodieDefaultTimeline candidateTimeline = datasetMetaClient.getActiveTimeline().findInstantsAfter(latestMetadataInstantTime, Integer.MAX_VALUE);
|
||||
Option<HoodieInstant> earliestIncompleteInstant = candidateTimeline.filterInflightsAndRequested().firstInstant();
|
||||
|
||||
if (earliestIncompleteInstant.isPresent()) {
|
||||
return candidateTimeline.filterCompletedInstants()
|
||||
.findInstantsBefore(earliestIncompleteInstant.get().getTimestamp())
|
||||
.getInstants().collect(Collectors.toList());
|
||||
} else {
|
||||
return candidateTimeline.filterCompletedInstants()
|
||||
.getInstants().collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the timestamp of the latest compaction instant.
|
||||
*/
|
||||
@Override
|
||||
public Option<String> getSyncedInstantTime() {
|
||||
if (!enabled) {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline();
|
||||
return timeline.getDeltaCommitTimeline().filterCompletedInstants()
|
||||
.lastInstant().map(HoodieInstant::getTimestamp);
|
||||
}
|
||||
|
||||
public boolean enabled() {
|
||||
return enabled;
|
||||
}
|
||||
|
||||
public SerializableConfiguration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
|
||||
public String getDatasetBasePath() {
|
||||
return datasetBasePath;
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient getMetaClient() {
|
||||
return metaClient;
|
||||
}
|
||||
|
||||
public Map<String, String> stats() {
|
||||
return metrics.map(m -> m.getStats(true, metaClient, this)).orElse(new HashMap<>());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,75 @@
|
||||
/*
|
||||
* 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 java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
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.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* A {@code HoodieMergedLogRecordScanner} implementation which only merged records matching providing keys. This is
|
||||
* useful in limiting memory usage when only a small subset of updates records are to be read.
|
||||
*/
|
||||
public class HoodieMetadataMergedLogRecordScanner extends HoodieMergedLogRecordScanner {
|
||||
// Set of all record keys that are to be read in memory
|
||||
private Set<String> mergeKeyFilter;
|
||||
|
||||
public HoodieMetadataMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
|
||||
Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, int bufferSize,
|
||||
String spillableMapBasePath, Set<String> mergeKeyFilter) {
|
||||
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize,
|
||||
spillableMapBasePath, false);
|
||||
this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet();
|
||||
|
||||
performScan();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) throws IOException {
|
||||
if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(hoodieRecord.getRecordKey())) {
|
||||
super.processNextRecord(hoodieRecord);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processNextDeletedKey(HoodieKey hoodieKey) {
|
||||
if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(hoodieKey.getRecordKey())) {
|
||||
super.processNextDeletedKey(hoodieKey);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve a record given its key.
|
||||
*
|
||||
* @param key Key of the record to retrieve
|
||||
* @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));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,147 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.metrics.Registry;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class HoodieMetadataMetrics implements Serializable {
|
||||
|
||||
// Metric names
|
||||
public static final String LOOKUP_PARTITIONS_STR = "lookup_partitions";
|
||||
public static final String LOOKUP_FILES_STR = "lookup_files";
|
||||
public static final String VALIDATE_PARTITIONS_STR = "validate_partitions";
|
||||
public static final String VALIDATE_FILES_STR = "validate_files";
|
||||
public static final String VALIDATE_ERRORS_STR = "validate_errors";
|
||||
public static final String SCAN_STR = "scan";
|
||||
public static final String BASEFILE_READ_STR = "basefile_read";
|
||||
public static final String INITIALIZE_STR = "initialize";
|
||||
public static final String SYNC_STR = "sync";
|
||||
|
||||
// Stats names
|
||||
public static final String STAT_TOTAL_BASE_FILE_SIZE = "totalBaseFileSizeInBytes";
|
||||
public static final String STAT_TOTAL_LOG_FILE_SIZE = "totalLogFileSizeInBytes";
|
||||
public static final String STAT_COUNT_BASE_FILES = "baseFileCount";
|
||||
public static final String STAT_COUNT_LOG_FILES = "logFileCount";
|
||||
public static final String STAT_COUNT_PARTITION = "partitionCount";
|
||||
public static final String STAT_IN_SYNC = "isInSync";
|
||||
public static final String STAT_LAST_COMPACTION_TIMESTAMP = "lastCompactionTimestamp";
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieMetadataMetrics.class);
|
||||
|
||||
private final Registry metricsRegistry;
|
||||
|
||||
public HoodieMetadataMetrics(Registry metricsRegistry) {
|
||||
this.metricsRegistry = metricsRegistry;
|
||||
}
|
||||
|
||||
public Map<String, String> getStats(boolean detailed, HoodieTableMetaClient metaClient, HoodieTableMetadata metadata) {
|
||||
try {
|
||||
metaClient.reloadActiveTimeline();
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline());
|
||||
return getStats(fsView, detailed, metadata);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Unable to get metadata stats.", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, String> getStats(HoodieTableFileSystemView fsView, boolean detailed, HoodieTableMetadata tableMetadata) throws IOException {
|
||||
Map<String, String> stats = new HashMap<>();
|
||||
|
||||
// Total size of the metadata and count of base/log files
|
||||
long totalBaseFileSizeInBytes = 0;
|
||||
long totalLogFileSizeInBytes = 0;
|
||||
int baseFileCount = 0;
|
||||
int logFileCount = 0;
|
||||
List<FileSlice> latestSlices = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath()).collect(Collectors.toList());
|
||||
|
||||
for (FileSlice slice : latestSlices) {
|
||||
if (slice.getBaseFile().isPresent()) {
|
||||
totalBaseFileSizeInBytes += slice.getBaseFile().get().getFileStatus().getLen();
|
||||
++baseFileCount;
|
||||
}
|
||||
Iterator<HoodieLogFile> it = slice.getLogFiles().iterator();
|
||||
while (it.hasNext()) {
|
||||
totalLogFileSizeInBytes += it.next().getFileStatus().getLen();
|
||||
++logFileCount;
|
||||
}
|
||||
}
|
||||
|
||||
stats.put(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE, String.valueOf(totalBaseFileSizeInBytes));
|
||||
stats.put(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE, String.valueOf(totalLogFileSizeInBytes));
|
||||
stats.put(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES, String.valueOf(baseFileCount));
|
||||
stats.put(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES, String.valueOf(logFileCount));
|
||||
|
||||
if (detailed) {
|
||||
stats.put(HoodieMetadataMetrics.STAT_COUNT_PARTITION, String.valueOf(tableMetadata.getAllPartitionPaths().size()));
|
||||
stats.put(HoodieMetadataMetrics.STAT_IN_SYNC, String.valueOf(tableMetadata.isInSync()));
|
||||
}
|
||||
|
||||
return stats;
|
||||
}
|
||||
|
||||
protected void updateMetrics(String action, long durationInMs) {
|
||||
if (metricsRegistry == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
// Update sum of duration and total for count
|
||||
String countKey = action + ".count";
|
||||
String durationKey = action + ".totalDuration";
|
||||
metricsRegistry.add(countKey, 1);
|
||||
metricsRegistry.add(durationKey, durationInMs);
|
||||
|
||||
LOG.info(String.format("Updating metadata metrics (%s=%dms, %s=1)", durationKey, durationInMs, countKey));
|
||||
}
|
||||
|
||||
public void updateMetrics(long totalBaseFileSizeInBytes, long totalLogFileSizeInBytes, int baseFileCount,
|
||||
int logFileCount) {
|
||||
if (metricsRegistry == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
// Update sizes and count for metadata table's data files
|
||||
metricsRegistry.add("basefile.size", totalBaseFileSizeInBytes);
|
||||
metricsRegistry.add("logfile.size", totalLogFileSizeInBytes);
|
||||
metricsRegistry.add("basefile.count", baseFileCount);
|
||||
metricsRegistry.add("logfile.count", logFileCount);
|
||||
|
||||
LOG.info(String.format("Updating metadata size metrics (basefile.size=%d, logfile.size=%d, basefile.count=%d, "
|
||||
+ "logfile.count=%d)", totalBaseFileSizeInBytes, totalLogFileSizeInBytes, baseFileCount, logFileCount));
|
||||
}
|
||||
|
||||
public Registry registry() {
|
||||
return metricsRegistry;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,233 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.avro.model.HoodieMetadataFileInfo;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataRecord;
|
||||
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.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieMetadataException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST;
|
||||
|
||||
/**
|
||||
* This is a payload which saves information about a single entry in the Metadata Table.
|
||||
*
|
||||
* The type of the entry is determined by the "type" saved within the record. The following types of entries are saved:
|
||||
*
|
||||
* 1. List of partitions: There is a single such record
|
||||
* key="__all_partitions__"
|
||||
*
|
||||
* 2. List of files in a Partition: There is one such record for each partition
|
||||
* key=Partition name
|
||||
*
|
||||
* During compaction on the table, the deletions are merged with additions and hence pruned.
|
||||
*
|
||||
* Metadata Table records are saved with the schema defined in HoodieMetadata.avsc. This class encapsulates the
|
||||
* HoodieMetadataRecord for ease of operations.
|
||||
*/
|
||||
public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadataPayload> {
|
||||
// Type of the record
|
||||
// This can be an enum in the schema but Avro 1.8 has a bug - https://issues.apache.org/jira/browse/AVRO-1810
|
||||
private static final int PARTITION_LIST = 1;
|
||||
private static final int FILE_LIST = 2;
|
||||
|
||||
private String key = null;
|
||||
private int type = 0;
|
||||
private Map<String, HoodieMetadataFileInfo> filesystemMetadata = null;
|
||||
|
||||
public HoodieMetadataPayload(Option<GenericRecord> record) {
|
||||
if (record.isPresent()) {
|
||||
// This can be simplified using SpecificData.deepcopy once this bug is fixed
|
||||
// https://issues.apache.org/jira/browse/AVRO-1811
|
||||
key = record.get().get("key").toString();
|
||||
type = (int) record.get().get("type");
|
||||
if (record.get().get("filesystemMetadata") != null) {
|
||||
filesystemMetadata = (Map<String, HoodieMetadataFileInfo>) record.get().get("filesystemMetadata");
|
||||
filesystemMetadata.keySet().forEach(k -> {
|
||||
GenericRecord v = filesystemMetadata.get(k);
|
||||
filesystemMetadata.put(k.toString(), new HoodieMetadataFileInfo((Long)v.get("size"), (Boolean)v.get("isDeleted")));
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private HoodieMetadataPayload(String key, int type, Map<String, HoodieMetadataFileInfo> filesystemMetadata) {
|
||||
this.key = key;
|
||||
this.type = type;
|
||||
this.filesystemMetadata = filesystemMetadata;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create and return a {@code HoodieMetadataPayload} to save list of partitions.
|
||||
*
|
||||
* @param partitions The list of partitions
|
||||
*/
|
||||
public static HoodieRecord<HoodieMetadataPayload> createPartitionListRecord(List<String> partitions) {
|
||||
Map<String, HoodieMetadataFileInfo> fileInfo = new HashMap<>();
|
||||
partitions.forEach(partition -> fileInfo.put(partition, new HoodieMetadataFileInfo(0L, false)));
|
||||
|
||||
HoodieKey key = new HoodieKey(RECORDKEY_PARTITION_LIST, MetadataPartitionType.FILES.partitionPath());
|
||||
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), PARTITION_LIST, fileInfo);
|
||||
return new HoodieRecord<>(key, payload);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create and return a {@code HoodieMetadataPayload} to save list of files within a partition.
|
||||
*
|
||||
* @param partition The name of the partition
|
||||
* @param filesAdded Mapping of files to their sizes for files which have been added to this partition
|
||||
* @param filesDeleted List of files which have been deleted from this partition
|
||||
*/
|
||||
public static HoodieRecord<HoodieMetadataPayload> createPartitionFilesRecord(String partition,
|
||||
Option<Map<String, Long>> filesAdded, Option<List<String>> filesDeleted) {
|
||||
Map<String, HoodieMetadataFileInfo> fileInfo = new HashMap<>();
|
||||
filesAdded.ifPresent(
|
||||
m -> m.forEach((filename, size) -> fileInfo.put(filename, new HoodieMetadataFileInfo(size, false))));
|
||||
filesDeleted.ifPresent(
|
||||
m -> m.forEach(filename -> fileInfo.put(filename, new HoodieMetadataFileInfo(0L, true))));
|
||||
|
||||
HoodieKey key = new HoodieKey(partition, MetadataPartitionType.FILES.partitionPath());
|
||||
HoodieMetadataPayload payload = new HoodieMetadataPayload(key.getRecordKey(), FILE_LIST, fileInfo);
|
||||
return new HoodieRecord<>(key, payload);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieMetadataPayload preCombine(HoodieMetadataPayload previousRecord) {
|
||||
ValidationUtils.checkArgument(previousRecord.type == type,
|
||||
"Cannot combine " + previousRecord.type + " with " + type);
|
||||
|
||||
Map<String, HoodieMetadataFileInfo> combinedFileInfo = null;
|
||||
|
||||
switch (type) {
|
||||
case PARTITION_LIST:
|
||||
case FILE_LIST:
|
||||
combinedFileInfo = combineFilesystemMetadata(previousRecord);
|
||||
break;
|
||||
default:
|
||||
throw new HoodieMetadataException("Unknown type of HoodieMetadataPayload: " + type);
|
||||
}
|
||||
|
||||
return new HoodieMetadataPayload(key, type, combinedFileInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRecord, Schema schema) throws IOException {
|
||||
HoodieMetadataPayload anotherPayload = new HoodieMetadataPayload(Option.of((GenericRecord)oldRecord));
|
||||
HoodieRecordPayload combinedPayload = preCombine(anotherPayload);
|
||||
return combinedPayload.getInsertValue(schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
||||
if (key == null) {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
HoodieMetadataRecord record = new HoodieMetadataRecord(key, type, filesystemMetadata);
|
||||
return Option.of(record);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the list of filenames added as part of this record.
|
||||
*/
|
||||
public List<String> getFilenames() {
|
||||
return filterFileInfoEntries(false).map(e -> e.getKey()).sorted().collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the list of filenames deleted as part of this record.
|
||||
*/
|
||||
public List<String> getDeletions() {
|
||||
return filterFileInfoEntries(true).map(e -> e.getKey()).sorted().collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the files added as part of this record.
|
||||
*/
|
||||
public FileStatus[] getFileStatuses(Path partitionPath) {
|
||||
return filterFileInfoEntries(false)
|
||||
.map(e -> new FileStatus(e.getValue().getSize(), false, 0, 0, 0, 0, null, null, null,
|
||||
new Path(partitionPath, e.getKey())))
|
||||
.toArray(FileStatus[]::new);
|
||||
}
|
||||
|
||||
private Stream<Map.Entry<String, HoodieMetadataFileInfo>> filterFileInfoEntries(boolean isDeleted) {
|
||||
if (filesystemMetadata == null) {
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
return filesystemMetadata.entrySet().stream().filter(e -> e.getValue().getIsDeleted() == isDeleted);
|
||||
}
|
||||
|
||||
private Map<String, HoodieMetadataFileInfo> combineFilesystemMetadata(HoodieMetadataPayload previousRecord) {
|
||||
Map<String, HoodieMetadataFileInfo> combinedFileInfo = new HashMap<>();
|
||||
if (previousRecord.filesystemMetadata != null) {
|
||||
combinedFileInfo.putAll(previousRecord.filesystemMetadata);
|
||||
}
|
||||
|
||||
if (filesystemMetadata != null) {
|
||||
filesystemMetadata.forEach((filename, fileInfo) -> {
|
||||
// If the filename wasnt present then we carry it forward
|
||||
if (!combinedFileInfo.containsKey(filename)) {
|
||||
combinedFileInfo.put(filename, fileInfo);
|
||||
} else {
|
||||
if (fileInfo.getIsDeleted()) {
|
||||
// file deletion
|
||||
combinedFileInfo.remove(filename);
|
||||
} else {
|
||||
// file appends.
|
||||
combinedFileInfo.merge(filename, fileInfo, (oldFileInfo, newFileInfo) -> {
|
||||
return new HoodieMetadataFileInfo(oldFileInfo.getSize() + newFileInfo.getSize(), false);
|
||||
});
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
return combinedFileInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieMetadataPayload {");
|
||||
sb.append("key=").append(key).append(", ");
|
||||
sb.append("type=").append(type).append(", ");
|
||||
sb.append("creations=").append(Arrays.toString(getFilenames().toArray())).append(", ");
|
||||
sb.append("deletions=").append(Arrays.toString(getDeletions().toArray())).append(", ");
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,93 @@
|
||||
/*
|
||||
* 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 org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Interface that supports querying various pieces of metadata about a hudi table.
|
||||
*/
|
||||
public interface HoodieTableMetadata extends Serializable {
|
||||
|
||||
// Table name suffix
|
||||
String METADATA_TABLE_NAME_SUFFIX = "_metadata";
|
||||
/**
|
||||
* Timestamp for a commit when the base dataset had not had any commits yet. this is < than even
|
||||
* {@link org.apache.hudi.common.table.timeline.HoodieTimeline#INIT_INSTANT_TS}, such that the metadata table
|
||||
* can be prepped even before bootstrap is done.
|
||||
*/
|
||||
String SOLO_COMMIT_TIMESTAMP = "0000000000000";
|
||||
// Key for the record which saves list of all partitions
|
||||
String RECORDKEY_PARTITION_LIST = "__all_partitions__";
|
||||
// The partition name used for non-partitioned tables
|
||||
String NON_PARTITIONED_NAME = ".";
|
||||
|
||||
// Base path of the Metadata Table relative to the dataset (.hoodie/metadata)
|
||||
static final String METADATA_TABLE_REL_PATH = HoodieTableMetaClient.METAFOLDER_NAME + Path.SEPARATOR + "metadata";
|
||||
|
||||
/**
|
||||
* Return the base path of the Metadata Table.
|
||||
*
|
||||
* @param tableBasePath The base path of the dataset
|
||||
*/
|
||||
static String getMetadataTableBasePath(String tableBasePath) {
|
||||
return tableBasePath + Path.SEPARATOR + METADATA_TABLE_REL_PATH;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns {@code True} if the given path contains a metadata table.
|
||||
*
|
||||
* @param basePath The base path to check
|
||||
*/
|
||||
static boolean isMetadataTable(String basePath) {
|
||||
return basePath.endsWith(METADATA_TABLE_REL_PATH);
|
||||
}
|
||||
|
||||
static HoodieTableMetadata create(Configuration conf, String datasetBasePath, String spillableMapPath, boolean useFileListingFromMetadata,
|
||||
boolean verifyListings, boolean enableMetrics, boolean shouldAssumeDatePartitioning) {
|
||||
return new HoodieBackedTableMetadata(conf, datasetBasePath, spillableMapPath, useFileListingFromMetadata, verifyListings,
|
||||
enableMetrics, shouldAssumeDatePartitioning);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch all the files at the given partition path, per the latest snapshot of the metadata.
|
||||
*/
|
||||
FileStatus[] getAllFilesInPartition(Path partitionPath) throws IOException;
|
||||
|
||||
/**
|
||||
* Fetch list of all partition paths, per the latest snapshot of the metadata.
|
||||
*/
|
||||
List<String> getAllPartitionPaths() throws IOException;
|
||||
|
||||
/**
|
||||
* Get the instant time to which the metadata is synced w.r.t data timeline.
|
||||
*/
|
||||
Option<String> getSyncedInstantTime();
|
||||
|
||||
boolean isInSync();
|
||||
}
|
||||
@@ -0,0 +1,33 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
public enum MetadataPartitionType {
|
||||
FILES("files");
|
||||
|
||||
private final String partitionPath;
|
||||
|
||||
MetadataPartitionType(String partitionPath) {
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
public String partitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user