[HUDI-2040] Make flink writer as exactly-once by default (#3106)
This commit is contained in:
@@ -651,7 +651,6 @@ public class TestWriteCopyOnWrite {
|
||||
@Test
|
||||
public void testWriteExactlyOnce() throws Exception {
|
||||
// reset the config option
|
||||
conf.setBoolean(FlinkOptions.WRITE_EXACTLY_ONCE_ENABLED, true);
|
||||
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, 3);
|
||||
conf.setDouble(FlinkOptions.WRITE_TASK_MAX_SIZE, 200.0006); // 630 bytes buffer size
|
||||
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
|
||||
|
||||
@@ -23,10 +23,10 @@ import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapFunction;
|
||||
import org.apache.hudi.sink.bootstrap.IndexRecord;
|
||||
import org.apache.hudi.sink.StreamWriteFunction;
|
||||
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
|
||||
import org.apache.hudi.sink.bootstrap.BootstrapFunction;
|
||||
import org.apache.hudi.sink.bootstrap.IndexRecord;
|
||||
import org.apache.hudi.sink.event.BatchWriteSuccessEvent;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignOperator;
|
||||
@@ -48,13 +48,12 @@ import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorEventG
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* A wrapper class to manipulate the {@link StreamWriteFunction} instance for testing.
|
||||
@@ -213,18 +212,8 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
|
||||
public void checkpointComplete(long checkpointId) {
|
||||
functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
|
||||
if (asyncCompaction) {
|
||||
// sleep for a while to give a change for scheduling compaction,
|
||||
// see HoodieActiveTimeline#createNewInstantTime for details.
|
||||
try {
|
||||
TimeUnit.SECONDS.sleep(2);
|
||||
} catch (InterruptedException e) {
|
||||
throw new HoodieException("Waiting for checkpoint success exception", e);
|
||||
}
|
||||
}
|
||||
coordinator.notifyCheckpointComplete(checkpointId);
|
||||
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
|
||||
this.writeFunction.notifyCheckpointComplete(checkpointId);
|
||||
if (asyncCompaction) {
|
||||
try {
|
||||
compactFunctionWrapper.compact(checkpointId);
|
||||
|
||||
Reference in New Issue
Block a user