[HUDI-1054] Several performance fixes during finalizing writes (#1768)
Co-authored-by: Udit Mehrotra <uditme@amazon.com>
This commit is contained in:
@@ -79,7 +79,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
.withParallelism(2, 2).forTable("test-trip-table").build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
}
|
||||
|
||||
@@ -157,7 +157,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
|
||||
assertTrue(archiveLog.archiveIfRequired());
|
||||
assertTrue(archiveLog.archiveIfRequired(jsc));
|
||||
|
||||
// reload the timeline and remove the remaining commits
|
||||
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
||||
@@ -246,7 +246,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match");
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(4, timeline.countInstants(), "Should not archive commits when maxCommitsToKeep is 5");
|
||||
@@ -289,7 +289,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match");
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertTrue(timeline.containsOrBeforeTimelineStarts("100"), "Archived commits should always be safe");
|
||||
@@ -315,7 +315,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match");
|
||||
assertTrue(archiveLog.archiveIfRequired());
|
||||
assertTrue(archiveLog.archiveIfRequired(jsc));
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(5, timeline.countInstants(),
|
||||
"Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)");
|
||||
@@ -349,7 +349,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline();
|
||||
assertEquals(8, timeline.countInstants(), "Loaded 6 commits and the count should match");
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline();
|
||||
assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")),
|
||||
@@ -397,7 +397,7 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
|
||||
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, dfs.getConf());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
|
||||
List<HoodieInstant> archivedInstants = Arrays.asList(instant1, instant2, instant3);
|
||||
|
||||
@@ -28,6 +28,9 @@ import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.io.IOType;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
@@ -45,16 +48,23 @@ public class TestMarkerFiles extends HoodieCommonTestHarness {
|
||||
private MarkerFiles markerFiles;
|
||||
private FileSystem fs;
|
||||
private Path markerFolderPath;
|
||||
private JavaSparkContext jsc;
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws IOException {
|
||||
initPath();
|
||||
initMetaClient();
|
||||
this.jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(TestMarkerFiles.class.getName()));
|
||||
this.fs = FSUtils.getFs(metaClient.getBasePath(), metaClient.getHadoopConf());
|
||||
this.markerFolderPath = new Path(metaClient.getMarkerFolderPath("000"));
|
||||
this.markerFiles = new MarkerFiles(fs, metaClient.getBasePath(), markerFolderPath.toString(), "000");
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void cleanup() {
|
||||
jsc.stop();
|
||||
}
|
||||
|
||||
private void createSomeMarkerFiles() {
|
||||
markerFiles.create("2020/06/01", "file1", IOType.MERGE);
|
||||
markerFiles.create("2020/06/02", "file2", IOType.APPEND);
|
||||
@@ -97,7 +107,7 @@ public class TestMarkerFiles extends HoodieCommonTestHarness {
|
||||
|
||||
// then
|
||||
assertTrue(markerFiles.doesMarkerDirExist());
|
||||
assertTrue(markerFiles.deleteMarkerDir());
|
||||
assertTrue(markerFiles.deleteMarkerDir(jsc, 2));
|
||||
assertFalse(markerFiles.doesMarkerDirExist());
|
||||
}
|
||||
|
||||
@@ -105,7 +115,7 @@ public class TestMarkerFiles extends HoodieCommonTestHarness {
|
||||
public void testDeletionWhenMarkerDirNotExists() throws IOException {
|
||||
// then
|
||||
assertFalse(markerFiles.doesMarkerDirExist());
|
||||
assertFalse(markerFiles.deleteMarkerDir());
|
||||
assertFalse(markerFiles.deleteMarkerDir(jsc, 2));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -120,7 +130,7 @@ public class TestMarkerFiles extends HoodieCommonTestHarness {
|
||||
// then
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(
|
||||
"2020/06/01/file1", "2020/06/03/file3"),
|
||||
markerFiles.createdAndMergedDataPaths().stream().sorted().collect(Collectors.toList())
|
||||
markerFiles.createdAndMergedDataPaths(jsc, 2).stream().sorted().collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user