[HUDI-2084] Resend the uncommitted write metadata when start up (#3168)
Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>
This commit is contained in:
@@ -200,7 +200,7 @@ public class StreamWriteITCase extends TestLogger {
|
||||
conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
|
||||
|
||||
// create metaClient
|
||||
HoodieTableMetaClient metaClient = CompactionUtil.createMetaClient(conf);
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf);
|
||||
|
||||
// set the table name
|
||||
conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName());
|
||||
|
||||
@@ -24,7 +24,7 @@ import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.sink.event.BatchWriteSuccessEvent;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.utils.MockCoordinatorExecutor;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
@@ -70,6 +70,23 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()), context);
|
||||
coordinator.start();
|
||||
coordinator.setExecutor(new MockCoordinatorExecutor(context));
|
||||
|
||||
final WriteMetadataEvent event0 = WriteMetadataEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime("")
|
||||
.writeStatus(Collections.emptyList())
|
||||
.isBootstrap(true)
|
||||
.build();
|
||||
|
||||
final WriteMetadataEvent event1 = WriteMetadataEvent.builder()
|
||||
.taskID(1)
|
||||
.instantTime("")
|
||||
.writeStatus(Collections.emptyList())
|
||||
.isBootstrap(true)
|
||||
.build();
|
||||
|
||||
coordinator.handleEventFromOperator(0, event0);
|
||||
coordinator.handleEventFromOperator(1, event1);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
@@ -85,7 +102,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
WriteStatus writeStatus = new WriteStatus(true, 0.1D);
|
||||
writeStatus.setPartitionPath("par1");
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
OperatorEvent event0 = BatchWriteSuccessEvent.builder()
|
||||
OperatorEvent event0 = WriteMetadataEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime(instant)
|
||||
.writeStatus(Collections.singletonList(writeStatus))
|
||||
@@ -95,7 +112,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
WriteStatus writeStatus1 = new WriteStatus(false, 0.2D);
|
||||
writeStatus1.setPartitionPath("par2");
|
||||
writeStatus1.setStat(new HoodieWriteStat());
|
||||
OperatorEvent event1 = BatchWriteSuccessEvent.builder()
|
||||
OperatorEvent event1 = WriteMetadataEvent.builder()
|
||||
.taskID(1)
|
||||
.instantTime(instant)
|
||||
.writeStatus(Collections.singletonList(writeStatus1))
|
||||
@@ -132,7 +149,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
public void testReceiveInvalidEvent() {
|
||||
CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(1, future);
|
||||
OperatorEvent event = BatchWriteSuccessEvent.builder()
|
||||
OperatorEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime("abc")
|
||||
.writeStatus(Collections.emptyList())
|
||||
@@ -147,7 +164,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
final CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(1, future);
|
||||
String instant = coordinator.getInstant();
|
||||
OperatorEvent event = BatchWriteSuccessEvent.builder()
|
||||
OperatorEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime(instant)
|
||||
.writeStatus(Collections.emptyList())
|
||||
@@ -163,7 +180,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
WriteStatus writeStatus1 = new WriteStatus(false, 0.2D);
|
||||
writeStatus1.setPartitionPath("par2");
|
||||
writeStatus1.setStat(new HoodieWriteStat());
|
||||
OperatorEvent event1 = BatchWriteSuccessEvent.builder()
|
||||
OperatorEvent event1 = WriteMetadataEvent.builder()
|
||||
.taskID(1)
|
||||
.instantTime(instant)
|
||||
.writeStatus(Collections.singletonList(writeStatus1))
|
||||
@@ -186,20 +203,30 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
coordinator.start();
|
||||
coordinator.setExecutor(new MockCoordinatorExecutor(context));
|
||||
|
||||
final WriteMetadataEvent event0 = WriteMetadataEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime("")
|
||||
.writeStatus(Collections.emptyList())
|
||||
.isBootstrap(true)
|
||||
.build();
|
||||
|
||||
coordinator.handleEventFromOperator(0, event0);
|
||||
|
||||
String instant = coordinator.getInstant();
|
||||
assertNotEquals("", instant);
|
||||
|
||||
WriteStatus writeStatus = new WriteStatus(true, 0.1D);
|
||||
writeStatus.setPartitionPath("par1");
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
OperatorEvent event0 = BatchWriteSuccessEvent.builder()
|
||||
|
||||
OperatorEvent event1 = WriteMetadataEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime(instant)
|
||||
.writeStatus(Collections.singletonList(writeStatus))
|
||||
.isLastBatch(true)
|
||||
.build();
|
||||
|
||||
coordinator.handleEventFromOperator(0, event0);
|
||||
coordinator.handleEventFromOperator(0, event1);
|
||||
|
||||
// never throw for hive synchronization now
|
||||
assertDoesNotThrow(() -> coordinator.notifyCheckpointComplete(1));
|
||||
|
||||
@@ -28,7 +28,7 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.event.BatchWriteSuccessEvent;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.utils.StreamWriteFunctionWrapper;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
@@ -135,8 +135,8 @@ public class TestWriteCopyOnWrite {
|
||||
String instant = funcWrapper.getWriteClient().getLastPendingInstant(getTableType());
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
List<WriteStatus> writeStatuses = ((BatchWriteSuccessEvent) nextEvent).getWriteStatuses();
|
||||
MatcherAssert.assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
List<WriteStatus> writeStatuses = ((WriteMetadataEvent) nextEvent).getWriteStatuses();
|
||||
assertNotNull(writeStatuses);
|
||||
MatcherAssert.assertThat(writeStatuses.size(), is(4)); // write 4 partition files
|
||||
assertThat(writeStatuses.stream()
|
||||
@@ -162,8 +162,8 @@ public class TestWriteCopyOnWrite {
|
||||
assertNotEquals(instant, instant2);
|
||||
|
||||
final OperatorEvent nextEvent2 = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent2, instanceOf(BatchWriteSuccessEvent.class));
|
||||
List<WriteStatus> writeStatuses2 = ((BatchWriteSuccessEvent) nextEvent2).getWriteStatuses();
|
||||
assertThat("The operator expect to send an event", nextEvent2, instanceOf(WriteMetadataEvent.class));
|
||||
List<WriteStatus> writeStatuses2 = ((WriteMetadataEvent) nextEvent2).getWriteStatuses();
|
||||
assertNotNull(writeStatuses2);
|
||||
assertThat(writeStatuses2.size(), is(0)); // write empty statuses
|
||||
|
||||
@@ -191,8 +191,8 @@ public class TestWriteCopyOnWrite {
|
||||
assertNotNull(instant);
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
List<WriteStatus> writeStatuses = ((BatchWriteSuccessEvent) nextEvent).getWriteStatuses();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
List<WriteStatus> writeStatuses = ((WriteMetadataEvent) nextEvent).getWriteStatuses();
|
||||
assertNotNull(writeStatuses);
|
||||
assertThat(writeStatuses.size(), is(0)); // no data write
|
||||
|
||||
@@ -210,7 +210,9 @@ public class TestWriteCopyOnWrite {
|
||||
}
|
||||
|
||||
// this returns early because there is no inflight instant
|
||||
funcWrapper.checkpointFunction(2);
|
||||
assertThrows(HoodieException.class,
|
||||
() -> funcWrapper.checkpointFunction(2),
|
||||
"Timeout(0ms) while waiting for");
|
||||
// do not sent the write event and fails the checkpoint,
|
||||
// behaves like the last checkpoint is successful.
|
||||
funcWrapper.checkpointFails(2);
|
||||
@@ -232,7 +234,7 @@ public class TestWriteCopyOnWrite {
|
||||
.getLastPendingInstant(getTableType());
|
||||
|
||||
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
@@ -262,7 +264,7 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
@@ -298,7 +300,7 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
@@ -318,7 +320,7 @@ public class TestWriteCopyOnWrite {
|
||||
.getLastPendingInstant(getTableType());
|
||||
|
||||
nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
@@ -343,7 +345,7 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
@@ -363,7 +365,7 @@ public class TestWriteCopyOnWrite {
|
||||
.getLastPendingInstant(getTableType());
|
||||
|
||||
nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
@@ -408,7 +410,7 @@ public class TestWriteCopyOnWrite {
|
||||
|
||||
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));
|
||||
assertThat("The operator expect to send an event", event2, instanceOf(WriteMetadataEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event1);
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event2);
|
||||
@@ -470,7 +472,7 @@ public class TestWriteCopyOnWrite {
|
||||
|
||||
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));
|
||||
assertThat("The operator expect to send an event", event2, instanceOf(WriteMetadataEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event1);
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event2);
|
||||
@@ -534,7 +536,7 @@ public class TestWriteCopyOnWrite {
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
final OperatorEvent event = funcWrapper.getNextEvent(); // remove the first event first
|
||||
assertThat("The operator expect to send an event", event, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", event, instanceOf(WriteMetadataEvent.class));
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event);
|
||||
}
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
@@ -592,7 +594,7 @@ public class TestWriteCopyOnWrite {
|
||||
funcWrapper.checkpointFunction(1);
|
||||
|
||||
OperatorEvent nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
|
||||
@@ -634,7 +636,7 @@ public class TestWriteCopyOnWrite {
|
||||
.getLastPendingInstant(getTableType());
|
||||
|
||||
nextEvent = funcWrapper.getNextEvent();
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
|
||||
checkWrittenData(tempFile, EXPECTED2);
|
||||
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
|
||||
@@ -673,7 +675,7 @@ public class TestWriteCopyOnWrite {
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
final OperatorEvent event = funcWrapper.getNextEvent(); // remove the first event first
|
||||
assertThat("The operator expect to send an event", event, instanceOf(BatchWriteSuccessEvent.class));
|
||||
assertThat("The operator expect to send an event", event, instanceOf(WriteMetadataEvent.class));
|
||||
funcWrapper.getCoordinator().handleEventFromOperator(0, event);
|
||||
}
|
||||
|
||||
|
||||
@@ -27,7 +27,7 @@ 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.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
|
||||
import org.apache.hudi.sink.partitioner.BucketAssignOperator;
|
||||
import org.apache.hudi.sink.transform.RowDataToHoodieFunction;
|
||||
@@ -70,15 +70,25 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
private final StreamWriteOperatorCoordinator coordinator;
|
||||
private final MockFunctionInitializationContext functionInitializationContext;
|
||||
|
||||
/** Function that converts row data to HoodieRecord. */
|
||||
/**
|
||||
* Function that converts row data to HoodieRecord.
|
||||
*/
|
||||
private RowDataToHoodieFunction<RowData, HoodieRecord<?>> toHoodieFunction;
|
||||
/** Function that load index in state. */
|
||||
/**
|
||||
* Function that load index in state.
|
||||
*/
|
||||
private BootstrapFunction<HoodieRecord<?>, HoodieRecord<?>> bootstrapFunction;
|
||||
/** Function that assigns bucket ID. */
|
||||
/**
|
||||
* Function that assigns bucket ID.
|
||||
*/
|
||||
private BucketAssignFunction<String, HoodieRecord<?>, HoodieRecord<?>> bucketAssignerFunction;
|
||||
/** BucketAssignOperator context. **/
|
||||
/**
|
||||
* BucketAssignOperator context.
|
||||
**/
|
||||
private MockBucketAssignOperatorContext bucketAssignOperatorContext;
|
||||
/** Stream write function. */
|
||||
/**
|
||||
* Stream write function.
|
||||
*/
|
||||
private StreamWriteFunction<Object, HoodieRecord<?>, Object> writeFunction;
|
||||
|
||||
private CompactFunctionWrapper compactFunctionWrapper;
|
||||
@@ -133,8 +143,12 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
writeFunction = new StreamWriteFunction<>(conf);
|
||||
writeFunction.setRuntimeContext(runtimeContext);
|
||||
writeFunction.setOperatorEventGateway(gateway);
|
||||
writeFunction.initializeState(this.functionInitializationContext);
|
||||
writeFunction.open(conf);
|
||||
|
||||
// handle the bootstrap event
|
||||
coordinator.handleEventFromOperator(0, getNextEvent());
|
||||
|
||||
if (asyncCompaction) {
|
||||
compactFunctionWrapper.openFunction();
|
||||
}
|
||||
@@ -184,7 +198,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
writeFunction.processElement(hoodieRecords[0], null, null);
|
||||
}
|
||||
|
||||
public BatchWriteSuccessEvent[] getEventBuffer() {
|
||||
public WriteMetadataEvent[] getEventBuffer() {
|
||||
return this.coordinator.getEventBuffer();
|
||||
}
|
||||
|
||||
@@ -201,7 +215,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
return this.writeFunction.getWriteClient();
|
||||
}
|
||||
|
||||
public void checkpointFunction(long checkpointId) {
|
||||
public void checkpointFunction(long checkpointId) throws Exception {
|
||||
// checkpoint the coordinator first
|
||||
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
|
||||
bucketAssignerFunction.snapshotState(null);
|
||||
|
||||
Reference in New Issue
Block a user