1
0

[HUDI-1325] [RFC-15] Merge updates of unsynced instants to metadata table (apache#2342)

[RFC-15] Fix partition key in metadata table when bootstrapping from file system (apache#2387)

Co-authored-by: Ryan Pifer <ryanpife@amazon.com>
This commit is contained in:
Ryan Pifer
2020-12-31 08:57:13 -08:00
committed by vinoth chandar
parent 2bd4a68731
commit 4b94529aaf
11 changed files with 909 additions and 468 deletions

View File

@@ -0,0 +1,293 @@
/*
* 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.avro.Schema;
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.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.HoodieTimer;
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 org.apache.hudi.exception.HoodieMetadataException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
public abstract class BaseTableMetadata implements HoodieTableMetadata {
private static final Logger LOG = LogManager.getLogger(BaseTableMetadata.class);
static final long MAX_MEMORY_SIZE_IN_BYTES = 1024 * 1024 * 1024;
static final int BUFFER_SIZE = 10 * 1024 * 1024;
protected final SerializableConfiguration hadoopConf;
protected final String datasetBasePath;
protected boolean enabled;
protected final Option<HoodieMetadataMetrics> metrics;
private final boolean validateLookups;
private final boolean assumeDatePartitioning;
// Directory used for Spillable Map when merging records
protected final String spillableMapDirectory;
private transient HoodieMetadataMergedInstantRecordScanner timelineRecordScanner;
protected BaseTableMetadata(Configuration hadoopConf, String datasetBasePath, String spillableMapDirectory,
boolean enabled, boolean validateLookups, boolean enableMetrics,
boolean assumeDatePartitioning) {
this.hadoopConf = new SerializableConfiguration(hadoopConf);
this.datasetBasePath = datasetBasePath;
this.spillableMapDirectory = spillableMapDirectory;
this.enabled = enabled;
this.validateLookups = validateLookups;
this.assumeDatePartitioning = assumeDatePartitioning;
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);
}
}
return new FileSystemBackedTableMetadata(hadoopConf, datasetBasePath, assumeDatePartitioning).getAllPartitionPaths();
}
/**
* 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 retrieve 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();
FileSystemBackedTableMetadata fileSystemBackedTableMetadata = new FileSystemBackedTableMetadata(hadoopConf, datasetBasePath, assumeDatePartitioning);
List<String> actualPartitions = fileSystemBackedTableMetadata.getAllPartitionPaths();
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
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath);
FileStatus[] directStatuses = metaClient.getFs().listStatus(partitionPath,
p -> !p.getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE));
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.VALIDATE_FILES_STR, timer.endTimer()));
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 {
Option<HoodieRecord<HoodieMetadataPayload>> mergedRecord;
openTimelineScanner();
Option<HoodieRecord<HoodieMetadataPayload>> metadataHoodieRecord = getRecordByKeyFromMetadata(key);
// Retrieve record from unsynced timeline instants
Option<HoodieRecord<HoodieMetadataPayload>> timelineHoodieRecord = timelineRecordScanner.getRecordByKey(key);
if (timelineHoodieRecord.isPresent()) {
if (metadataHoodieRecord.isPresent()) {
HoodieRecordPayload mergedPayload = timelineHoodieRecord.get().getData().preCombine(metadataHoodieRecord.get().getData());
mergedRecord = Option.of(new HoodieRecord(metadataHoodieRecord.get().getKey(), mergedPayload));
} else {
mergedRecord = timelineHoodieRecord;
}
} else {
mergedRecord = metadataHoodieRecord;
}
return mergedRecord;
}
protected abstract Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKeyFromMetadata(String key) throws IOException;
private void openTimelineScanner() throws IOException {
if (timelineRecordScanner != null) {
// Already opened
return;
}
HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath);
List<HoodieInstant> unsyncedInstants = findInstantsToSync(datasetMetaClient);
Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema());
timelineRecordScanner =
new HoodieMetadataMergedInstantRecordScanner(datasetMetaClient, unsyncedInstants, getSyncedInstantTime(), schema, MAX_MEMORY_SIZE_IN_BYTES, spillableMapDirectory, null);
}
protected List<HoodieInstant> findInstantsToSync() {
HoodieTableMetaClient datasetMetaClient = new HoodieTableMetaClient(hadoopConf.get(), datasetBasePath);
return findInstantsToSync(datasetMetaClient);
}
protected abstract List<HoodieInstant> findInstantsToSync(HoodieTableMetaClient datasetMetaClient);
public boolean isInSync() {
return enabled && findInstantsToSync().isEmpty();
}
protected void closeReaders() {
timelineRecordScanner = null;
}
}

View File

@@ -18,28 +18,12 @@
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.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;
@@ -52,37 +36,36 @@ 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.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* 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 {
public class HoodieBackedTableMetadata extends BaseTableMetadata {
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;
@@ -95,14 +78,8 @@ public class HoodieBackedTableMetadata implements HoodieTableMetadata {
public HoodieBackedTableMetadata(Configuration conf, String datasetBasePath, String spillableMapDirectory,
boolean enabled, boolean validateLookups, boolean enableMetrics,
boolean assumeDatePartitioning) {
this.hadoopConf = new SerializableConfiguration(conf);
this.datasetBasePath = datasetBasePath;
super(conf, datasetBasePath, spillableMapDirectory, enabled, validateLookups, enableMetrics, assumeDatePartitioning);
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);
@@ -116,171 +93,10 @@ public class HoodieBackedTableMetadata implements HoodieTableMetadata {
} 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);
}
}
return new FileSystemBackedTableMetadata(hadoopConf, datasetBasePath, assumeDatePartitioning).getAllPartitionPaths();
}
/**
* 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();
FileSystemBackedTableMetadata fileSystemBackedTableMetadata = new FileSystemBackedTableMetadata(hadoopConf, datasetBasePath, assumeDatePartitioning);
List<String> actualPartitions = fileSystemBackedTableMetadata.getAllPartitionPaths();
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 {
protected Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKeyFromMetadata(String key) throws IOException {
openBaseAndLogFiles();
// Retrieve record from base file
@@ -313,7 +129,7 @@ public class HoodieBackedTableMetadata implements HoodieTableMetadata {
/**
* Open readers to the base and log files.
*/
private synchronized void openBaseAndLogFiles() throws IOException {
protected synchronized void openBaseAndLogFiles() throws IOException {
if (logRecordScanner != null) {
// Already opened
return;
@@ -370,19 +186,6 @@ public class HoodieBackedTableMetadata implements HoodieTableMetadata {
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

View File

@@ -0,0 +1,115 @@
/*
* 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.avro.Schema;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.exception.HoodieException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Set;
/**
* Provides functionality to convert timeline instants to table metadata records and then merge by key. Specify
* a filter to limit keys that are merged and stored in memory.
*/
public class HoodieMetadataMergedInstantRecordScanner {
private static final Logger LOG = LogManager.getLogger(HoodieMetadataMergedInstantRecordScanner.class);
HoodieTableMetaClient metaClient;
private List<HoodieInstant> instants;
private Option<String> lastSyncTs;
private Set<String> mergeKeyFilter;
protected final ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records;
public HoodieMetadataMergedInstantRecordScanner(HoodieTableMetaClient metaClient, List<HoodieInstant> instants,
Option<String> lastSyncTs, Schema readerSchema, Long maxMemorySizeInBytes,
String spillableMapBasePath, Set<String> mergeKeyFilter) throws IOException {
this.metaClient = metaClient;
this.instants = instants;
this.lastSyncTs = lastSyncTs;
this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet();
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(),
new HoodieRecordSizeEstimator(readerSchema));
scan();
}
/**
* Converts instants in scanner to metadata table records and processes each record.
*
* @param
* @throws IOException
*/
private void scan() {
for (HoodieInstant instant : instants) {
try {
Option<List<HoodieRecord>> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(metaClient, instant, lastSyncTs);
if (records.isPresent()) {
records.get().forEach(record -> processNextRecord(record));
}
} catch (Exception e) {
LOG.error(String.format("Got exception when processing timeline instant %s", instant.getTimestamp()), e);
throw new HoodieException(String.format("Got exception when processing timeline instant %s", instant.getTimestamp()), e);
}
}
}
/**
* Process metadata table record by merging with existing record if it is a part of the key filter.
*
* @param hoodieRecord
*/
private void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) {
String key = hoodieRecord.getRecordKey();
if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(key)) {
if (records.containsKey(key)) {
// Merge and store the merged record
HoodieRecordPayload combinedValue = hoodieRecord.getData().preCombine(records.get(key).getData());
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue));
} else {
// Put the record as is
records.put(key, hoodieRecord);
}
}
}
/**
* Retrieve merged hoodie record for given key.
*
* @param 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));
}
}

View File

@@ -0,0 +1,311 @@
/*
* 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.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME;
/**
* A utility to convert timeline information to metadata table records.
*/
public class HoodieTableMetadataUtil {
private static final Logger LOG = LogManager.getLogger(HoodieTableMetadataUtil.class);
/**
* Converts a timeline instant to metadata table records.
*
* @param datasetMetaClient The meta client associated with the timeline instant
* @param instant to fetch and convert to metadata table records
* @return a list of metadata table records
* @throws IOException
*/
public static Option<List<HoodieRecord>> convertInstantToMetaRecords(HoodieTableMetaClient datasetMetaClient, HoodieInstant instant, Option<String> lastSyncTs) throws IOException {
HoodieTimeline timeline = datasetMetaClient.getActiveTimeline();
Option<List<HoodieRecord>> records = Option.empty();
ValidationUtils.checkArgument(instant.isCompleted(), "Only completed instants can be synced.");
switch (instant.getAction()) {
case HoodieTimeline.CLEAN_ACTION:
HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(datasetMetaClient, instant);
records = Option.of(convertMetadataToRecords(cleanMetadata, instant.getTimestamp()));
break;
case HoodieTimeline.DELTA_COMMIT_ACTION:
case HoodieTimeline.COMMIT_ACTION:
case HoodieTimeline.COMPACTION_ACTION:
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class);
records = Option.of(convertMetadataToRecords(commitMetadata, instant.getTimestamp()));
break;
case HoodieTimeline.ROLLBACK_ACTION:
HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata(
timeline.getInstantDetails(instant).get());
records = Option.of(convertMetadataToRecords(rollbackMetadata, instant.getTimestamp(), lastSyncTs));
break;
case HoodieTimeline.RESTORE_ACTION:
HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata(
timeline.getInstantDetails(instant).get());
records = Option.of(convertMetadataToRecords(restoreMetadata, instant.getTimestamp(), lastSyncTs));
break;
case HoodieTimeline.SAVEPOINT_ACTION:
// Nothing to be done here
break;
default:
throw new HoodieException("Unknown type of action " + instant.getAction());
}
return records;
}
/**
* Finds all new files/partitions created as part of commit and creates metadata table records for them.
*
* @param commitMetadata
* @param instantTime
* @return a list of metadata table records
*/
public static List<HoodieRecord> convertMetadataToRecords(HoodieCommitMetadata commitMetadata, String instantTime) {
List<HoodieRecord> records = new LinkedList<>();
List<String> allPartitions = new LinkedList<>();
commitMetadata.getPartitionToWriteStats().forEach((partitionStatName, writeStats) -> {
final String partition = partitionStatName.equals("") ? NON_PARTITIONED_NAME : partitionStatName;
allPartitions.add(partition);
Map<String, Long> newFiles = new HashMap<>(writeStats.size());
writeStats.forEach(hoodieWriteStat -> {
String pathWithPartition = hoodieWriteStat.getPath();
if (pathWithPartition == null) {
// Empty partition
LOG.warn("Unable to find path in write stat to update metadata table " + hoodieWriteStat);
return;
}
int offset = partition.equals(NON_PARTITIONED_NAME) ? 0 : partition.length() + 1;
String filename = pathWithPartition.substring(offset);
ValidationUtils.checkState(!newFiles.containsKey(filename), "Duplicate files in HoodieCommitMetadata");
newFiles.put(filename, hoodieWriteStat.getTotalWriteBytes());
});
// New files added to a partition
HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(
partition, Option.of(newFiles), Option.empty());
records.add(record);
});
// New partitions created
HoodieRecord record = HoodieMetadataPayload.createPartitionListRecord(new ArrayList<>(allPartitions));
records.add(record);
LOG.info("Updating at " + instantTime + " from Commit/" + commitMetadata.getOperationType()
+ ". #partitions_updated=" + records.size());
return records;
}
/**
* Finds all files that will be deleted as part of a planned clean and creates metadata table records for them.
*
* @param cleanerPlan from timeline to convert
* @param instantTime
* @return a list of metadata table records
*/
public static List<HoodieRecord> convertMetadataToRecords(HoodieCleanerPlan cleanerPlan, String instantTime) {
List<HoodieRecord> records = new LinkedList<>();
int[] fileDeleteCount = {0};
cleanerPlan.getFilePathsToBeDeletedPerPartition().forEach((partition, deletedPathInfo) -> {
fileDeleteCount[0] += deletedPathInfo.size();
// Files deleted from a partition
List<String> deletedFilenames = deletedPathInfo.stream().map(p -> new Path(p.getFilePath()).getName())
.collect(Collectors.toList());
HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(),
Option.of(deletedFilenames));
records.add(record);
});
LOG.info("Found at " + instantTime + " from CleanerPlan. #partitions_updated=" + records.size()
+ ", #files_deleted=" + fileDeleteCount[0]);
return records;
}
/**
* Finds all files that were deleted as part of a clean and creates metadata table records for them.
*
* @param cleanMetadata
* @param instantTime
* @return a list of metadata table records
*/
public static List<HoodieRecord> convertMetadataToRecords(HoodieCleanMetadata cleanMetadata, String instantTime) {
List<HoodieRecord> records = new LinkedList<>();
int[] fileDeleteCount = {0};
cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> {
// Files deleted from a partition
List<String> deletedFiles = partitionMetadata.getSuccessDeleteFiles();
HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.empty(),
Option.of(new ArrayList<>(deletedFiles)));
records.add(record);
fileDeleteCount[0] += deletedFiles.size();
});
LOG.info("Updating at " + instantTime + " from Clean. #partitions_updated=" + records.size()
+ ", #files_deleted=" + fileDeleteCount[0]);
return records;
}
/**
* Aggregates all files deleted and appended to from all rollbacks associated with a restore operation then
* creates metadata table records for them.
*
* @param restoreMetadata
* @param instantTime
* @return a list of metadata table records
*/
public static List<HoodieRecord> convertMetadataToRecords(HoodieRestoreMetadata restoreMetadata, String instantTime, Option<String> lastSyncTs) {
Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>();
Map<String, List<String>> partitionToDeletedFiles = new HashMap<>();
restoreMetadata.getHoodieRestoreMetadata().values().forEach(rms -> {
rms.forEach(rm -> processRollbackMetadata(rm, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs));
});
return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Restore");
}
public static List<HoodieRecord> convertMetadataToRecords(HoodieRollbackMetadata rollbackMetadata, String instantTime, Option<String> lastSyncTs) {
Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>();
Map<String, List<String>> partitionToDeletedFiles = new HashMap<>();
processRollbackMetadata(rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs);
return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Rollback");
}
/**
* Extracts information about the deleted and append files from the {@code HoodieRollbackMetadata}.
*
* During a rollback files may be deleted (COW, MOR) or rollback blocks be appended (MOR only) to files. This
* function will extract this change file for each partition.
*
* @param rollbackMetadata {@code HoodieRollbackMetadata}
* @param partitionToDeletedFiles The {@code Map} to fill with files deleted per partition.
* @param partitionToAppendedFiles The {@code Map} to fill with files appended per partition and their sizes.
*/
private static void processRollbackMetadata(HoodieRollbackMetadata rollbackMetadata,
Map<String, List<String>> partitionToDeletedFiles,
Map<String, Map<String, Long>> partitionToAppendedFiles,
Option<String> lastSyncTs) {
rollbackMetadata.getPartitionMetadata().values().forEach(pm -> {
// If commit being rolled back has not been synced to metadata table yet then there is no need to update metadata
if (lastSyncTs.isPresent() && HoodieTimeline.compareTimestamps(rollbackMetadata.getCommitsRollback().get(0), HoodieTimeline.GREATER_THAN, lastSyncTs.get())) {
return;
}
final String partition = pm.getPartitionPath();
if (!pm.getSuccessDeleteFiles().isEmpty()) {
if (!partitionToDeletedFiles.containsKey(partition)) {
partitionToDeletedFiles.put(partition, new ArrayList<>());
}
// Extract deleted file name from the absolute paths saved in getSuccessDeleteFiles()
List<String> deletedFiles = pm.getSuccessDeleteFiles().stream().map(p -> new Path(p).getName())
.collect(Collectors.toList());
partitionToDeletedFiles.get(partition).addAll(deletedFiles);
}
if (!pm.getAppendFiles().isEmpty()) {
if (!partitionToAppendedFiles.containsKey(partition)) {
partitionToAppendedFiles.put(partition, new HashMap<>());
}
// Extract appended file name from the absolute paths saved in getAppendFiles()
pm.getAppendFiles().forEach((path, size) -> {
partitionToAppendedFiles.get(partition).merge(new Path(path).getName(), size, (oldSize, newSizeCopy) -> {
return size + oldSize;
});
});
}
});
}
private static List<HoodieRecord> convertFilesToRecords(Map<String, List<String>> partitionToDeletedFiles,
Map<String, Map<String, Long>> partitionToAppendedFiles, String instantTime,
String operation) {
List<HoodieRecord> records = new LinkedList<>();
int[] fileChangeCount = {0, 0}; // deletes, appends
partitionToDeletedFiles.forEach((partition, deletedFiles) -> {
fileChangeCount[0] += deletedFiles.size();
Option<Map<String, Long>> filesAdded = Option.empty();
if (partitionToAppendedFiles.containsKey(partition)) {
filesAdded = Option.of(partitionToAppendedFiles.remove(partition));
}
HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, filesAdded,
Option.of(new ArrayList<>(deletedFiles)));
records.add(record);
});
partitionToAppendedFiles.forEach((partition, appendedFileMap) -> {
fileChangeCount[1] += appendedFileMap.size();
// Validate that no appended file has been deleted
ValidationUtils.checkState(
!appendedFileMap.keySet().removeAll(partitionToDeletedFiles.getOrDefault(partition, Collections.emptyList())),
"Rollback file cannot both be appended and deleted");
// New files added to a partition
HoodieRecord record = HoodieMetadataPayload.createPartitionFilesRecord(partition, Option.of(appendedFileMap),
Option.empty());
records.add(record);
});
LOG.info("Found at " + instantTime + " from " + operation + ". #partitions_updated=" + records.size()
+ ", #files_deleted=" + fileChangeCount[0] + ", #files_appended=" + fileChangeCount[1]);
return records;
}
}