[HUDI-3634] Could read empty or partial HoodieCommitMetaData in downstream if using HDFS (#5048)
Add the differentiated logic of creating immutable file in HDFS by first creating the file.tmp and then renaming the file
This commit is contained in:
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.common.table.timeline;
|
||||
|
||||
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
||||
import org.apache.hudi.common.fs.NoOpConsistencyGuard;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
@@ -32,6 +34,7 @@ import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.ParseException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
@@ -45,6 +48,7 @@ import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.BiConsumer;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
@@ -199,6 +203,25 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
assertTrue(activeCommitTimeline.isBeforeTimelineStarts("00"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAllowTempCommit() {
|
||||
shouldAllowTempCommit(true, hoodieMetaClient -> {
|
||||
timeline = new HoodieActiveTimeline(hoodieMetaClient);
|
||||
|
||||
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
timeline.createNewInstant(instant1);
|
||||
|
||||
byte[] data = "commit".getBytes(StandardCharsets.UTF_8);
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant1.getAction(),
|
||||
instant1.getTimestamp()), Option.of(data));
|
||||
|
||||
timeline = timeline.reload();
|
||||
|
||||
assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent());
|
||||
assertEquals(instant1.getTimestamp(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp());
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetContiguousCompletedWriteTimeline() {
|
||||
// a mock timeline with holes
|
||||
@@ -594,4 +617,25 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
}
|
||||
return allInstants;
|
||||
}
|
||||
|
||||
private void shouldAllowTempCommit(boolean allowTempCommit, Consumer<HoodieTableMetaClient> fun) {
|
||||
if (allowTempCommit) {
|
||||
HoodieWrapperFileSystem fs = metaClient.getFs();
|
||||
HoodieWrapperFileSystem newFs = new HoodieWrapperFileSystem(fs.getFileSystem(), new NoOpConsistencyGuard()) {
|
||||
@Override
|
||||
protected boolean needCreateTempFile() {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
metaClient.setFs(newFs);
|
||||
try {
|
||||
fun.accept(metaClient);
|
||||
} finally {
|
||||
metaClient.setFs(fs);
|
||||
}
|
||||
return;
|
||||
}
|
||||
fun.accept(metaClient);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user