[HUDI-3370] The files recorded in the commit may not match the actual ones for MOR Compaction (#4753)
* use HoodieCommitMetadata to replace writeStatuses computation Co-authored-by: yuezhang <yuezhang@freewheel.tv>
This commit is contained in:
@@ -38,6 +38,7 @@ import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieLockConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieWriteConflictException;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -364,8 +365,8 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
|
||||
latchCountDownAndWait(runCountDownLatch, 30000);
|
||||
if (tableType == HoodieTableType.MERGE_ON_READ) {
|
||||
assertDoesNotThrow(() -> {
|
||||
JavaRDD<WriteStatus> writeStatusJavaRDD = (JavaRDD<WriteStatus>) client2.compact("005");
|
||||
client2.commitCompaction("005", writeStatusJavaRDD, Option.empty());
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = client2.compact("005");
|
||||
client2.commitCompaction("005", compactionMetadata.getCommitMetadata().get(), Option.empty());
|
||||
validInstants.add("005");
|
||||
});
|
||||
}
|
||||
|
||||
@@ -45,6 +45,7 @@ import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.deltacommit.BaseSparkDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
@@ -258,7 +259,7 @@ public class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness
|
||||
|
||||
// Do a compaction
|
||||
String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
JavaRDD<WriteStatus> result = (JavaRDD<WriteStatus>) writeClient.compact(compactionInstantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = writeClient.compact(compactionInstantTime);
|
||||
|
||||
// Verify that recently written compacted data file has no log file
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
@@ -275,8 +276,7 @@ public class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness
|
||||
for (FileSlice slice : groupedLogFiles) {
|
||||
assertEquals(0, slice.getLogFiles().count(), "After compaction there should be no log files visible on a full view");
|
||||
}
|
||||
List<WriteStatus> writeStatuses = result.collect();
|
||||
assertTrue(writeStatuses.stream().anyMatch(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)));
|
||||
assertTrue(result.getCommitMetadata().get().getWritePartitionPaths().stream().anyMatch(part -> part.contentEquals(partitionPath)));
|
||||
}
|
||||
|
||||
// Check the entire dataset has all records still
|
||||
@@ -442,8 +442,9 @@ public class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness
|
||||
// Test small file handling after compaction
|
||||
instantTime = "002";
|
||||
client.scheduleCompactionAtInstant(instantTime, Option.of(metadata.getExtraMetadata()));
|
||||
statuses = (JavaRDD<WriteStatus>) client.compact(instantTime);
|
||||
client.commitCompaction(instantTime, statuses, Option.empty());
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = client.compact(instantTime);
|
||||
statuses = compactionMetadata.getWriteStatuses();
|
||||
client.commitCompaction(instantTime, compactionMetadata.getCommitMetadata().get(), Option.empty());
|
||||
|
||||
// Read from commit file
|
||||
table = HoodieSparkTable.create(cfg, context());
|
||||
|
||||
@@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
@@ -41,6 +42,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils;
|
||||
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness;
|
||||
@@ -56,6 +58,7 @@ import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
@@ -307,11 +310,12 @@ public class TestHoodieSparkMergeOnReadTableInsertUpdateDelete extends SparkClie
|
||||
assertTrue(numLogFiles > 0);
|
||||
// Do a compaction
|
||||
String instantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
statuses = (JavaRDD<WriteStatus>) writeClient.compact(instantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = writeClient.compact(instantTime);
|
||||
String extension = table.getBaseFileExtension();
|
||||
assertEquals(numLogFiles, statuses.map(status -> status.getStat().getPath().contains(extension)).count());
|
||||
assertEquals(numLogFiles, statuses.count());
|
||||
writeClient.commitCompaction(instantTime, statuses, Option.empty());
|
||||
Collection<List<HoodieWriteStat>> stats = compactionMetadata.getCommitMetadata().get().getPartitionToWriteStats().values();
|
||||
assertEquals(numLogFiles, stats.stream().flatMap(Collection::stream).filter(state -> state.getPath().contains(extension)).count());
|
||||
assertEquals(numLogFiles, stats.stream().mapToLong(Collection::size).sum());
|
||||
writeClient.commitCompaction(instantTime, compactionMetadata.getCommitMetadata().get(), Option.empty());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -30,6 +30,7 @@ import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.marker.MarkerType;
|
||||
@@ -50,6 +51,7 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils;
|
||||
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness;
|
||||
|
||||
@@ -455,8 +457,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
||||
|
||||
compactionInstantTime = "006";
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
|
||||
JavaRDD<WriteStatus> ws = (JavaRDD<WriteStatus>) client.compact(compactionInstantTime);
|
||||
client.commitCompaction(compactionInstantTime, ws, Option.empty());
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = client.compact(compactionInstantTime);
|
||||
client.commitCompaction(compactionInstantTime, compactionMetadata.getCommitMetadata().get(), Option.empty());
|
||||
|
||||
allFiles = listAllBaseFilesInPath(hoodieTable);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
@@ -543,8 +545,8 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
String compactionInstantTime = "005";
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
|
||||
JavaRDD<WriteStatus> ws = (JavaRDD<WriteStatus>) client.compact(compactionInstantTime);
|
||||
client.commitCompaction(compactionInstantTime, ws, Option.empty());
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = client.compact(compactionInstantTime);
|
||||
client.commitCompaction(compactionInstantTime, compactionMetadata.getCommitMetadata().get(), Option.empty());
|
||||
|
||||
validateRecords(cfg, metaClient, updates3);
|
||||
List<HoodieRecord> updates4 = updateAndGetRecords("006", client, dataGen, records);
|
||||
@@ -755,11 +757,14 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
||||
assertTrue(numLogFiles > 0);
|
||||
// Do a compaction
|
||||
newCommitTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
statuses = (JavaRDD<WriteStatus>) writeClient.compact(newCommitTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = writeClient.compact(newCommitTime);
|
||||
statuses = compactionMetadata.getWriteStatuses();
|
||||
// Ensure all log files have been compacted into base files
|
||||
String extension = table.getBaseFileExtension();
|
||||
assertEquals(numLogFiles, statuses.map(status -> status.getStat().getPath().contains(extension)).count());
|
||||
assertEquals(numLogFiles, statuses.count());
|
||||
Collection<List<HoodieWriteStat>> stats = compactionMetadata.getCommitMetadata().get().getPartitionToWriteStats().values();
|
||||
assertEquals(numLogFiles, stats.stream().flatMap(Collection::stream).filter(state -> state.getPath().contains(extension)).count());
|
||||
assertEquals(numLogFiles, stats.stream().mapToLong(Collection::size).sum());
|
||||
|
||||
//writeClient.commitCompaction(newCommitTime, statuses, Option.empty());
|
||||
// Trigger a rollback of compaction
|
||||
table.getActiveTimeline().reload();
|
||||
@@ -862,14 +867,15 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
|
||||
private long doCompaction(SparkRDDWriteClient client, HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, long numLogFiles) throws IOException {
|
||||
// Do a compaction
|
||||
String instantTime = client.scheduleCompaction(Option.empty()).get().toString();
|
||||
JavaRDD<WriteStatus> writeStatuses = (JavaRDD<WriteStatus>) client.compact(instantTime);
|
||||
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = client.compact(instantTime);
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context(), metaClient);
|
||||
String extension = table.getBaseFileExtension();
|
||||
assertEquals(numLogFiles, writeStatuses.map(status -> status.getStat().getPath().contains(extension)).count());
|
||||
assertEquals(numLogFiles, writeStatuses.count());
|
||||
client.commitCompaction(instantTime, writeStatuses, Option.empty());
|
||||
Collection<List<HoodieWriteStat>> stats = compactionMetadata.getCommitMetadata().get().getPartitionToWriteStats().values();
|
||||
assertEquals(numLogFiles, stats.stream().flatMap(Collection::stream).filter(state -> state.getPath().contains(extension)).count());
|
||||
assertEquals(numLogFiles, stats.stream().mapToLong(Collection::size).sum());
|
||||
client.commitCompaction(instantTime, compactionMetadata.getCommitMetadata().get(), Option.empty());
|
||||
return numLogFiles;
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user