1
0

[HUDI-3953]Flink Hudi module should support low-level source and sink api (#5445)

Co-authored-by: jerryyue <jerryyue@didiglobal.com>
This commit is contained in:
JerryYue-M
2022-07-02 08:38:46 +08:00
committed by GitHub
parent 62a0c962ac
commit bdf73b2650
2 changed files with 397 additions and 0 deletions

View File

@@ -26,9 +26,11 @@ import org.apache.hudi.sink.transform.ChainedTransformer;
import org.apache.hudi.sink.transform.Transformer;
import org.apache.hudi.sink.utils.Pipelines;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.HoodiePipeline;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.utils.TestConfigurations;
import org.apache.hudi.utils.TestData;
import org.apache.hudi.utils.TestUtils;
import org.apache.hudi.utils.source.ContinuousFileSource;
import org.apache.flink.api.common.JobStatus;
@@ -57,6 +59,7 @@ import org.junit.jupiter.params.provider.ValueSource;
import java.io.File;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
@@ -326,4 +329,127 @@ public class ITTestDataStreamWrite extends TestLogger {
TestData.checkWrittenFullData(tempFile, expected);
}
public void execute(StreamExecutionEnvironment execEnv, boolean isMor, String jobName) throws Exception {
JobClient client = execEnv.executeAsync(jobName);
if (isMor) {
if (client.getJobStatus().get() != JobStatus.FAILED) {
try {
TimeUnit.SECONDS.sleep(20); // wait long enough for the compaction to finish
client.cancel();
} catch (Throwable var1) {
// ignored
}
}
} else {
// wait for the streaming job to finish
client.getJobExecutionResult().get();
}
}
@Test
public void testHoodiePipelineBuilderSource() throws Exception {
//create a StreamExecutionEnvironment instance.
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
execEnv.getConfig().disableObjectReuse();
execEnv.setParallelism(1);
// set up checkpoint interval
execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE);
execEnv.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setString(FlinkOptions.TABLE_NAME, "t1");
conf.setString(FlinkOptions.TABLE_TYPE, "MERGE_ON_READ");
// write 3 batches of data set
TestData.writeData(TestData.dataSetInsert(1, 2), conf);
TestData.writeData(TestData.dataSetInsert(3, 4), conf);
TestData.writeData(TestData.dataSetInsert(5, 6), conf);
String latestCommit = TestUtils.getLastCompleteInstant(tempFile.getAbsolutePath());
Map<String, String> options = new HashMap<>();
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
options.put(FlinkOptions.READ_START_COMMIT.key(), latestCommit);
//read a hoodie table use low-level source api.
HoodiePipeline.Builder builder = HoodiePipeline.builder("test_source")
.column("uuid string not null")
.column("name string")
.column("age int")
.column("`ts` timestamp(3)")
.column("`partition` string")
.pk("uuid")
.partition("partition")
.options(options);
DataStream<RowData> rowDataDataStream = builder.source(execEnv);
List<RowData> result = new ArrayList<>();
rowDataDataStream.executeAndCollect().forEachRemaining(result::add);
TimeUnit.SECONDS.sleep(2);//sleep 2 second for collect data
TestData.assertRowDataEquals(result, TestData.dataSetInsert(5, 6));
}
@Test
public void testHoodiePipelineBuilderSink() throws Exception {
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
Map<String, String> options = new HashMap<>();
execEnv.getConfig().disableObjectReuse();
execEnv.setParallelism(4);
// set up checkpoint interval
execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE);
execEnv.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
options.put(FlinkOptions.INDEX_TYPE.key(), "FLINK_STATE");
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
options.put(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS.key(), "4");
options.put("table.type", HoodieTableType.MERGE_ON_READ.name());
options.put(FlinkOptions.INDEX_KEY_FIELD.key(), "id");
options.put(FlinkOptions.COMPACTION_DELTA_COMMITS.key(), "1");
options.put(FlinkOptions.TABLE_TYPE.key(), HoodieTableType.MERGE_ON_READ.name());
options.put(FlinkOptions.SOURCE_AVRO_SCHEMA_PATH.key(), Objects.requireNonNull(Thread.currentThread().getContextClassLoader().getResource("test_read_schema.avsc")).toString());
Configuration conf = Configuration.fromMap(options);
// Read from file source
RowType rowType =
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
.getLogicalType();
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType,
InternalTypeInfo.of(rowType),
false,
true,
TimestampFormat.ISO_8601
);
String sourcePath = Objects.requireNonNull(Thread.currentThread()
.getContextClassLoader().getResource("test_source.data")).toString();
TextInputFormat format = new TextInputFormat(new Path(sourcePath));
format.setFilesFilter(FilePathFilter.createDefaultFilter());
TypeInformation<String> typeInfo = BasicTypeInfo.STRING_TYPE_INFO;
format.setCharsetName("UTF-8");
DataStream dataStream = execEnv
// use PROCESS_CONTINUOUSLY mode to trigger checkpoint
.readFile(format, sourcePath, FileProcessingMode.PROCESS_CONTINUOUSLY, 1000, typeInfo)
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)))
.setParallelism(1);
//sink to hoodie table use low-level sink api.
HoodiePipeline.Builder builder = HoodiePipeline.builder("test_sink")
.column("uuid string not null")
.column("name string")
.column("age int")
.column("`ts` timestamp(3)")
.column("`partition` string")
.pk("uuid")
.partition("partition")
.options(options);
builder.sink(dataStream, false);
execute(execEnv, true, "Api_Sink_Test");
TestData.checkWrittenFullData(tempFile, EXPECTED);
}
}