[HUDI-1598] Write as minor batches during one checkpoint interval for the new writer (#2553)
This commit is contained in:
@@ -20,6 +20,7 @@ package org.apache.hudi.operator;
|
||||
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
|
||||
@@ -150,7 +151,8 @@ public class StreamWriteFunctionTest {
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
funcWrapper.checkpointComplete(2);
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, null);
|
||||
// started a new instant already
|
||||
checkInflightInstant(funcWrapper.getWriteClient());
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
}
|
||||
|
||||
@@ -187,12 +189,13 @@ public class StreamWriteFunctionTest {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(2);
|
||||
// Do not sent the write event and fails the checkpoint
|
||||
assertThrows(HoodieException.class,
|
||||
() -> funcWrapper.checkpointFails(2),
|
||||
"The last checkpoint was aborted, roll back the last write and throw");
|
||||
// this triggers NPE cause there is no inflight instant
|
||||
assertThrows(NullPointerException.class,
|
||||
() -> funcWrapper.checkpointFunction(2),
|
||||
"No inflight instant when flushing data");
|
||||
// do not sent the write event and fails the checkpoint,
|
||||
// behaves like the last checkpoint is successful.
|
||||
funcWrapper.checkpointFails(2);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -212,13 +215,13 @@ public class StreamWriteFunctionTest {
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
checkWrittenData(tempFile, EXPECTED1);
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.REQUESTED, instant);
|
||||
funcWrapper.checkpointComplete(1);
|
||||
checkWrittenData(tempFile, EXPECTED1);
|
||||
// the coordinator checkpoint commits the inflight instant.
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
checkWrittenData(tempFile, EXPECTED1);
|
||||
@@ -241,15 +244,16 @@ public class StreamWriteFunctionTest {
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
checkWrittenData(tempFile, EXPECTED3, 1);
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
funcWrapper.checkpointComplete(1);
|
||||
|
||||
checkWrittenData(tempFile, EXPECTED3, 1);
|
||||
|
||||
// insert duplicates again
|
||||
for (RowData rowData : TestData.DATA_SET_THREE) {
|
||||
funcWrapper.invoke(rowData);
|
||||
@@ -257,6 +261,10 @@ public class StreamWriteFunctionTest {
|
||||
|
||||
funcWrapper.checkpointFunction(2);
|
||||
|
||||
nextEvent = funcWrapper.getNextEvent();
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
funcWrapper.checkpointComplete(2);
|
||||
|
||||
checkWrittenData(tempFile, EXPECTED3, 1);
|
||||
}
|
||||
|
||||
@@ -306,10 +314,84 @@ public class StreamWriteFunctionTest {
|
||||
checkWrittenData(tempFile, EXPECTED2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertWithMiniBatches() throws Exception {
|
||||
// reset the config option
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setDouble(FlinkOptions.WRITE_BATCH_SIZE, 0.001); // 1Kb batch size
|
||||
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
|
||||
|
||||
// open the function and ingest data
|
||||
funcWrapper.openFunction();
|
||||
// Each record is 424 bytes. so 3 records expect to trigger a mini-batch write
|
||||
for (RowData rowData : TestData.DATA_SET_THREE) {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
|
||||
Map<String, List<HoodieRecord>> dataBuffer = funcWrapper.getDataBuffer();
|
||||
assertThat("Should have 1 data bucket", dataBuffer.size(), is(1));
|
||||
assertThat("2 records expect to flush out as a mini-batch",
|
||||
dataBuffer.values().stream().findFirst().map(List::size).orElse(-1),
|
||||
is(3));
|
||||
|
||||
// this triggers the data write and event send
|
||||
funcWrapper.checkpointFunction(1);
|
||||
assertThat("All data should be flushed out", dataBuffer.size(), is(0));
|
||||
|
||||
final OperatorEvent event1 = funcWrapper.getNextEvent(); // remove the first event first
|
||||
final OperatorEvent event2 = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", event2, instanceOf(BatchWriteSuccessEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event1);
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event2);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
|
||||
String instant = funcWrapper.getWriteClient()
|
||||
.getInflightAndRequestedInstant("COPY_ON_WRITE");
|
||||
|
||||
funcWrapper.checkpointComplete(1);
|
||||
|
||||
Map<String, String> expected = new HashMap<>();
|
||||
expected.put("par1", "[id1,par1,id1,Danny,23,1,par1, "
|
||||
+ "id1,par1,id1,Danny,23,1,par1, "
|
||||
+ "id1,par1,id1,Danny,23,1,par1, "
|
||||
+ "id1,par1,id1,Danny,23,1,par1, "
|
||||
+ "id1,par1,id1,Danny,23,1,par1]");
|
||||
checkWrittenData(tempFile, expected, 1);
|
||||
|
||||
// started a new instant already
|
||||
checkInflightInstant(funcWrapper.getWriteClient());
|
||||
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
|
||||
|
||||
// insert duplicates again
|
||||
for (RowData rowData : TestData.DATA_SET_THREE) {
|
||||
funcWrapper.invoke(rowData);
|
||||
}
|
||||
|
||||
funcWrapper.checkpointFunction(2);
|
||||
|
||||
final OperatorEvent event3 = funcWrapper.getNextEvent(); // remove the first event first
|
||||
final OperatorEvent event4 = funcWrapper.getNextEvent();
|
||||
final OperatorEvent event5 = funcWrapper.getNextEvent();
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event3);
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event4);
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event5);
|
||||
funcWrapper.checkpointComplete(2);
|
||||
|
||||
// Same the original base file content.
|
||||
checkWrittenData(tempFile, expected, 1);
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
private void checkInflightInstant(HoodieFlinkWriteClient writeClient) {
|
||||
final String instant = writeClient.getInflightAndRequestedInstant("COPY_ON_WRITE");
|
||||
assertNotNull(instant);
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
private void checkInstantState(
|
||||
HoodieFlinkWriteClient writeClient,
|
||||
|
||||
@@ -18,8 +18,11 @@
|
||||
|
||||
package org.apache.hudi.operator;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
|
||||
import org.apache.hudi.operator.utils.TestConfigurations;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
@@ -37,6 +40,9 @@ import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@@ -61,6 +67,34 @@ public class StreamWriteOperatorCoordinatorTest {
|
||||
coordinator.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInstantState() {
|
||||
String instant = coordinator.getInstant();
|
||||
assertNotEquals("", instant);
|
||||
|
||||
WriteStatus writeStatus = new WriteStatus(true, 0.1D);
|
||||
writeStatus.setPartitionPath("par1");
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
OperatorEvent event0 =
|
||||
new BatchWriteSuccessEvent(0, instant, Collections.singletonList(writeStatus), true);
|
||||
|
||||
WriteStatus writeStatus1 = new WriteStatus(false, 0.2D);
|
||||
writeStatus1.setPartitionPath("par2");
|
||||
writeStatus1.setStat(new HoodieWriteStat());
|
||||
OperatorEvent event1 =
|
||||
new BatchWriteSuccessEvent(1, instant, Collections.singletonList(writeStatus1), true);
|
||||
coordinator.handleEventFromOperator(0, event0);
|
||||
coordinator.handleEventFromOperator(1, event1);
|
||||
|
||||
coordinator.checkpointComplete(1);
|
||||
String inflight = coordinator.getWriteClient()
|
||||
.getInflightAndRequestedInstant("COPY_ON_WRITE");
|
||||
String lastCompleted = coordinator.getWriteClient().getLastCompletedInstant("COPY_ON_WRITE");
|
||||
assertThat("Instant should be complete", lastCompleted, is(instant));
|
||||
assertNotEquals("", inflight, "Should start a new instant");
|
||||
assertNotEquals(instant, inflight, "Should start a new instant");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTableInitialized() throws IOException {
|
||||
final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
|
||||
@@ -88,14 +122,14 @@ public class StreamWriteOperatorCoordinatorTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckpointInvalid() {
|
||||
public void testCheckpointCompleteWithRetry() {
|
||||
final CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(1, future);
|
||||
String inflightInstant = coordinator.getInFlightInstant();
|
||||
String inflightInstant = coordinator.getInstant();
|
||||
OperatorEvent event = new BatchWriteSuccessEvent(0, inflightInstant, Collections.emptyList());
|
||||
coordinator.handleEventFromOperator(0, event);
|
||||
final CompletableFuture<byte[]> future2 = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(2, future2);
|
||||
assertTrue(future2.isCompletedExceptionally());
|
||||
assertThrows(HoodieException.class,
|
||||
() -> coordinator.checkpointComplete(1),
|
||||
"Try 3 to commit instant");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -38,6 +38,8 @@ 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.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
||||
/**
|
||||
@@ -77,11 +79,11 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
this.conf = conf;
|
||||
// one function
|
||||
this.coordinator = new StreamWriteOperatorCoordinator(conf, 1);
|
||||
this.coordinator.start();
|
||||
this.functionInitializationContext = new MockFunctionInitializationContext();
|
||||
}
|
||||
|
||||
public void openFunction() throws Exception {
|
||||
this.coordinator.start();
|
||||
toHoodieFunction = new RowDataToHoodieFunction<>(TestConfigurations.ROW_TYPE, conf);
|
||||
toHoodieFunction.setRuntimeContext(runtimeContext);
|
||||
toHoodieFunction.open(conf);
|
||||
@@ -123,6 +125,10 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
return this.gateway.getNextEvent();
|
||||
}
|
||||
|
||||
public Map<String, List<HoodieRecord>> getDataBuffer() {
|
||||
return this.writeFunction.getBuffer();
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
public HoodieFlinkWriteClient getWriteClient() {
|
||||
return this.writeFunction.getWriteClient();
|
||||
@@ -141,6 +147,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
|
||||
coordinator.checkpointComplete(checkpointId);
|
||||
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
|
||||
this.writeFunction.notifyCheckpointComplete(checkpointId);
|
||||
}
|
||||
|
||||
public void checkpointFails(long checkpointId) {
|
||||
|
||||
Reference in New Issue
Block a user