1
0

[HUDI-1557] Make Flink write pipeline write task scalable (#2506)

This is the #step 2 of RFC-24:
https://cwiki.apache.org/confluence/display/HUDI/RFC+-+24%3A+Hoodie+Flink+Writer+Proposal

This PR introduce a BucketAssigner that assigns bucket ID (partition
path & fileID) for each stream record.

There is no need to look up index and partition the records anymore in
the following pipeline for these records,
we actually decide the write target location before the write and each
record computes its location when the BucketAssigner receives it, thus,
the indexing is with streaming style.

Computing locations for a batch of records all at a time is resource
consuming so a pressure to the engine,
we should avoid that in streaming system.
This commit is contained in:
Danny Chan
2021-02-06 22:03:52 +08:00
committed by GitHub
parent 291f92069e
commit 4c5b6923cc
30 changed files with 1435 additions and 393 deletions

View File

@@ -27,6 +27,7 @@ import org.apache.hudi.operator.utils.StreamWriteFunctionWrapper;
import org.apache.hudi.operator.utils.TestConfigurations;
import org.apache.hudi.operator.utils.TestData;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.table.data.RowData;
import org.hamcrest.MatcherAssert;
@@ -56,24 +57,26 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
*/
public class StreamWriteFunctionTest {
private static final Map<String, String> EXPECTED = new HashMap<>();
static {
EXPECTED.put("par1", "[id1,par1,id1,Danny,23,1,par1, id2,par1,id2,Stephen,33,2,par1]");
EXPECTED.put("par2", "[id3,par2,id3,Julian,53,3,par2, id4,par2,id4,Fabian,31,4,par2]");
EXPECTED.put("par3", "[id5,par3,id5,Sophia,18,5,par3, id6,par3,id6,Emma,20,6,par3]");
EXPECTED.put("par4", "[id7,par4,id7,Bob,44,7,par4, id8,par4,id8,Han,56,8,par4]");
}
private static final Map<String, String> EXPECTED1 = new HashMap<>();
private static final Map<String, String> EXPECTED2 = new HashMap<>();
private static final Map<String, String> EXPECTED3 = new HashMap<>();
static {
EXPECTED1.put("par1", "[id1,par1,id1,Danny,23,1,par1, id2,par1,id2,Stephen,33,2,par1]");
EXPECTED1.put("par2", "[id3,par2,id3,Julian,53,3,par2, id4,par2,id4,Fabian,31,4,par2]");
EXPECTED1.put("par3", "[id5,par3,id5,Sophia,18,5,par3, id6,par3,id6,Emma,20,6,par3]");
EXPECTED1.put("par4", "[id7,par4,id7,Bob,44,7,par4, id8,par4,id8,Han,56,8,par4]");
EXPECTED2.put("par1", "[id1,par1,id1,Danny,24,1,par1, id2,par1,id2,Stephen,34,2,par1]");
EXPECTED2.put("par2", "[id3,par2,id3,Julian,54,3,par2, id4,par2,id4,Fabian,32,4,par2]");
EXPECTED2.put("par3", "[id5,par3,id5,Sophia,18,5,par3, id6,par3,id6,Emma,20,6,par3, "
+ "id9,par3,id9,Jane,19,6,par3]");
EXPECTED2.put("par4", "[id10,par4,id10,Ella,38,7,par4, id11,par4,id11,Phoebe,52,8,par4, "
+ "id7,par4,id7,Bob,44,7,par4, id8,par4,id8,Han,56,8,par4]");
EXPECTED3.put("par1", "[id1,par1,id1,Danny,23,1,par1]");
}
private StreamWriteFunctionWrapper<RowData> funcWrapper;
@@ -83,9 +86,7 @@ public class StreamWriteFunctionTest {
@BeforeEach
public void before() throws Exception {
this.funcWrapper = new StreamWriteFunctionWrapper<>(
tempFile.getAbsolutePath(),
TestConfigurations.SERIALIZER);
this.funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath());
}
@AfterEach
@@ -211,7 +212,7 @@ public class StreamWriteFunctionTest {
final OperatorEvent nextEvent = funcWrapper.getNextEvent();
assertThat("The operator expect to send an event", nextEvent, instanceOf(BatchWriteSuccessEvent.class));
checkWrittenData(tempFile, EXPECTED);
checkWrittenData(tempFile, EXPECTED1);
funcWrapper.getCoordinator().handleEventFromOperator(0, nextEvent);
assertNotNull(funcWrapper.getEventBuffer()[0], "The coordinator missed the event");
@@ -220,7 +221,43 @@ public class StreamWriteFunctionTest {
funcWrapper.checkpointComplete(1);
// the coordinator checkpoint commits the inflight instant.
checkInstantState(funcWrapper.getWriteClient(), HoodieInstant.State.COMPLETED, instant);
checkWrittenData(tempFile, EXPECTED);
checkWrittenData(tempFile, EXPECTED1);
}
@Test
public void testInsertDuplicates() throws Exception {
// reset the config option
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
conf.setBoolean(FlinkOptions.INSERT_DROP_DUPS, true);
funcWrapper = new StreamWriteFunctionWrapper<>(tempFile.getAbsolutePath(), conf);
// open the function and ingest data
funcWrapper.openFunction();
for (RowData rowData : TestData.DATA_SET_THREE) {
funcWrapper.invoke(rowData);
}
assertEmptyDataFiles();
// this triggers the data write and event send
funcWrapper.checkpointFunction(1);
final 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);
// insert duplicates again
for (RowData rowData : TestData.DATA_SET_THREE) {
funcWrapper.invoke(rowData);
}
funcWrapper.checkpointFunction(2);
checkWrittenData(tempFile, EXPECTED3, 1);
}
@Test
@@ -248,7 +285,7 @@ public class StreamWriteFunctionTest {
funcWrapper.invoke(rowData);
}
// the data is not flushed yet
checkWrittenData(tempFile, EXPECTED);
checkWrittenData(tempFile, EXPECTED1);
// this triggers the data write and event send
funcWrapper.checkpointFunction(2);

View File

@@ -18,16 +18,24 @@
package org.apache.hudi.operator;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.operator.partitioner.BucketAssignFunction;
import org.apache.hudi.operator.transform.RowDataToHoodieFunction;
import org.apache.hudi.operator.utils.TestConfigurations;
import org.apache.hudi.operator.utils.TestData;
import org.apache.hudi.sink.CommitSink;
import org.apache.hudi.streamer.FlinkStreamerConfig;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.common.io.FilePathFilter;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeHint;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.io.TextInputFormat;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.core.fs.Path;
@@ -37,7 +45,7 @@ import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
import org.apache.flink.table.data.RowData;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.util.TestLogger;
@@ -47,6 +55,7 @@ import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
@@ -74,19 +83,15 @@ public class StreamWriteITCase extends TestLogger {
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
execEnv.getConfig().disableObjectReuse();
execEnv.setParallelism(4);
// 1 second a time
execEnv.enableCheckpointing(2000, CheckpointingMode.EXACTLY_ONCE);
// set up checkpoint interval
execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE);
// Read from kafka source
// Read from file source
RowType rowType =
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
.getLogicalType();
StreamWriteOperatorFactory<RowData> operatorFactory =
new StreamWriteOperatorFactory<>(rowType, conf, 4);
int partitionFieldIndex = rowType.getFieldIndex(conf.getString(FlinkOptions.PARTITION_PATH_FIELD));
final RowData.FieldGetter partitionFieldGetter =
RowData.createFieldGetter(rowType.getTypeAt(partitionFieldIndex), partitionFieldIndex);
StreamWriteOperatorFactory<HoodieRecord> operatorFactory =
new StreamWriteOperatorFactory<>(conf, 4);
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType,
@@ -107,17 +112,103 @@ public class StreamWriteITCase extends TestLogger {
// 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(4)
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class))
// Key-by partition path, to avoid multiple subtasks write to a partition at the same time
.keyBy(partitionFieldGetter::getFieldOrNull)
.keyBy(HoodieRecord::getPartitionPath)
.transform(
"bucket_assigner",
TypeInformation.of(HoodieRecord.class),
new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
.uid("uid_bucket_assigner")
// shuffle by fileId(bucket id)
.keyBy(record -> record.getCurrentLocation().getFileId())
.transform("hoodie_stream_write", null, operatorFactory)
.uid("uid_hoodie_stream_write")
.setParallelism(4);
.uid("uid_hoodie_stream_write");
execEnv.addOperator(dataStream.getTransformation());
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
if (client.getJobStatus().get() != JobStatus.FAILED) {
try {
TimeUnit.SECONDS.sleep(10);
TimeUnit.SECONDS.sleep(8);
client.cancel();
} catch (Throwable var1) {
// ignored
}
}
TestData.checkWrittenData(tempFile, EXPECTED);
}
@Test
public void testWriteToHoodieLegacy() throws Exception {
FlinkStreamerConfig streamerConf = TestConfigurations.getDefaultStreamerConf(tempFile.getAbsolutePath());
Configuration conf = FlinkOptions.fromStreamerConfig(streamerConf);
StreamerUtil.initTableIfNotExists(conf);
StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
execEnv.getConfig().disableObjectReuse();
execEnv.setParallelism(4);
// set up checkpoint interval
execEnv.enableCheckpointing(4000, CheckpointingMode.EXACTLY_ONCE);
execEnv.getConfig().setGlobalJobParameters(streamerConf);
// Read from file source
RowType rowType =
(RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf))
.getLogicalType();
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType,
new RowDataTypeInfo(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");
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(4)
.map(new RowDataToHoodieFunction<>(rowType, conf), TypeInformation.of(HoodieRecord.class))
.transform(InstantGenerateOperator.NAME, TypeInformation.of(HoodieRecord.class), new InstantGenerateOperator())
.name("instant_generator")
.uid("instant_generator_id")
// Keyby partition path, to avoid multiple subtasks writing to a partition at the same time
.keyBy(HoodieRecord::getPartitionPath)
// use the bucket assigner to generate bucket IDs
.transform(
"bucket_assigner",
TypeInformation.of(HoodieRecord.class),
new KeyedProcessOperator<>(new BucketAssignFunction<>(conf)))
.uid("uid_bucket_assigner")
// shuffle by fileId(bucket id)
.keyBy(record -> record.getCurrentLocation().getFileId())
// write operator, where the write operation really happens
.transform(KeyedWriteProcessOperator.NAME, TypeInformation.of(new TypeHint<Tuple3<String, List<WriteStatus>, Integer>>() {
}), new KeyedWriteProcessOperator(new KeyedWriteProcessFunction()))
.name("write_process")
.uid("write_process_uid")
.setParallelism(4)
// Commit can only be executed once, so make it one parallelism
.addSink(new CommitSink())
.name("commit_sink")
.uid("commit_sink_uid")
.setParallelism(1);
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
if (client.getJobStatus().get() != JobStatus.FAILED) {
try {
TimeUnit.SECONDS.sleep(8);
client.cancel();
} catch (Throwable var1) {
// ignored

View File

@@ -0,0 +1,235 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.operator.partitioner;
import org.apache.hudi.client.FlinkTaskContextSupplier;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.operator.utils.TestConfigurations;
import org.apache.hudi.table.action.commit.BucketInfo;
import org.apache.hudi.table.action.commit.BucketType;
import org.apache.hudi.table.action.commit.SmallFile;
import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.configuration.Configuration;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
/**
* Test cases for {@link BucketAssigner}.
*/
public class TestBucketAssigner {
private HoodieWriteConfig writeConfig;
private HoodieFlinkEngineContext context;
@TempDir
File tempFile;
@BeforeEach
public void before() throws IOException {
final String basePath = tempFile.getAbsolutePath();
final Configuration conf = TestConfigurations.getDefaultConf(basePath);
writeConfig = StreamerUtil.getHoodieClientConfig(conf);
context = new HoodieFlinkEngineContext(
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
new FlinkTaskContextSupplier(null));
StreamerUtil.initTableIfNotExists(conf);
}
@Test
public void testAddUpdate() {
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig);
BucketInfo bucketInfo = mockBucketAssigner.addUpdate("par1", "file_id_0");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "file_id_0");
mockBucketAssigner.addUpdate("par1", "file_id_0");
bucketInfo = mockBucketAssigner.addUpdate("par1", "file_id_0");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "file_id_0");
mockBucketAssigner.addUpdate("par1", "file_id_1");
bucketInfo = mockBucketAssigner.addUpdate("par1", "file_id_1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "file_id_1");
bucketInfo = mockBucketAssigner.addUpdate("par2", "file_id_0");
assertBucketEquals(bucketInfo, "par2", BucketType.UPDATE, "file_id_0");
bucketInfo = mockBucketAssigner.addUpdate("par3", "file_id_2");
assertBucketEquals(bucketInfo, "par3", BucketType.UPDATE, "file_id_2");
}
@Test
public void testAddInsert() {
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig);
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.INSERT);
mockBucketAssigner.addInsert("par1");
bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.INSERT);
mockBucketAssigner.addInsert("par2");
bucketInfo = mockBucketAssigner.addInsert("par2");
assertBucketEquals(bucketInfo, "par2", BucketType.INSERT);
bucketInfo = mockBucketAssigner.addInsert("par3");
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
bucketInfo = mockBucketAssigner.addInsert("par3");
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
}
@Test
public void testInsertWithSmallFiles() {
SmallFile f0 = new SmallFile();
f0.location = new HoodieRecordLocation("t0", "f0");
f0.sizeBytes = 12;
SmallFile f1 = new SmallFile();
f1.location = new HoodieRecordLocation("t0", "f1");
f1.sizeBytes = 122879; // no left space to append new records to this bucket
SmallFile f2 = new SmallFile();
f2.location = new HoodieRecordLocation("t0", "f2");
f2.sizeBytes = 56;
Map<String, List<SmallFile>> smallFilesMap = new HashMap<>();
smallFilesMap.put("par1", Arrays.asList(f0, f1));
smallFilesMap.put("par2", Collections.singletonList(f2));
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig, smallFilesMap);
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
mockBucketAssigner.addInsert("par1");
bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
mockBucketAssigner.addInsert("par2");
bucketInfo = mockBucketAssigner.addInsert("par2");
assertBucketEquals(bucketInfo, "par2", BucketType.UPDATE, "f2");
bucketInfo = mockBucketAssigner.addInsert("par3");
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
bucketInfo = mockBucketAssigner.addInsert("par3");
assertBucketEquals(bucketInfo, "par3", BucketType.INSERT);
}
@Test
public void testUpdateAndInsertWithSmallFiles() {
SmallFile f0 = new SmallFile();
f0.location = new HoodieRecordLocation("t0", "f0");
f0.sizeBytes = 12;
SmallFile f1 = new SmallFile();
f1.location = new HoodieRecordLocation("t0", "f1");
f1.sizeBytes = 122879; // no left space to append new records to this bucket
SmallFile f2 = new SmallFile();
f2.location = new HoodieRecordLocation("t0", "f2");
f2.sizeBytes = 56;
Map<String, List<SmallFile>> smallFilesMap = new HashMap<>();
smallFilesMap.put("par1", Arrays.asList(f0, f1));
smallFilesMap.put("par2", Collections.singletonList(f2));
MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig, smallFilesMap);
mockBucketAssigner.addUpdate("par1", "f0");
BucketInfo bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
mockBucketAssigner.addInsert("par1");
bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
mockBucketAssigner.addUpdate("par1", "f2");
mockBucketAssigner.addInsert("par1");
bucketInfo = mockBucketAssigner.addInsert("par1");
assertBucketEquals(bucketInfo, "par1", BucketType.UPDATE, "f0");
mockBucketAssigner.addUpdate("par2", "f0");
mockBucketAssigner.addInsert("par2");
bucketInfo = mockBucketAssigner.addInsert("par2");
assertBucketEquals(bucketInfo, "par2", BucketType.UPDATE, "f2");
}
private void assertBucketEquals(
BucketInfo bucketInfo,
String partition,
BucketType bucketType,
String fileId) {
BucketInfo actual = new BucketInfo(bucketType, fileId, partition);
assertThat(bucketInfo, is(actual));
}
private void assertBucketEquals(
BucketInfo bucketInfo,
String partition,
BucketType bucketType) {
assertThat(bucketInfo.getPartitionPath(), is(partition));
assertThat(bucketInfo.getBucketType(), is(bucketType));
}
/**
* Mock BucketAssigner that can specify small files explicitly.
*/
static class MockBucketAssigner extends BucketAssigner {
private final Map<String, List<SmallFile>> smallFilesMap;
MockBucketAssigner(
HoodieFlinkEngineContext context,
HoodieWriteConfig config) {
this(context, config, Collections.emptyMap());
}
MockBucketAssigner(
HoodieFlinkEngineContext context,
HoodieWriteConfig config,
Map<String, List<SmallFile>> smallFilesMap) {
super(context, config);
this.smallFilesMap = smallFilesMap;
}
@Override
protected List<SmallFile> getSmallFiles(String partitionPath) {
if (this.smallFilesMap.containsKey(partitionPath)) {
return this.smallFilesMap.get(partitionPath);
}
return Collections.emptyList();
}
}
}

View File

@@ -18,7 +18,14 @@
package org.apache.hudi.operator.utils;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.operator.StreamWriteFunction;
import org.apache.hudi.operator.StreamWriteOperatorCoordinator;
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
import org.apache.hudi.operator.partitioner.BucketAssignFunction;
import org.apache.hudi.operator.transform.RowDataToHoodieFunction;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
@@ -27,13 +34,9 @@ import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.streaming.api.operators.collect.utils.MockFunctionSnapshotContext;
import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorEventGateway;
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.operator.StreamWriteFunction;
import org.apache.hudi.operator.StreamWriteOperatorCoordinator;
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
import org.apache.flink.table.data.RowData;
import org.apache.flink.util.Collector;
import java.util.concurrent.CompletableFuture;
@@ -43,7 +46,6 @@ import java.util.concurrent.CompletableFuture;
* @param <I> Input type
*/
public class StreamWriteFunctionWrapper<I> {
private final TypeSerializer<I> serializer;
private final Configuration conf;
private final IOManager ioManager;
@@ -52,10 +54,18 @@ public class StreamWriteFunctionWrapper<I> {
private final StreamWriteOperatorCoordinator coordinator;
private final MockFunctionInitializationContext functionInitializationContext;
private StreamWriteFunction<Object, I, Object> function;
/** Function that converts row data to HoodieRecord. */
private RowDataToHoodieFunction<RowData, HoodieRecord<?>> toHoodieFunction;
/** Function that assigns bucket ID. */
private BucketAssignFunction<String, HoodieRecord<?>, HoodieRecord<?>> bucketAssignerFunction;
/** Stream write function. */
private StreamWriteFunction<Object, HoodieRecord<?>, Object> writeFunction;
public StreamWriteFunctionWrapper(String tablePath, TypeSerializer<I> serializer) throws Exception {
this.serializer = serializer;
public StreamWriteFunctionWrapper(String tablePath) throws Exception {
this(tablePath, TestConfigurations.getDefaultConf(tablePath));
}
public StreamWriteFunctionWrapper(String tablePath, Configuration conf) throws Exception {
this.ioManager = new IOManagerAsync();
MockEnvironment environment = new MockEnvironmentBuilder()
.setTaskName("mockTask")
@@ -64,7 +74,7 @@ public class StreamWriteFunctionWrapper<I> {
.build();
this.runtimeContext = new MockStreamingRuntimeContext(false, 1, 0, environment);
this.gateway = new MockOperatorEventGateway();
this.conf = TestConfigurations.getDefaultConf(tablePath);
this.conf = conf;
// one function
this.coordinator = new StreamWriteOperatorCoordinator(conf, 1);
this.coordinator.start();
@@ -72,14 +82,37 @@ public class StreamWriteFunctionWrapper<I> {
}
public void openFunction() throws Exception {
function = new StreamWriteFunction<>(TestConfigurations.ROW_TYPE, this.conf);
function.setRuntimeContext(runtimeContext);
function.setOperatorEventGateway(gateway);
function.open(this.conf);
toHoodieFunction = new RowDataToHoodieFunction<>(TestConfigurations.ROW_TYPE, conf);
toHoodieFunction.setRuntimeContext(runtimeContext);
toHoodieFunction.open(conf);
bucketAssignerFunction = new BucketAssignFunction<>(conf);
bucketAssignerFunction.setRuntimeContext(runtimeContext);
bucketAssignerFunction.open(conf);
bucketAssignerFunction.initializeState(this.functionInitializationContext);
writeFunction = new StreamWriteFunction<>(conf);
writeFunction.setRuntimeContext(runtimeContext);
writeFunction.setOperatorEventGateway(gateway);
writeFunction.open(conf);
}
public void invoke(I record) throws Exception {
function.processElement(record, null, null);
HoodieRecord<?> hoodieRecord = toHoodieFunction.map((RowData) record);
HoodieRecord<?>[] hoodieRecords = new HoodieRecord[1];
Collector<HoodieRecord<?>> collector = new Collector<HoodieRecord<?>>() {
@Override
public void collect(HoodieRecord<?> record) {
hoodieRecords[0] = record;
}
@Override
public void close() {
}
};
bucketAssignerFunction.processElement(hoodieRecord, null, collector);
writeFunction.processElement(hoodieRecords[0], null, null);
}
public BatchWriteSuccessEvent[] getEventBuffer() {
@@ -92,19 +125,22 @@ public class StreamWriteFunctionWrapper<I> {
@SuppressWarnings("rawtypes")
public HoodieFlinkWriteClient getWriteClient() {
return this.function.getWriteClient();
return this.writeFunction.getWriteClient();
}
public void checkpointFunction(long checkpointId) throws Exception {
// checkpoint the coordinator first
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
function.snapshotState(new MockFunctionSnapshotContext(checkpointId));
bucketAssignerFunction.snapshotState(null);
writeFunction.snapshotState(null);
functionInitializationContext.getOperatorStateStore().checkpointBegin(checkpointId);
}
public void checkpointComplete(long checkpointId) {
functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
coordinator.checkpointComplete(checkpointId);
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
}
public void checkpointFails(long checkpointId) {

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.operator.utils;
import org.apache.hudi.operator.FlinkOptions;
import org.apache.hudi.streamer.FlinkStreamerConfig;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.configuration.Configuration;
@@ -56,4 +57,16 @@ public class TestConfigurations {
conf.setString(FlinkOptions.PARTITION_PATH_FIELD, "partition");
return conf;
}
public static FlinkStreamerConfig getDefaultStreamerConf(String tablePath) {
FlinkStreamerConfig streamerConf = new FlinkStreamerConfig();
streamerConf.targetBasePath = tablePath;
streamerConf.readSchemaFilePath = Objects.requireNonNull(Thread.currentThread()
.getContextClassLoader().getResource("test_read_schema.avsc")).toString();
streamerConf.targetTableName = "TestHoodieTable";
streamerConf.partitionPathField = "partition";
streamerConf.tableType = "COPY_ON_WRITE";
streamerConf.checkpointInterval = 4000L;
return streamerConf;
}
}

View File

@@ -43,6 +43,7 @@ import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.stream.IntStream;
import static junit.framework.TestCase.assertEquals;
import static org.hamcrest.CoreMatchers.is;
@@ -92,6 +93,13 @@ public class TestData {
TimestampData.fromEpochMillis(8), StringData.fromString("par4"))
);
public static List<RowData> DATA_SET_THREE = new ArrayList<>();
static {
IntStream.range(0, 5).forEach(i -> DATA_SET_THREE.add(
binaryRow(StringData.fromString("id1"), StringData.fromString("Danny"), 23,
TimestampData.fromEpochMillis(1), StringData.fromString("par1"))));
}
/**
* Checks the source data TestConfigurations.DATA_SET_ONE are written as expected.
*
@@ -101,13 +109,29 @@ public class TestData {
* @param expected The expected results mapping, the key should be the partition path
*/
public static void checkWrittenData(File baseFile, Map<String, String> expected) throws IOException {
checkWrittenData(baseFile, expected, 4);
}
/**
* Checks the source data TestConfigurations.DATA_SET_ONE are written as expected.
*
* <p>Note: Replace it with the Flink reader when it is supported.
*
* @param baseFile The file base to check, should be a directly
* @param expected The expected results mapping, the key should be the partition path
* @param partitions The expected partition number
*/
public static void checkWrittenData(
File baseFile,
Map<String, String> expected,
int partitions) throws IOException {
assert baseFile.isDirectory();
FileFilter filter = file -> !file.getName().startsWith(".");
File[] partitionDirs = baseFile.listFiles(filter);
assertNotNull(partitionDirs);
assertThat(partitionDirs.length, is(4));
assertThat(partitionDirs.length, is(partitions));
for (File partitionDir : partitionDirs) {
File[] dataFiles = partitionDir.listFiles(file -> file.getName().endsWith(".parquet"));
File[] dataFiles = partitionDir.listFiles(filter);
assertNotNull(dataFiles);
File latestDataFile = Arrays.stream(dataFiles)
.max(Comparator.comparing(f -> FSUtils.getCommitTime(f.getName())))