[HUDI-1692] Bounded source for stream writer (#2674)
Supports bounded source such as VALUES for stream mode writer.
This commit is contained in:
@@ -58,7 +58,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
@BeforeEach
|
||||
public void before() throws Exception {
|
||||
coordinator = new StreamWriteOperatorCoordinator(
|
||||
TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()), 2, false);
|
||||
TestConfigurations.getDefaultConf(tempFile.getAbsolutePath()), 2);
|
||||
coordinator.start();
|
||||
}
|
||||
|
||||
@@ -75,14 +75,22 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
WriteStatus writeStatus = new WriteStatus(true, 0.1D);
|
||||
writeStatus.setPartitionPath("par1");
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
OperatorEvent event0 =
|
||||
new BatchWriteSuccessEvent(0, instant, Collections.singletonList(writeStatus), true);
|
||||
OperatorEvent event0 = BatchWriteSuccessEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime(instant)
|
||||
.writeStatus(Collections.singletonList(writeStatus))
|
||||
.isLastBatch(true)
|
||||
.build();
|
||||
|
||||
WriteStatus writeStatus1 = new WriteStatus(false, 0.2D);
|
||||
writeStatus1.setPartitionPath("par2");
|
||||
writeStatus1.setStat(new HoodieWriteStat());
|
||||
OperatorEvent event1 =
|
||||
new BatchWriteSuccessEvent(1, instant, Collections.singletonList(writeStatus1), true);
|
||||
OperatorEvent event1 = BatchWriteSuccessEvent.builder()
|
||||
.taskID(1)
|
||||
.instantTime(instant)
|
||||
.writeStatus(Collections.singletonList(writeStatus1))
|
||||
.isLastBatch(true)
|
||||
.build();
|
||||
coordinator.handleEventFromOperator(0, event0);
|
||||
coordinator.handleEventFromOperator(1, event1);
|
||||
|
||||
@@ -115,7 +123,11 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
public void testReceiveInvalidEvent() {
|
||||
CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(1, future);
|
||||
OperatorEvent event = new BatchWriteSuccessEvent(0, "abc", Collections.emptyList());
|
||||
OperatorEvent event = BatchWriteSuccessEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime("abc")
|
||||
.writeStatus(Collections.emptyList())
|
||||
.build();
|
||||
assertThrows(IllegalStateException.class,
|
||||
() -> coordinator.handleEventFromOperator(0, event),
|
||||
"Receive an unexpected event for instant abc from task 0");
|
||||
@@ -126,7 +138,11 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
final CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(1, future);
|
||||
String inflightInstant = coordinator.getInstant();
|
||||
OperatorEvent event = new BatchWriteSuccessEvent(0, inflightInstant, Collections.emptyList());
|
||||
OperatorEvent event = BatchWriteSuccessEvent.builder()
|
||||
.taskID(0)
|
||||
.instantTime(inflightInstant)
|
||||
.writeStatus(Collections.emptyList())
|
||||
.build();
|
||||
coordinator.handleEventFromOperator(0, event);
|
||||
assertThrows(HoodieException.class,
|
||||
() -> coordinator.checkpointComplete(1),
|
||||
|
||||
@@ -84,7 +84,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
this.gateway = new MockOperatorEventGateway();
|
||||
this.conf = conf;
|
||||
// one function
|
||||
this.coordinator = new StreamWriteOperatorCoordinator(conf, 1, false);
|
||||
this.coordinator = new StreamWriteOperatorCoordinator(conf, 1);
|
||||
this.functionInitializationContext = new MockFunctionInitializationContext();
|
||||
this.compactFunctionWrapper = new CompactFunctionWrapper(this.conf);
|
||||
}
|
||||
|
||||
@@ -36,6 +36,8 @@ import org.apache.flink.util.CollectionUtil;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Collection;
|
||||
@@ -149,12 +151,14 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testBatchWriteAndRead() {
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = ExecMode.class)
|
||||
void testWriteAndRead(ExecMode execMode) {
|
||||
TableEnvironment tableEnv = execMode == ExecMode.BATCH ? batchTableEnv : streamTableEnv;
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
|
||||
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
|
||||
batchTableEnv.executeSql(hoodieTableDDL);
|
||||
tableEnv.executeSql(hoodieTableDDL);
|
||||
String insertInto = "insert into t1 values\n"
|
||||
+ "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1'),\n"
|
||||
+ "('id2','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
|
||||
@@ -165,13 +169,20 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
+ "('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07','par4'),\n"
|
||||
+ "('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08','par4')";
|
||||
|
||||
execInsertSql(batchTableEnv, insertInto);
|
||||
execInsertSql(tableEnv, insertInto);
|
||||
|
||||
List<Row> rows = CollectionUtil.iterableToList(
|
||||
() -> batchTableEnv.sqlQuery("select * from t1").execute().collect());
|
||||
() -> tableEnv.sqlQuery("select * from t1").execute().collect());
|
||||
assertRowsEquals(rows, TestData.DATA_SET_SOURCE_INSERT);
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
private enum ExecMode {
|
||||
BATCH, STREAM
|
||||
}
|
||||
|
||||
private void execInsertSql(TableEnvironment tEnv, String insert) {
|
||||
TableResult tableResult = tEnv.executeSql(insert);
|
||||
// wait to finish
|
||||
|
||||
Reference in New Issue
Block a user