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

@@ -39,9 +39,7 @@ 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;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
@@ -49,7 +47,6 @@ import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieMetricsConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieMetadataException;
@@ -61,18 +58,14 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX;
import static org.apache.hudi.metadata.HoodieTableMetadata.NON_PARTITIONED_NAME;
import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP;
/**
@@ -211,7 +204,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
return metadataWriteConfig;
}
public HoodieTableMetadata metadata() {
public HoodieBackedTableMetadata metadata() {
return metadata;
}
@@ -340,7 +333,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
if (p.getRight().length > filesInDir.size()) {
// Is a partition. Add all data files to result.
partitionToFileStatus.put(p.getLeft().getName(), filesInDir);
String partitionName = FSUtils.getRelativePartitionPath(new Path(datasetMetaClient.getBasePath()), p.getLeft());
partitionToFileStatus.put(partitionName, filesInDir);
} else {
// Add sub-dirs to the queue
pathsToList.addAll(Arrays.stream(p.getRight())
@@ -374,35 +368,10 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
final HoodieActiveTimeline timeline = datasetMetaClient.getActiveTimeline();
for (HoodieInstant instant : instantsToSync) {
LOG.info("Syncing instant " + instant + " to metadata table");
ValidationUtils.checkArgument(instant.isCompleted(), "Only completed instants can be synced.");
switch (instant.getAction()) {
case HoodieTimeline.CLEAN_ACTION:
HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(datasetMetaClient, instant);
update(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);
update(commitMetadata, instant.getTimestamp());
break;
case HoodieTimeline.ROLLBACK_ACTION:
HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata(
timeline.getInstantDetails(instant).get());
update(rollbackMetadata, instant.getTimestamp());
break;
case HoodieTimeline.RESTORE_ACTION:
HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata(
timeline.getInstantDetails(instant).get());
update(restoreMetadata, instant.getTimestamp());
break;
case HoodieTimeline.SAVEPOINT_ACTION:
// Nothing to be done here
break;
default:
throw new HoodieException("Unknown type of action " + instant.getAction());
Option<List<HoodieRecord>> records = HoodieTableMetadataUtil.convertInstantToMetaRecords(datasetMetaClient, instant, metadata.getSyncedInstantTime());
if (records.isPresent()) {
commit(records.get(), MetadataPartitionType.FILES.partitionPath(), instant.getTimestamp());
}
}
// re-init the table metadata, for any future writes.
@@ -420,44 +389,10 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
*/
@Override
public void update(HoodieCommitMetadata commitMetadata, String instantTime) {
if (!enabled) {
return;
if (enabled) {
List<HoodieRecord> records = HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime);
commit(records, MetadataPartitionType.FILES.partitionPath(), 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());
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime);
}
/**
@@ -468,26 +403,10 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
*/
@Override
public void update(HoodieCleanerPlan cleanerPlan, String instantTime) {
if (!enabled) {
return;
if (enabled) {
List<HoodieRecord> records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanerPlan, instantTime);
commit(records, MetadataPartitionType.FILES.partitionPath(), 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("Updating at " + instantTime + " from CleanerPlan. #partitions_updated=" + records.size()
+ ", #files_deleted=" + fileDeleteCount[0]);
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime);
}
/**
@@ -498,26 +417,10 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
*/
@Override
public void update(HoodieCleanMetadata cleanMetadata, String instantTime) {
if (!enabled) {
return;
if (enabled) {
List<HoodieRecord> records = HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime);
commit(records, MetadataPartitionType.FILES.partitionPath(), 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]);
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime);
}
/**
@@ -528,16 +431,10 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
*/
@Override
public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) {
if (!enabled) {
return;
if (enabled) {
List<HoodieRecord> records = HoodieTableMetadataUtil.convertMetadataToRecords(restoreMetadata, instantTime, metadata.getSyncedInstantTime());
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime);
}
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));
});
commitRollback(partitionToDeletedFiles, partitionToAppendedFiles, instantTime, "Restore");
}
/**
@@ -548,119 +445,10 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
*/
@Override
public void update(HoodieRollbackMetadata rollbackMetadata, String instantTime) {
if (!enabled) {
return;
if (enabled) {
List<HoodieRecord> records = HoodieTableMetadataUtil.convertMetadataToRecords(rollbackMetadata, instantTime, metadata.getSyncedInstantTime());
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime);
}
Map<String, Map<String, Long>> partitionToAppendedFiles = new HashMap<>();
Map<String, List<String>> partitionToDeletedFiles = new HashMap<>();
processRollbackMetadata(rollbackMetadata, partitionToDeletedFiles, partitionToAppendedFiles);
commitRollback(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 void processRollbackMetadata(HoodieRollbackMetadata rollbackMetadata,
Map<String, List<String>> partitionToDeletedFiles,
Map<String, Map<String, Long>> partitionToAppendedFiles) {
rollbackMetadata.getPartitionMetadata().values().forEach(pm -> {
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;
});
});
}
});
}
/**
* Create file delete records and commit.
*
* @param partitionToDeletedFiles {@code Map} of partitions and the deleted files
* @param instantTime Timestamp at which the deletes took place
* @param operation Type of the operation which caused the files to be deleted
*/
private void commitRollback(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) -> {
// Rollbacks deletes instants from timeline. The instant being rolled-back may not have been synced to the
// metadata table. Hence, the deleted filed need to be checked against the metadata.
try {
FileStatus[] existingStatuses = metadata.fetchAllFilesInPartition(new Path(metadata.getDatasetBasePath(), partition));
Set<String> currentFiles =
Arrays.stream(existingStatuses).map(s -> s.getPath().getName()).collect(Collectors.toSet());
int origCount = deletedFiles.size();
deletedFiles.removeIf(f -> !currentFiles.contains(f));
if (deletedFiles.size() != origCount) {
LOG.warn("Some Files to be deleted as part of " + operation + " at " + instantTime + " were not found in the "
+ " metadata for partition " + partition
+ ". To delete = " + origCount + ", found=" + deletedFiles.size());
}
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);
} catch (IOException e) {
throw new HoodieMetadataException("Failed to commit rollback deletes at instant " + instantTime, e);
}
});
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("Updating at " + instantTime + " from " + operation + ". #partitions_updated=" + records.size()
+ ", #files_deleted=" + fileChangeCount[0] + ", #files_appended=" + fileChangeCount[1]);
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime);
}
/**

View File

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

View File

@@ -37,6 +37,7 @@ import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -70,6 +71,11 @@ public class TestCompactionAdminClient extends HoodieClientTestBase {
client = new CompactionAdminClient(context, basePath);
}
@AfterEach
public void cleanUp() throws Exception {
cleanupResources();
}
@Test
public void testUnscheduleCompactionPlan() throws Exception {
int numEntriesPerInstant = 10;

View File

@@ -79,8 +79,8 @@ import org.junit.jupiter.api.io.TempDir;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
public class TestHoodieFsMetadata extends HoodieClientTestHarness {
private static final Logger LOG = LogManager.getLogger(TestHoodieFsMetadata.class);
public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
private static final Logger LOG = LogManager.getLogger(TestHoodieBackedMetadata.class);
@TempDir
public java.nio.file.Path tempFolder;
@@ -95,7 +95,7 @@ public class TestHoodieFsMetadata extends HoodieClientTestHarness {
initSparkContexts("TestHoodieMetadata");
initFileSystem();
fs.mkdirs(new Path(basePath));
initMetaClient();
initMetaClient(tableType);
initTestDataGenerator();
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
@@ -371,7 +371,41 @@ public class TestHoodieFsMetadata extends HoodieClientTestHarness {
client.syncTableMetadata();
validateMetadata(client);
}
}
/**
* Test when syncing rollback to metadata if the commit being rolled back has not been synced that essentially a no-op
* occurs to metadata.
* @throws Exception
*/
@Test
public void testRollbackUnsyncedCommit() throws Exception {
init(HoodieTableType.COPY_ON_WRITE);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// Initialize table with metadata
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
client.startCommitWithTime(newCommitTime);
List<WriteStatus> writeStatuses = client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect();
assertNoWriteErrors(writeStatuses);
validateMetadata(client);
}
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
// Commit with metadata disabled
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateUpdates(newCommitTime, 10);
List<WriteStatus> writeStatuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
assertNoWriteErrors(writeStatuses);
client.rollback(newCommitTime);
}
try (SparkRDDWriteClient client = new SparkRDDWriteClient<>(engineContext, getWriteConfig(true, true))) {
validateMetadata(client);
}
}
/**
@@ -636,6 +670,84 @@ public class TestHoodieFsMetadata extends HoodieClientTestHarness {
}
}
/**
* Test when reading from metadata table which is out of sync with dataset that results are still consistent.
*/
// @ParameterizedTest
// @EnumSource(HoodieTableType.class)
@Test
public void testMetadataOutOfSync() throws Exception {
init(HoodieTableType.COPY_ON_WRITE);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
SparkRDDWriteClient unsyncedClient = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true));
// Enable metadata so table is initialized
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
// Perform Bulk Insert
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
client.bulkInsert(jsc.parallelize(records, 1), newCommitTime).collect();
}
// Perform commit operations with metadata disabled
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
// Perform Insert
String newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 20);
client.insert(jsc.parallelize(records, 1), newCommitTime).collect();
// Perform Upsert
newCommitTime = "003";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUniqueUpdates(newCommitTime, 20);
client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Compaction
if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) {
newCommitTime = "004";
client.scheduleCompactionAtInstant(newCommitTime, Option.empty());
client.compact(newCommitTime);
}
}
assertFalse(metadata(unsyncedClient).isInSync());
validateMetadata(unsyncedClient);
// Perform clean operation with metadata disabled
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
// One more commit needed to trigger clean so upsert and compact
String newCommitTime = "005";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateUpdates(newCommitTime, 20);
client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) {
newCommitTime = "006";
client.scheduleCompactionAtInstant(newCommitTime, Option.empty());
client.compact(newCommitTime);
}
// Clean
newCommitTime = "007";
client.clean(newCommitTime);
}
assertFalse(metadata(unsyncedClient).isInSync());
validateMetadata(unsyncedClient);
// Perform restore with metadata disabled
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, false))) {
client.restoreToInstant("004");
}
assertFalse(metadata(unsyncedClient).isInSync());
validateMetadata(unsyncedClient);
}
/**
* Validate the metadata tables contents to ensure it matches what is on the file system.
*
@@ -643,8 +755,9 @@ public class TestHoodieFsMetadata extends HoodieClientTestHarness {
*/
private void validateMetadata(SparkRDDWriteClient client) throws IOException {
HoodieWriteConfig config = client.getConfig();
HoodieBackedTableMetadataWriter metadataWriter = metadataWriter(client);
assertNotNull(metadataWriter, "MetadataWriter should have been initialized");
HoodieBackedTableMetadata tableMetadata = metadata(client);
assertNotNull(tableMetadata, "MetadataReader should have been initialized");
if (!config.useFileListingMetadata()) {
return;
}
@@ -652,17 +765,9 @@ public class TestHoodieFsMetadata extends HoodieClientTestHarness {
HoodieTimer timer = new HoodieTimer().startTimer();
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
// Validate write config for metadata table
HoodieWriteConfig metadataWriteConfig = metadataWriter.getWriteConfig();
assertFalse(metadataWriteConfig.useFileListingMetadata(), "No metadata table for metadata table");
assertFalse(metadataWriteConfig.getFileListingMetadataVerify(), "No verify for metadata table");
// Metadata table should be in sync with the dataset
assertTrue(metadata(client).isInSync());
// Partitions should match
List<String> fsPartitions = FSUtils.getAllFoldersWithPartitionMetaFile(fs, basePath);
List<String> metadataPartitions = metadataWriter.metadata().getAllPartitionPaths();
List<String> metadataPartitions = tableMetadata.getAllPartitionPaths();
Collections.sort(fsPartitions);
Collections.sort(metadataPartitions);
@@ -684,7 +789,7 @@ public class TestHoodieFsMetadata extends HoodieClientTestHarness {
partitionPath = new Path(basePath, partition);
}
FileStatus[] fsStatuses = FSUtils.getAllDataFilesInPartition(fs, partitionPath);
FileStatus[] metaStatuses = metadataWriter.metadata().getAllFilesInPartition(partitionPath);
FileStatus[] metaStatuses = tableMetadata.getAllFilesInPartition(partitionPath);
List<String> fsFileNames = Arrays.stream(fsStatuses)
.map(s -> s.getPath().getName()).collect(Collectors.toList());
List<String> metadataFilenames = Arrays.stream(metaStatuses)
@@ -705,9 +810,9 @@ public class TestHoodieFsMetadata extends HoodieClientTestHarness {
// FileSystemView should expose the same data
List<HoodieFileGroup> fileGroups = tableView.getAllFileGroups(partition).collect(Collectors.toList());
fileGroups.forEach(g -> LogManager.getLogger(TestHoodieFsMetadata.class).info(g));
fileGroups.forEach(g -> g.getAllBaseFiles().forEach(b -> LogManager.getLogger(TestHoodieFsMetadata.class).info(b)));
fileGroups.forEach(g -> g.getAllFileSlices().forEach(s -> LogManager.getLogger(TestHoodieFsMetadata.class).info(s)));
fileGroups.forEach(g -> LogManager.getLogger(TestHoodieBackedMetadata.class).info(g));
fileGroups.forEach(g -> g.getAllBaseFiles().forEach(b -> LogManager.getLogger(TestHoodieBackedMetadata.class).info(b)));
fileGroups.forEach(g -> g.getAllFileSlices().forEach(s -> LogManager.getLogger(TestHoodieBackedMetadata.class).info(s)));
long numFiles = fileGroups.stream()
.mapToLong(g -> g.getAllBaseFiles().count() + g.getAllFileSlices().mapToLong(s -> s.getLogFiles().count()).sum())
@@ -720,10 +825,17 @@ public class TestHoodieFsMetadata extends HoodieClientTestHarness {
}
});
HoodieTableMetaClient metadataMetaClient = new HoodieTableMetaClient(hadoopConf, metadataTableBasePath);
HoodieBackedTableMetadataWriter metadataWriter = metadataWriter(client);
assertNotNull(metadataWriter, "MetadataWriter should have been initialized");
// Validate write config for metadata table
HoodieWriteConfig metadataWriteConfig = metadataWriter.getWriteConfig();
assertFalse(metadataWriteConfig.useFileListingMetadata(), "No metadata table for metadata table");
assertFalse(metadataWriteConfig.getFileListingMetadataVerify(), "No verify for metadata table");
// Metadata table should be in sync with the dataset
assertTrue(metadataWriter.metadata().isInSync());
assertTrue(metadata(client).isInSync());
HoodieTableMetaClient metadataMetaClient = new HoodieTableMetaClient(hadoopConf, metadataTableBasePath);
// Metadata table is MOR
assertEquals(metadataMetaClient.getTableType(), HoodieTableType.MERGE_ON_READ, "Metadata Table should be MOR");

View File

@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
@@ -91,6 +92,11 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
initDFSMetaClient();
}
@AfterEach
public void cleanUp() throws Exception {
cleanupResources();
}
@Test
public void testLeftOverUpdatedPropFileCleanup() throws IOException {
testUpgradeInternal(true, true, HoodieTableType.MERGE_ON_READ);

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
@@ -204,6 +205,10 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
* @throws IOException
*/
protected void initMetaClient() throws IOException {
initMetaClient(getTableType());
}
protected void initMetaClient(HoodieTableType tableType) throws IOException {
if (basePath == null) {
throw new IllegalStateException("The base path has not been initialized.");
}
@@ -212,7 +217,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
throw new IllegalStateException("The Spark context has not been initialized.");
}
metaClient = HoodieTestUtils.init(context.getHadoopConf().get(), basePath, getTableType());
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType);
}
/**