1
0

[HUDI-2052] Support load logFile in BootstrapFunction (#3134)

Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>
This commit is contained in:
yuzhaojing
2021-06-30 20:37:00 +08:00
committed by GitHub
parent 94f0f40fec
commit 07e93de8b4
4 changed files with 104 additions and 36 deletions

View File

@@ -571,7 +571,7 @@ public class TestWriteCopyOnWrite {
checkWrittenData(tempFile, expected, 1);
}
Map<String, String> getMiniBatchExpected() {
protected Map<String, String> getMiniBatchExpected() {
Map<String, String> expected = new HashMap<>();
// the last 2 lines are merged
expected.put("par1", "["
@@ -581,6 +581,10 @@ public class TestWriteCopyOnWrite {
return expected;
}
protected Map<String, String> getExpectedBeforeCheckpointComplete() {
return EXPECTED2;
}
@Test
public void testIndexStateBootstrap() throws Exception {
// open the function and ingest data
@@ -637,7 +641,9 @@ public class TestWriteCopyOnWrite {
nextEvent = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", nextEvent, instanceOf(WriteMetadataEvent.class));
checkWrittenData(tempFile, EXPECTED2);
Map<String, String> expected = getExpectedBeforeCheckpointComplete();
checkWrittenData(tempFile, expected);
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");

View File

@@ -37,8 +37,6 @@ import org.apache.avro.Schema;
import org.apache.flink.configuration.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import java.io.File;
import java.util.Comparator;
@@ -83,13 +81,12 @@ public class TestWriteMergeOnRead extends TestWriteCopyOnWrite {
TestData.checkWrittenDataMOR(fs, latestInstant, baseFile, expected, partitions, schema);
}
@Disabled
@Test
public void testIndexStateBootstrap() {
// Ignore the index bootstrap because we only support parquet load now.
@Override
protected Map<String, String> getExpectedBeforeCheckpointComplete() {
return EXPECTED1;
}
Map<String, String> getMiniBatchExpected() {
protected Map<String, String> getMiniBatchExpected() {
Map<String, String> expected = new HashMap<>();
// MOR mode merges the messages with the same key.
expected.put("par1", "[id1,par1,id1,Danny,23,1,par1]");

View File

@@ -45,7 +45,7 @@ public class TestWriteMergeOnReadWithCompact extends TestWriteCopyOnWrite {
// Ignore the index bootstrap because we only support parquet load now.
}
Map<String, String> getMiniBatchExpected() {
protected Map<String, String> getMiniBatchExpected() {
Map<String, String> expected = new HashMap<>();
// MOR mode merges the messages with the same key.
expected.put("par1", "[id1,par1,id1,Danny,23,1,par1]");