[HUDI-2119] Ensure the rolled-back instance was previously synced to the Metadata Table when syncing a Rollback Instant. (#3210)
* [HUDI-2119] Ensure the rolled-back instance was previously synced to the Metadata Table when syncing a Rollback Instant. If the rolled-back instant was synced to the Metadata Table, a corresponding deltacommit with the same timestamp should have been created on the Metadata Table timeline. To ensure we can always perfomr this check, the Metadata Table instants should not be archived until their corresponding instants are present in the dataset timeline. But ensuring this requires a large number of instants to be kept on the metadata table. In this change, the metadata table will keep atleast the number of instants that the main dataset is keeping. If the instant being rolled back was before the metadata table timeline, the code will throw an exception and the metadata table will have to be re-bootstrapped. This should be a very rare occurance and should occur only when the dataset is being repaired by rolling back multiple commits or restoring to an much older time. * Fixed checkstyle * Improvements from review comments. Fixed checkstyle Replaced explicit null check with Option.ofNullable Removed redundant function getSynedInstantTime * Renamed getSyncedInstantTime and getSyncedInstantTimeForReader. Sync is confusing so renamed to getUpdateTime() and getReaderTime(). * Removed getReaderTime which is only for testing as the same method can be accessed during testing differently without making it part of the public interface. * Fix compilation error * Reverting changes to HoodieMetadataFileSystemView Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -28,6 +28,7 @@ import org.apache.hudi.common.metrics.Registry;
|
||||
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.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
@@ -64,7 +65,6 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||
protected final HoodieMetadataConfig metadataConfig;
|
||||
// Directory used for Spillable Map when merging records
|
||||
protected final String spillableMapDirectory;
|
||||
private String syncedInstantTime;
|
||||
|
||||
protected boolean enabled;
|
||||
private TimelineMergedTableMetadata timelineMergedMetadata;
|
||||
@@ -84,9 +84,6 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||
} else {
|
||||
this.metrics = Option.empty();
|
||||
}
|
||||
if (enabled) {
|
||||
openTimelineScanner();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -298,29 +295,14 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||
|
||||
protected abstract Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKeyFromMetadata(String key);
|
||||
|
||||
private void openTimelineScanner() {
|
||||
protected void openTimelineScanner(HoodieActiveTimeline metadataTableTimeline) {
|
||||
if (timelineMergedMetadata == null) {
|
||||
List<HoodieInstant> unSyncedInstants = findInstantsToSyncForReader();
|
||||
timelineMergedMetadata =
|
||||
new TimelineMergedTableMetadata(datasetMetaClient, unSyncedInstants, getSyncedInstantTime(), null);
|
||||
|
||||
syncedInstantTime = unSyncedInstants.isEmpty() ? getLatestDatasetInstantTime()
|
||||
: unSyncedInstants.get(unSyncedInstants.size() - 1).getTimestamp();
|
||||
new TimelineMergedTableMetadata(datasetMetaClient, metadataTableTimeline, unSyncedInstants, getUpdateTime(), null);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the timestamp of the latest synced instant.
|
||||
*/
|
||||
@Override
|
||||
public Option<String> getSyncedInstantTime() {
|
||||
if (!enabled) {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
return Option.ofNullable(syncedInstantTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the instants which are not-synced to the {@code HoodieTableMetadata}.
|
||||
*
|
||||
@@ -344,8 +326,19 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
|
||||
return engineContext != null ? engineContext : new HoodieLocalEngineContext(hadoopConf.get());
|
||||
}
|
||||
|
||||
public HoodieMetadataConfig getMetadataConfig() {
|
||||
return metadataConfig;
|
||||
}
|
||||
|
||||
protected String getLatestDatasetInstantTime() {
|
||||
return datasetMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant()
|
||||
.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP);
|
||||
}
|
||||
|
||||
public Option<String> getReaderTime() {
|
||||
if (timelineMergedMetadata == null) {
|
||||
return Option.empty();
|
||||
}
|
||||
return timelineMergedMetadata.getSyncedInstantTime();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -126,7 +126,7 @@ public class FileSystemBackedTableMetadata implements HoodieTableMetadata {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<String> getSyncedInstantTime() {
|
||||
public Option<String> getUpdateTime() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
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.view.HoodieTableFileSystemView;
|
||||
@@ -115,6 +116,10 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
this.metaClient = null;
|
||||
this.tableConfig = null;
|
||||
}
|
||||
|
||||
if (enabled) {
|
||||
openTimelineScanner(metaClient.getActiveTimeline());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -272,6 +277,20 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
logRecordScanner = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the timestamp of the latest synced instant.
|
||||
*/
|
||||
@Override
|
||||
public Option<String> getUpdateTime() {
|
||||
if (!enabled) {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline();
|
||||
return timeline.getDeltaCommitTimeline().filterCompletedInstants()
|
||||
.lastInstant().map(HoodieInstant::getTimestamp);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return an ordered list of instants which have not been synced to the Metadata Table.
|
||||
*/
|
||||
|
||||
@@ -18,11 +18,6 @@
|
||||
|
||||
package org.apache.hudi.metadata;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
@@ -31,6 +26,11 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* {@code HoodieTableFileSystemView} implementation that retrieved partition listings from the Metadata Table.
|
||||
*/
|
||||
|
||||
@@ -68,6 +68,9 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable {
|
||||
* @param basePath The base path to check
|
||||
*/
|
||||
static boolean isMetadataTable(String basePath) {
|
||||
if (basePath.endsWith(Path.SEPARATOR)) {
|
||||
basePath = basePath.substring(0, basePath.length() - 1);
|
||||
}
|
||||
return basePath.endsWith(METADATA_TABLE_REL_PATH);
|
||||
}
|
||||
|
||||
@@ -102,9 +105,11 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable {
|
||||
Map<String, FileStatus[]> getAllFilesInPartitions(List<String> partitionPaths) throws IOException;
|
||||
|
||||
/**
|
||||
* Get the instant time to which the metadata is synced w.r.t data timeline.
|
||||
* Get the instant time at which Metadata Table was last updated.
|
||||
*
|
||||
* This is the timestamp of the Instant on the dataset which was last synced to the Metadata Table.
|
||||
*/
|
||||
Option<String> getSyncedInstantTime();
|
||||
Option<String> getUpdateTime();
|
||||
|
||||
boolean isInSync();
|
||||
}
|
||||
|
||||
@@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
@@ -33,7 +34,7 @@ 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.hudi.exception.HoodieMetadataException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -61,11 +62,13 @@ public class HoodieTableMetadataUtil {
|
||||
* Converts a timeline instant to metadata table records.
|
||||
*
|
||||
* @param datasetMetaClient The meta client associated with the timeline instant
|
||||
* @param metadataTableTimeline Current timeline of the Metadata Table
|
||||
* @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 {
|
||||
public static Option<List<HoodieRecord>> convertInstantToMetaRecords(HoodieTableMetaClient datasetMetaClient,
|
||||
HoodieActiveTimeline metadataTableTimeline, 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.");
|
||||
@@ -85,12 +88,12 @@ public class HoodieTableMetadataUtil {
|
||||
case HoodieTimeline.ROLLBACK_ACTION:
|
||||
HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata(
|
||||
timeline.getInstantDetails(instant).get());
|
||||
records = Option.of(convertMetadataToRecords(rollbackMetadata, instant.getTimestamp(), lastSyncTs));
|
||||
records = Option.of(convertMetadataToRecords(metadataTableTimeline, 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));
|
||||
records = Option.of(convertMetadataToRecords(metadataTableTimeline, restoreMetadata, instant.getTimestamp(), lastSyncTs));
|
||||
break;
|
||||
case HoodieTimeline.SAVEPOINT_ACTION:
|
||||
// Nothing to be done here
|
||||
@@ -213,21 +216,22 @@ public class HoodieTableMetadataUtil {
|
||||
* @param instantTime
|
||||
* @return a list of metadata table records
|
||||
*/
|
||||
public static List<HoodieRecord> convertMetadataToRecords(HoodieRestoreMetadata restoreMetadata, String instantTime, Option<String> lastSyncTs) {
|
||||
public static List<HoodieRecord> convertMetadataToRecords(HoodieActiveTimeline metadataTableTimeline,
|
||||
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));
|
||||
rms.forEach(rm -> processRollbackMetadata(metadataTableTimeline, rm, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs));
|
||||
});
|
||||
|
||||
return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Restore");
|
||||
}
|
||||
|
||||
public static List<HoodieRecord> convertMetadataToRecords(HoodieRollbackMetadata rollbackMetadata, String instantTime, Option<String> lastSyncTs) {
|
||||
|
||||
public static List<HoodieRecord> convertMetadataToRecords(HoodieActiveTimeline metadataTableTimeline,
|
||||
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);
|
||||
processRollbackMetadata(metadataTableTimeline, rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles, lastSyncTs);
|
||||
return convertFilesToRecords(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Rollback");
|
||||
}
|
||||
|
||||
@@ -236,27 +240,47 @@ public class HoodieTableMetadataUtil {
|
||||
*
|
||||
* 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 metadataTableTimeline Current timeline of the Metdata Table
|
||||
* @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,
|
||||
private static void processRollbackMetadata(HoodieActiveTimeline metadataTableTimeline, HoodieRollbackMetadata rollbackMetadata,
|
||||
Map<String, List<String>> partitionToDeletedFiles,
|
||||
Map<String, Map<String, Long>> partitionToAppendedFiles,
|
||||
Option<String> lastSyncTs) {
|
||||
|
||||
rollbackMetadata.getPartitionMetadata().values().forEach(pm -> {
|
||||
final String instantToRollback = rollbackMetadata.getCommitsRollback().get(0);
|
||||
// Has this rollback produced new files?
|
||||
boolean hasRollbackLogFiles = pm.getRollbackLogFiles() != null && !pm.getRollbackLogFiles().isEmpty();
|
||||
boolean hasNonZeroRollbackLogFiles = hasRollbackLogFiles && pm.getRollbackLogFiles().values().stream().mapToLong(Long::longValue).sum() > 0;
|
||||
// If commit being rolled back has not been synced to metadata table yet then there is no need to update metadata
|
||||
|
||||
// If instant-to-rollback has not been synced to metadata table yet then there is no need to update metadata
|
||||
// This can happen in two cases:
|
||||
// Case 1: Metadata Table timeline is behind the instant-to-rollback.
|
||||
boolean shouldSkip = lastSyncTs.isPresent()
|
||||
&& HoodieTimeline.compareTimestamps(rollbackMetadata.getCommitsRollback().get(0), HoodieTimeline.GREATER_THAN, lastSyncTs.get());
|
||||
&& HoodieTimeline.compareTimestamps(instantToRollback, HoodieTimeline.GREATER_THAN, lastSyncTs.get());
|
||||
|
||||
if (!hasNonZeroRollbackLogFiles && shouldSkip) {
|
||||
LOG.info(String.format("Skipping syncing of rollbackMetadata at %s, given metadata table is already synced upto to %s",
|
||||
rollbackMetadata.getCommitsRollback().get(0), lastSyncTs.get()));
|
||||
instantToRollback, lastSyncTs.get()));
|
||||
return;
|
||||
}
|
||||
|
||||
// Case 2: The instant-to-rollback was never committed to Metadata Table. This can happen if the instant-to-rollback
|
||||
// was a failed commit (never completed) as only completed instants are synced to Metadata Table.
|
||||
// But the required Metadata Table instants should not have been archived
|
||||
HoodieInstant syncedInstant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantToRollback);
|
||||
if (metadataTableTimeline.getCommitsTimeline().isBeforeTimelineStarts(syncedInstant.getTimestamp())) {
|
||||
throw new HoodieMetadataException(String.format("The instant %s required to sync rollback of %s has been archived",
|
||||
syncedInstant, instantToRollback));
|
||||
}
|
||||
|
||||
shouldSkip = !metadataTableTimeline.containsInstant(syncedInstant);
|
||||
if (!hasNonZeroRollbackLogFiles && shouldSkip) {
|
||||
LOG.info(String.format("Skipping syncing of rollbackMetadata at %s, since this instant was never committed to Metadata Table",
|
||||
instantToRollback));
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
@@ -22,6 +22,7 @@ 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.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
@@ -49,16 +50,18 @@ public class TimelineMergedTableMetadata implements Serializable {
|
||||
private List<HoodieInstant> instants;
|
||||
private Option<String> lastSyncTs;
|
||||
private Set<String> mergeKeyFilter;
|
||||
private HoodieActiveTimeline metadataTableTimeline;
|
||||
|
||||
// keep it a simple hash map, so it can be easily passed onto the executors, once merged.
|
||||
protected final Map<String, HoodieRecord<? extends HoodieRecordPayload>> timelineMergedRecords;
|
||||
|
||||
public TimelineMergedTableMetadata(HoodieTableMetaClient metaClient, List<HoodieInstant> instants,
|
||||
Option<String> lastSyncTs, Set<String> mergeKeyFilter) {
|
||||
public TimelineMergedTableMetadata(HoodieTableMetaClient metaClient, HoodieActiveTimeline metadataTableTimeline,
|
||||
List<HoodieInstant> instants, Option<String> lastSyncTs, Set<String> mergeKeyFilter) {
|
||||
this.metaClient = metaClient;
|
||||
this.instants = instants;
|
||||
this.lastSyncTs = lastSyncTs;
|
||||
this.mergeKeyFilter = mergeKeyFilter != null ? mergeKeyFilter : Collections.emptySet();
|
||||
this.metadataTableTimeline = metadataTableTimeline;
|
||||
this.timelineMergedRecords = new HashMap<>();
|
||||
|
||||
scan();
|
||||
@@ -73,7 +76,8 @@ public class TimelineMergedTableMetadata implements Serializable {
|
||||
private void scan() {
|
||||
for (HoodieInstant instant : instants) {
|
||||
try {
|
||||
Option<List<HoodieRecord>> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(metaClient, instant, lastSyncTs);
|
||||
Option<List<HoodieRecord>> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(metaClient,
|
||||
metadataTableTimeline, instant, lastSyncTs);
|
||||
if (records.isPresent()) {
|
||||
records.get().forEach(record -> processNextRecord(record));
|
||||
}
|
||||
@@ -112,4 +116,15 @@ public class TimelineMergedTableMetadata implements Serializable {
|
||||
public Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKey(String key) {
|
||||
return Option.ofNullable((HoodieRecord) timelineMergedRecords.get(key));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the timestamp of the latest synced instant.
|
||||
*/
|
||||
public Option<String> getSyncedInstantTime() {
|
||||
if (instants.isEmpty()) {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
return Option.of(instants.get(instants.size() - 1).getTimestamp());
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user