[HUDI-1949] Refactor BucketAssigner to make it more efficient (#3017)
Add a process single class WriteProfile, the record and small files profile re-construction can be more efficient if we reuse by same checkpoint id.
This commit is contained in:
@@ -23,6 +23,7 @@ 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.sink.partitioner.profile.WriteProfile;
|
||||
import org.apache.hudi.table.action.commit.BucketInfo;
|
||||
import org.apache.hudi.table.action.commit.BucketType;
|
||||
import org.apache.hudi.table.action.commit.SmallFile;
|
||||
@@ -311,7 +312,6 @@ public class TestBucketAssigner {
|
||||
* Mock BucketAssigner that can specify small files explicitly.
|
||||
*/
|
||||
static class MockBucketAssigner extends BucketAssigner {
|
||||
private final Map<String, List<SmallFile>> smallFilesMap;
|
||||
|
||||
MockBucketAssigner(
|
||||
HoodieFlinkEngineContext context,
|
||||
@@ -332,12 +332,23 @@ public class TestBucketAssigner {
|
||||
HoodieFlinkEngineContext context,
|
||||
HoodieWriteConfig config,
|
||||
Map<String, List<SmallFile>> smallFilesMap) {
|
||||
super(taskID, numTasks, context, config);
|
||||
super(taskID, numTasks, new MockWriteProfile(config, context, smallFilesMap), config);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Mock WriteProfile that can specify small files explicitly.
|
||||
*/
|
||||
static class MockWriteProfile extends WriteProfile {
|
||||
private final Map<String, List<SmallFile>> smallFilesMap;
|
||||
|
||||
public MockWriteProfile(HoodieWriteConfig config, HoodieFlinkEngineContext context, Map<String, List<SmallFile>> smallFilesMap) {
|
||||
super(config, context);
|
||||
this.smallFilesMap = smallFilesMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
protected List<SmallFile> smallFilesProfile(String partitionPath) {
|
||||
if (this.smallFilesMap.containsKey(partitionPath)) {
|
||||
return this.smallFilesMap.get(partitionPath);
|
||||
}
|
||||
|
||||
@@ -23,15 +23,12 @@ import org.apache.hudi.streamer.FlinkStreamerConfig;
|
||||
import org.apache.hudi.utils.factory.CollectSinkTableFactory;
|
||||
import org.apache.hudi.utils.factory.ContinuousFileSourceFactory;
|
||||
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.api.common.typeinfo.Types;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.table.api.DataTypes;
|
||||
import org.apache.flink.table.api.TableSchema;
|
||||
import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
import org.apache.flink.types.Row;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
@@ -59,13 +56,6 @@ public class TestConfigurations {
|
||||
ROW_DATA_TYPE.getChildren().toArray(new DataType[0]))
|
||||
.build();
|
||||
|
||||
public static final TypeInformation<Row> ROW_TYPE_INFO = Types.ROW(
|
||||
Types.STRING,
|
||||
Types.STRING,
|
||||
Types.INT,
|
||||
Types.LOCAL_DATE_TIME,
|
||||
Types.STRING);
|
||||
|
||||
public static String getCreateHoodieTableDDL(String tableName, Map<String, String> options) {
|
||||
String createTable = "create table " + tableName + "(\n"
|
||||
+ " uuid varchar(20),\n"
|
||||
|
||||
@@ -18,10 +18,9 @@
|
||||
|
||||
package org.apache.hudi.utils.factory;
|
||||
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
import org.apache.flink.api.common.state.ListState;
|
||||
import org.apache.flink.api.common.state.ListStateDescriptor;
|
||||
import org.apache.flink.api.java.typeutils.RowTypeInfo;
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.runtime.state.FunctionInitializationContext;
|
||||
import org.apache.flink.runtime.state.FunctionSnapshotContext;
|
||||
@@ -35,6 +34,8 @@ import org.apache.flink.table.connector.sink.SinkFunctionProvider;
|
||||
import org.apache.flink.table.data.RowData;
|
||||
import org.apache.flink.table.factories.DynamicTableSinkFactory;
|
||||
import org.apache.flink.table.factories.FactoryUtil;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.types.utils.TypeConversions;
|
||||
import org.apache.flink.types.Row;
|
||||
import org.apache.flink.types.RowKind;
|
||||
|
||||
@@ -112,8 +113,10 @@ public class CollectSinkTableFactory implements DynamicTableSinkFactory {
|
||||
|
||||
@Override
|
||||
public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
|
||||
final DataType rowType = schema.toPhysicalRowDataType();
|
||||
final RowTypeInfo rowTypeInfo = (RowTypeInfo) TypeConversions.fromDataTypeToLegacyInfo(rowType);
|
||||
DataStructureConverter converter = context.createDataStructureConverter(schema.toPhysicalRowDataType());
|
||||
return SinkFunctionProvider.of(new CollectSinkFunction(converter));
|
||||
return SinkFunctionProvider.of(new CollectSinkFunction(converter, rowTypeInfo));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -131,14 +134,16 @@ public class CollectSinkTableFactory implements DynamicTableSinkFactory {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
private final DynamicTableSink.DataStructureConverter converter;
|
||||
private final RowTypeInfo rowTypeInfo;
|
||||
|
||||
protected transient ListState<Row> resultState;
|
||||
protected transient List<Row> localResult;
|
||||
|
||||
private int taskID;
|
||||
|
||||
protected CollectSinkFunction(DynamicTableSink.DataStructureConverter converter) {
|
||||
protected CollectSinkFunction(DynamicTableSink.DataStructureConverter converter, RowTypeInfo rowTypeInfo) {
|
||||
this.converter = converter;
|
||||
this.rowTypeInfo = rowTypeInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -151,7 +156,7 @@ public class CollectSinkTableFactory implements DynamicTableSinkFactory {
|
||||
@Override
|
||||
public void initializeState(FunctionInitializationContext context) throws Exception {
|
||||
this.resultState = context.getOperatorStateStore().getListState(
|
||||
new ListStateDescriptor<>("sink-results", TestConfigurations.ROW_TYPE_INFO));
|
||||
new ListStateDescriptor<>("sink-results", rowTypeInfo));
|
||||
this.localResult = new ArrayList<>();
|
||||
if (context.isRestored()) {
|
||||
for (Row value : resultState.get()) {
|
||||
|
||||
Reference in New Issue
Block a user