1
0

[HUDI-1495] Bump Flink version to 1.12.2 (#2718)

This commit is contained in:
Danny Chan
2021-03-26 14:25:57 +08:00
committed by GitHub
parent 6e803e08b1
commit 8b774fe331
27 changed files with 413 additions and 408 deletions

View File

@@ -30,6 +30,7 @@
<properties> <properties>
<main.basedir>${project.parent.basedir}</main.basedir> <main.basedir>${project.parent.basedir}</main.basedir>
<parquet.version>1.11.1</parquet.version>
</properties> </properties>
<build> <build>
@@ -170,10 +171,23 @@
<scope>provided</scope> <scope>provided</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
<version>${parquet.version}</version>
<exclusions>
<exclusion>
<groupId>org.xerial.snappy</groupId>
<artifactId>snappy-java</artifactId>
</exclusion>
</exclusions>
</dependency>
<!-- Parquet --> <!-- Parquet -->
<dependency> <dependency>
<groupId>org.apache.parquet</groupId> <groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId> <artifactId>parquet-avro</artifactId>
<version>${parquet.version}</version>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.configuration;
import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.keygen.SimpleAvroKeyGenerator; import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.streamer.FlinkStreamerConfig; import org.apache.hudi.streamer.FlinkStreamerConfig;
@@ -30,10 +31,13 @@ import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import java.util.Arrays; import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
/** /**
* Hoodie Flink config options. * Hoodie Flink config options.
@@ -287,12 +291,6 @@ public class FlinkOptions {
// Utilities // Utilities
// ------------------------------------------------------------------------- // -------------------------------------------------------------------------
// Remember to update the set when adding new options.
public static final List<ConfigOption<?>> OPTIONAL_OPTIONS = Arrays.asList(
TABLE_TYPE, OPERATION, PRECOMBINE_FIELD, PAYLOAD_CLASS, INSERT_DROP_DUPS, RETRY_TIMES,
RETRY_INTERVAL_MS, IGNORE_FAILED, RECORD_KEY_FIELD, PARTITION_PATH_FIELD, KEYGEN_CLASS
);
// Prefix for Hoodie specific properties. // Prefix for Hoodie specific properties.
private static final String PROPERTIES_PREFIX = "properties."; private static final String PROPERTIES_PREFIX = "properties.";
@@ -385,4 +383,32 @@ public class FlinkOptions {
return !conf.getOptional(option).isPresent() return !conf.getOptional(option).isPresent()
|| conf.get(option).equals(option.defaultValue()); || conf.get(option).equals(option.defaultValue());
} }
/**
* Returns all the optional config options.
*/
public static Set<ConfigOption<?>> optionalOptions() {
Set<ConfigOption<?>> options = new HashSet<>(allOptions());
options.remove(PATH);
return options;
}
/**
* Returns all the config options.
*/
public static List<ConfigOption<?>> allOptions() {
Field[] declaredFields = FlinkOptions.class.getDeclaredFields();
List<ConfigOption<?>> options = new ArrayList<>();
for (Field field : declaredFields) {
if (java.lang.reflect.Modifier.isStatic(field.getModifiers())
&& field.getType().equals(ConfigOption.class)) {
try {
options.add((ConfigOption<?>) field.get(ConfigOption.class));
} catch (IllegalAccessException e) {
throw new HoodieException("Error while fetching static config option", e);
}
}
}
return options;
}
} }

View File

@@ -157,7 +157,7 @@ public class StreamWriteOperatorCoordinator
} }
@Override @Override
public void checkpointComplete(long checkpointId) { public void notifyCheckpointComplete(long checkpointId) {
// start to commit the instant. // start to commit the instant.
checkAndCommitWithRetry(); checkAndCommitWithRetry();
// if async compaction is on, schedule the compaction // if async compaction is on, schedule the compaction
@@ -182,7 +182,7 @@ public class StreamWriteOperatorCoordinator
} }
@Override @Override
public void resetToCheckpoint(@Nullable byte[] checkpointData) throws Exception { public void resetToCheckpoint(long checkpointID, @Nullable byte[] checkpointData) throws Exception {
if (checkpointData != null) { if (checkpointData != null) {
// restore when any checkpoint completed // restore when any checkpoint completed
deserializeCheckpointAndRestore(checkpointData); deserializeCheckpointAndRestore(checkpointData);
@@ -215,6 +215,11 @@ public class StreamWriteOperatorCoordinator
// no operation // no operation
} }
@Override
public void subtaskReset(int i, long l) {
// no operation
}
// ------------------------------------------------------------------------- // -------------------------------------------------------------------------
// Utilities // Utilities
// ------------------------------------------------------------------------- // -------------------------------------------------------------------------

View File

@@ -36,7 +36,7 @@ import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.RowType;
import java.util.Properties; import java.util.Properties;
@@ -80,7 +80,7 @@ public class HoodieFlinkStreamerV2 {
cfg.kafkaTopic, cfg.kafkaTopic,
new JsonRowDataDeserializationSchema( new JsonRowDataDeserializationSchema(
rowType, rowType,
new RowDataTypeInfo(rowType), InternalTypeInfo.of(rowType),
false, false,
true, true,
TimestampFormat.ISO_8601 TimestampFormat.ISO_8601

View File

@@ -19,21 +19,20 @@
package org.apache.hudi.table; package org.apache.hudi.table;
import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.keygen.ComplexAvroKeyGenerator; import org.apache.hudi.keygen.ComplexAvroKeyGenerator;
import org.apache.hudi.util.AvroSchemaConverter; import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.api.constraints.UniqueConstraint; import org.apache.flink.table.api.constraints.UniqueConstraint;
import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.data.RowData; import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.factories.DynamicTableSinkFactory;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.factories.TableSinkFactory;
import org.apache.flink.table.factories.TableSourceFactory;
import org.apache.flink.table.sinks.TableSink;
import org.apache.flink.table.sources.TableSource;
import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.utils.TableSchemaUtils; import org.apache.flink.table.utils.TableSchemaUtils;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@@ -41,62 +40,57 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Set;
/** /**
* Hoodie data source/sink factory. * Hoodie data source/sink factory.
*/ */
public class HoodieTableFactory implements TableSourceFactory<RowData>, TableSinkFactory<RowData> { public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory {
private static final Logger LOG = LoggerFactory.getLogger(HoodieTableFactory.class); private static final Logger LOG = LoggerFactory.getLogger(HoodieTableFactory.class);
public static final String FACTORY_ID = "hudi"; public static final String FACTORY_ID = "hudi";
@Override @Override
public TableSource<RowData> createTableSource(TableSourceFactory.Context context) { public DynamicTableSource createDynamicTableSource(Context context) {
Configuration conf = FlinkOptions.fromMap(context.getTable().getOptions()); FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
TableSchema schema = TableSchemaUtils.getPhysicalSchema(context.getTable().getSchema()); helper.validate();
setupConfOptions(conf, context.getObjectIdentifier().getObjectName(), context.getTable(), schema);
// enclosing the code within a try catch block so that we can log the error message. Configuration conf = (Configuration) helper.getOptions();
// Flink 1.11 did a bad compatibility for the old table factory, it uses the old factory TableSchema schema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
// to create the source/sink and catches all the exceptions then tries the new factory. setupConfOptions(conf, context.getObjectIdentifier().getObjectName(), context.getCatalogTable(), schema);
//
// log the error message first so that there is a chance to show the real failure cause.
try {
Path path = new Path(conf.getOptional(FlinkOptions.PATH).orElseThrow(() -> Path path = new Path(conf.getOptional(FlinkOptions.PATH).orElseThrow(() ->
new ValidationException("Option [path] should not be empty."))); new ValidationException("Option [path] should not be empty.")));
return new HoodieTableSource( return new HoodieTableSource(
schema, schema,
path, path,
context.getTable().getPartitionKeys(), context.getCatalogTable().getPartitionKeys(),
conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME), conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME),
conf); conf);
} catch (Throwable throwable) {
LOG.error("Create table source error", throwable);
throw new HoodieException(throwable);
}
} }
@Override @Override
public TableSink<RowData> createTableSink(TableSinkFactory.Context context) { public DynamicTableSink createDynamicTableSink(Context context) {
Configuration conf = FlinkOptions.fromMap(context.getTable().getOptions()); Configuration conf = FlinkOptions.fromMap(context.getCatalogTable().getOptions());
TableSchema schema = TableSchemaUtils.getPhysicalSchema(context.getTable().getSchema()); TableSchema schema = TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema());
setupConfOptions(conf, context.getObjectIdentifier().getObjectName(), context.getTable(), schema); setupConfOptions(conf, context.getObjectIdentifier().getObjectName(), context.getCatalogTable(), schema);
return new HoodieTableSink(conf, schema); return new HoodieTableSink(conf, schema);
} }
@Override @Override
public Map<String, String> requiredContext() { public String factoryIdentifier() {
Map<String, String> context = new HashMap<>(); return FACTORY_ID;
context.put(FactoryUtil.CONNECTOR.key(), FACTORY_ID);
return context;
} }
@Override @Override
public List<String> supportedProperties() { public Set<ConfigOption<?>> requiredOptions() {
// contains format properties. return Collections.singleton(FlinkOptions.PATH);
return Collections.singletonList("*"); }
@Override
public Set<ConfigOption<?>> optionalOptions() {
return FlinkOptions.optionalOptions();
} }
// ------------------------------------------------------------------------- // -------------------------------------------------------------------------

View File

@@ -34,23 +34,22 @@ import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.data.RowData; import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
import org.apache.flink.table.sinks.PartitionableTableSink; import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.types.RowKind;
import java.util.Map; import java.util.Map;
/** /**
* Hoodie table sink. * Hoodie table sink.
*/ */
public class HoodieTableSink implements AppendStreamTableSink<RowData>, PartitionableTableSink { public class HoodieTableSink implements DynamicTableSink, SupportsPartitioning {
private final Configuration conf; private final Configuration conf;
private final TableSchema schema; private final TableSchema schema;
@@ -61,10 +60,11 @@ public class HoodieTableSink implements AppendStreamTableSink<RowData>, Partitio
} }
@Override @Override
public DataStreamSink<?> consumeDataStream(DataStream<RowData> dataStream) { public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
return (DataStreamSinkProvider) dataStream -> {
// Read from kafka source // Read from kafka source
RowType rowType = (RowType) this.schema.toRowDataType().notNull().getLogicalType(); RowType rowType = (RowType) schema.toRowDataType().notNull().getLogicalType();
int numWriteTasks = this.conf.getInteger(FlinkOptions.WRITE_TASKS); int numWriteTasks = conf.getInteger(FlinkOptions.WRITE_TASKS);
StreamWriteOperatorFactory<HoodieRecord> operatorFactory = new StreamWriteOperatorFactory<>(conf); StreamWriteOperatorFactory<HoodieRecord> operatorFactory = new StreamWriteOperatorFactory<>(conf);
DataStream<Object> pipeline = dataStream DataStream<Object> pipeline = dataStream
@@ -96,28 +96,10 @@ public class HoodieTableSink implements AppendStreamTableSink<RowData>, Partitio
.setParallelism(1); // compaction commit should be singleton .setParallelism(1); // compaction commit should be singleton
} else { } else {
return pipeline.addSink(new DummySinkFunction<>()) return pipeline.addSink(new DummySinkFunction<>())
.setParallelism(1)
.name("dummy").uid("uid_dummy"); .name("dummy").uid("uid_dummy");
} }
} };
@Override
public TableSink<RowData> configure(String[] strings, TypeInformation<?>[] infos) {
return this;
}
@Override
public TableSchema getTableSchema() {
return this.schema;
}
@Override
public DataType getConsumedDataType() {
return this.schema.toRowDataType().bridgedTo(RowData.class);
}
@Override
public void setStaticPartition(Map<String, String> partitions) {
// no operation
} }
@VisibleForTesting @VisibleForTesting
@@ -125,6 +107,31 @@ public class HoodieTableSink implements AppendStreamTableSink<RowData>, Partitio
return this.conf; return this.conf;
} }
@Override
public ChangelogMode getChangelogMode(ChangelogMode changelogMode) {
// ignore RowKind.UPDATE_BEFORE
return ChangelogMode.newBuilder()
.addContainedKind(RowKind.DELETE)
.addContainedKind(RowKind.INSERT)
.addContainedKind(RowKind.UPDATE_AFTER)
.build();
}
@Override
public DynamicTableSink copy() {
return new HoodieTableSink(this.conf, this.schema);
}
@Override
public String asSummaryString() {
return "HoodieTableSink";
}
@Override
public void applyStaticPartition(Map<String, String> map) {
// no operation
}
// Dummy sink function that does nothing. // Dummy sink function that does nothing.
private static class DummySinkFunction<T> implements SinkFunction<T> {} private static class DummySinkFunction<T> implements SinkFunction<T> {}
} }

View File

@@ -57,18 +57,20 @@ import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown;
import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
import org.apache.flink.table.connector.source.abilities.SupportsPartitionPushDown;
import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.RowData;
import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.ResolvedExpression;
import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter;
import org.apache.flink.table.sources.FilterableTableSource;
import org.apache.flink.table.sources.LimitableTableSource;
import org.apache.flink.table.sources.PartitionableTableSource;
import org.apache.flink.table.sources.ProjectableTableSource;
import org.apache.flink.table.sources.StreamTableSource;
import org.apache.flink.table.sources.TableSource;
import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.utils.TableConnectorUtils;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
@@ -83,6 +85,7 @@ import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
@@ -94,11 +97,11 @@ import static org.apache.hudi.table.format.FormatUtils.getParquetConf;
* Hoodie batch table source that always read the latest snapshot of the underneath table. * Hoodie batch table source that always read the latest snapshot of the underneath table.
*/ */
public class HoodieTableSource implements public class HoodieTableSource implements
StreamTableSource<RowData>, ScanTableSource,
PartitionableTableSource, SupportsPartitionPushDown,
ProjectableTableSource<RowData>, SupportsProjectionPushDown,
LimitableTableSource<RowData>, SupportsLimitPushDown,
FilterableTableSource<RowData> { SupportsFilterPushDown {
private static final Logger LOG = LoggerFactory.getLogger(HoodieTableSource.class); private static final Logger LOG = LoggerFactory.getLogger(HoodieTableSource.class);
private static final int NO_LIMIT_CONSTANT = -1; private static final int NO_LIMIT_CONSTANT = -1;
@@ -113,9 +116,9 @@ public class HoodieTableSource implements
private final String defaultPartName; private final String defaultPartName;
private final Configuration conf; private final Configuration conf;
private final int[] requiredPos; private int[] requiredPos;
private final long limit; private long limit;
private final List<Expression> filters; private List<Expression> filters;
private List<Map<String, String>> requiredPartitions; private List<Map<String, String>> requiredPartitions;
@@ -161,7 +164,16 @@ public class HoodieTableSource implements
} }
@Override @Override
public DataStream<RowData> getDataStream(StreamExecutionEnvironment execEnv) { public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
return new DataStreamScanProvider() {
@Override
public boolean isBounded() {
return !conf.getBoolean(FlinkOptions.READ_AS_STREAMING);
}
@Override
public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv) {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
TypeInformation<RowData> typeInfo = TypeInformation<RowData> typeInfo =
(TypeInformation<RowData>) TypeInfoDataTypeConverter.fromDataTypeToTypeInfo(getProducedDataType()); (TypeInformation<RowData>) TypeInfoDataTypeConverter.fromDataTypeToTypeInfo(getProducedDataType());
@@ -178,77 +190,66 @@ public class HoodieTableSource implements
return new DataStreamSource<>(source); return new DataStreamSource<>(source);
} else { } else {
InputFormatSourceFunction<RowData> func = new InputFormatSourceFunction<>(getInputFormat(), typeInfo); InputFormatSourceFunction<RowData> func = new InputFormatSourceFunction<>(getInputFormat(), typeInfo);
DataStreamSource<RowData> source = execEnv.addSource(func, explainSource(), typeInfo); DataStreamSource<RowData> source = execEnv.addSource(func, asSummaryString(), typeInfo);
return source.name("streaming_source") return source.name("streaming_source")
.setParallelism(conf.getInteger(FlinkOptions.READ_TASKS)) .setParallelism(conf.getInteger(FlinkOptions.READ_TASKS))
.uid("uid_streaming_source"); .uid("uid_streaming_source");
} }
} }
};
@Override
public boolean isBounded() {
return !conf.getBoolean(FlinkOptions.READ_AS_STREAMING);
} }
@Override @Override
public TableSource<RowData> applyPredicate(List<Expression> predicates) { public ChangelogMode getChangelogMode() {
return new HoodieTableSource(schema, path, partitionKeys, defaultPartName, conf, return ChangelogMode.insertOnly();
requiredPartitions, requiredPos, limit, new ArrayList<>(predicates));
} }
@Override @Override
public boolean isFilterPushedDown() { public DynamicTableSource copy() {
return this.filters != null && this.filters.size() > 0; return new HoodieTableSource(schema, path, partitionKeys, defaultPartName,
conf, requiredPartitions, requiredPos, limit, filters);
} }
@Override @Override
public boolean isLimitPushedDown() { public String asSummaryString() {
return this.limit != NO_LIMIT_CONSTANT; return "HudiTableSource";
} }
@Override @Override
public TableSource<RowData> applyLimit(long limit) { public Result applyFilters(List<ResolvedExpression> filters) {
return new HoodieTableSource(schema, path, partitionKeys, defaultPartName, conf, this.filters = new ArrayList<>(filters);
requiredPartitions, requiredPos, limit, filters); return Result.of(new ArrayList<>(filters), new ArrayList<>(filters));
} }
@Override @Override
public List<Map<String, String>> getPartitions() { public Optional<List<Map<String, String>>> listPartitions() {
return FilePathUtils.getPartitions(path, hadoopConf, partitionKeys, defaultPartName, List<Map<String, String>> partitions = FilePathUtils.getPartitions(path, hadoopConf,
conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITION)); partitionKeys, defaultPartName, conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITION));
return Optional.of(partitions);
} }
@Override @Override
public TableSource applyPartitionPruning(List<Map<String, String>> requiredPartitions) { public void applyPartitions(List<Map<String, String>> partitions) {
return new HoodieTableSource(schema, path, partitionKeys, defaultPartName, conf, this.requiredPartitions = partitions;
requiredPartitions, requiredPos, limit, filters);
} }
@Override @Override
public TableSource<RowData> projectFields(int[] requiredPos) { public boolean supportsNestedProjection() {
return new HoodieTableSource(schema, path, partitionKeys, defaultPartName, conf, return false;
requiredPartitions, requiredPos, limit, filters);
} }
@Override @Override
public TableSchema getTableSchema() { public void applyProjection(int[][] projections) {
return schema; // nested projection is not supported.
this.requiredPos = Arrays.stream(projections).mapToInt(array -> array[0]).toArray();
} }
@Override @Override
public String explainSource() { public void applyLimit(long limit) {
final String filterString = filters.stream() this.limit = limit;
.map(Expression::asSummaryString)
.collect(Collectors.joining(","));
return TableConnectorUtils.generateRuntimeName(getClass(), getTableSchema().getFieldNames())
+ (requiredPartitions == null ? "" : ", requiredPartition=" + requiredPartitions)
+ (requiredPos == null ? "" : ", requiredPos=" + Arrays.toString(requiredPos))
+ (limit == -1 ? "" : ", limit=" + limit)
+ (filters.size() == 0 ? "" : ", filters=" + filterString);
} }
@Override private DataType getProducedDataType() {
public DataType getProducedDataType() {
String[] schemaFieldNames = this.schema.getFieldNames(); String[] schemaFieldNames = this.schema.getFieldNames();
DataType[] schemaTypes = this.schema.getFieldDataTypes(); DataType[] schemaTypes = this.schema.getFieldDataTypes();
@@ -260,7 +261,7 @@ public class HoodieTableSource implements
private List<Map<String, String>> getOrFetchPartitions() { private List<Map<String, String>> getOrFetchPartitions() {
if (requiredPartitions == null) { if (requiredPartitions == null) {
requiredPartitions = getPartitions(); requiredPartitions = listPartitions().orElse(Collections.emptyList());
} }
return requiredPartitions; return requiredPartitions;
} }

View File

@@ -225,8 +225,8 @@ public class StreamerUtil {
// put all the set up options // put all the set up options
conf.addAllToProperties(properties); conf.addAllToProperties(properties);
// put all the default options // put all the default options
for (ConfigOption<?> option : FlinkOptions.OPTIONAL_OPTIONS) { for (ConfigOption<?> option : FlinkOptions.optionalOptions()) {
if (!conf.contains(option)) { if (!conf.contains(option) && option.hasDefaultValue()) {
properties.put(option.key(), option.defaultValue()); properties.put(option.key(), option.defaultValue());
} }
} }

View File

@@ -53,7 +53,7 @@ import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.FileProcessingMode; import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.util.TestLogger; import org.apache.flink.util.TestLogger;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@@ -104,7 +104,7 @@ public class StreamWriteITCase extends TestLogger {
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType, rowType,
new RowDataTypeInfo(rowType), InternalTypeInfo.of(rowType),
false, false,
true, true,
TimestampFormat.ISO_8601 TimestampFormat.ISO_8601
@@ -135,7 +135,7 @@ public class StreamWriteITCase extends TestLogger {
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME))); JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
// wait for the streaming job to finish // wait for the streaming job to finish
client.getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get(); client.getJobExecutionResult().get();
TestData.checkWrittenFullData(tempFile, EXPECTED); TestData.checkWrittenFullData(tempFile, EXPECTED);
} }
@@ -159,7 +159,7 @@ public class StreamWriteITCase extends TestLogger {
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType, rowType,
new RowDataTypeInfo(rowType), InternalTypeInfo.of(rowType),
false, false,
true, true,
TimestampFormat.ISO_8601 TimestampFormat.ISO_8601
@@ -204,7 +204,7 @@ public class StreamWriteITCase extends TestLogger {
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME))); JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
// wait for the streaming job to finish // wait for the streaming job to finish
client.getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get(); client.getJobExecutionResult().get();
TestData.checkWrittenFullData(tempFile, EXPECTED); TestData.checkWrittenFullData(tempFile, EXPECTED);
} }
@@ -230,7 +230,7 @@ public class StreamWriteITCase extends TestLogger {
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType, rowType,
new RowDataTypeInfo(rowType), InternalTypeInfo.of(rowType),
false, false,
true, true,
TimestampFormat.ISO_8601 TimestampFormat.ISO_8601

View File

@@ -94,7 +94,7 @@ public class TestStreamWriteOperatorCoordinator {
coordinator.handleEventFromOperator(0, event0); coordinator.handleEventFromOperator(0, event0);
coordinator.handleEventFromOperator(1, event1); coordinator.handleEventFromOperator(1, event1);
coordinator.checkpointComplete(1); coordinator.notifyCheckpointComplete(1);
String inflight = coordinator.getWriteClient() String inflight = coordinator.getWriteClient()
.getInflightAndRequestedInstant("COPY_ON_WRITE"); .getInflightAndRequestedInstant("COPY_ON_WRITE");
String lastCompleted = coordinator.getWriteClient().getLastCompletedInstant("COPY_ON_WRITE"); String lastCompleted = coordinator.getWriteClient().getLastCompletedInstant("COPY_ON_WRITE");
@@ -116,7 +116,7 @@ public class TestStreamWriteOperatorCoordinator {
public void testCheckpointAndRestore() throws Exception { public void testCheckpointAndRestore() throws Exception {
CompletableFuture<byte[]> future = new CompletableFuture<>(); CompletableFuture<byte[]> future = new CompletableFuture<>();
coordinator.checkpointCoordinator(1, future); coordinator.checkpointCoordinator(1, future);
coordinator.resetToCheckpoint(future.get()); coordinator.resetToCheckpoint(1, future.get());
} }
@Test @Test
@@ -145,7 +145,7 @@ public class TestStreamWriteOperatorCoordinator {
.build(); .build();
coordinator.handleEventFromOperator(0, event); coordinator.handleEventFromOperator(0, event);
assertThrows(HoodieException.class, assertThrows(HoodieException.class,
() -> coordinator.checkpointComplete(1), () -> coordinator.notifyCheckpointComplete(1),
"Try 3 to commit instant"); "Try 3 to commit instant");
} }
} }

View File

@@ -20,8 +20,6 @@ package org.apache.hudi.sink.utils;
import org.apache.flink.api.common.state.AggregatingState; import org.apache.flink.api.common.state.AggregatingState;
import org.apache.flink.api.common.state.AggregatingStateDescriptor; import org.apache.flink.api.common.state.AggregatingStateDescriptor;
import org.apache.flink.api.common.state.BroadcastState; import org.apache.flink.api.common.state.BroadcastState;
import org.apache.flink.api.common.state.FoldingState;
import org.apache.flink.api.common.state.FoldingStateDescriptor;
import org.apache.flink.api.common.state.KeyedStateStore; import org.apache.flink.api.common.state.KeyedStateStore;
import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ListStateDescriptor;
@@ -89,11 +87,6 @@ public class MockOperatorStateStore implements KeyedStateStore, OperatorStateSto
return null; return null;
} }
@Override
public <T, A> FoldingState<T, A> getFoldingState(FoldingStateDescriptor<T, A> foldingStateDescriptor) {
return null;
}
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public <K, V> MapState<K, V> getMapState(MapStateDescriptor<K, V> mapStateDescriptor) { public <K, V> MapState<K, V> getMapState(MapStateDescriptor<K, V> mapStateDescriptor) {

View File

@@ -156,7 +156,7 @@ public class StreamWriteFunctionWrapper<I> {
public void checkpointComplete(long checkpointId) { public void checkpointComplete(long checkpointId) {
functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId); functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
coordinator.checkpointComplete(checkpointId); coordinator.notifyCheckpointComplete(checkpointId);
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId); this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
this.writeFunction.notifyCheckpointComplete(checkpointId); this.writeFunction.notifyCheckpointComplete(checkpointId);
if (conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) { if (conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) {

View File

@@ -218,8 +218,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
TableResult tableResult = tEnv.executeSql(insert); TableResult tableResult = tEnv.executeSql(insert);
// wait to finish // wait to finish
try { try {
tableResult.getJobClient().get() tableResult.getJobClient().get().getJobExecutionResult().get();
.getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
} catch (InterruptedException | ExecutionException ex) { } catch (InterruptedException | ExecutionException ex) {
throw new RuntimeException(ex); throw new RuntimeException(ex);
} }

View File

@@ -30,8 +30,7 @@ import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTable;
import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.CatalogTableImpl;
import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectIdentifier;
import org.apache.flink.table.factories.TableSinkFactory; import org.apache.flink.table.factories.DynamicTableFactory;
import org.apache.flink.table.factories.TableSourceFactory;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.api.io.TempDir;
@@ -78,14 +77,14 @@ public class TestHoodieTableFactory {
void testInferAvroSchemaForSource() { void testInferAvroSchemaForSource() {
// infer the schema if not specified // infer the schema if not specified
final HoodieTableSource tableSource1 = final HoodieTableSource tableSource1 =
(HoodieTableSource) new HoodieTableFactory().createTableSource(MockSourceContext.getInstance(this.conf)); (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(MockContext.getInstance(this.conf));
final Configuration conf1 = tableSource1.getConf(); final Configuration conf1 = tableSource1.getConf();
assertThat(conf1.get(FlinkOptions.READ_AVRO_SCHEMA), is(INFERRED_SCHEMA)); assertThat(conf1.get(FlinkOptions.READ_AVRO_SCHEMA), is(INFERRED_SCHEMA));
// set up the explicit schema using the file path // set up the explicit schema using the file path
this.conf.setString(FlinkOptions.READ_AVRO_SCHEMA_PATH, AVRO_SCHEMA_FILE_PATH); this.conf.setString(FlinkOptions.READ_AVRO_SCHEMA_PATH, AVRO_SCHEMA_FILE_PATH);
HoodieTableSource tableSource2 = HoodieTableSource tableSource2 =
(HoodieTableSource) new HoodieTableFactory().createTableSource(MockSourceContext.getInstance(this.conf)); (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(MockContext.getInstance(this.conf));
Configuration conf2 = tableSource2.getConf(); Configuration conf2 = tableSource2.getConf();
assertNull(conf2.get(FlinkOptions.READ_AVRO_SCHEMA), "expect schema string as null"); assertNull(conf2.get(FlinkOptions.READ_AVRO_SCHEMA), "expect schema string as null");
} }
@@ -101,8 +100,8 @@ public class TestHoodieTableFactory {
.field("f2", DataTypes.TIMESTAMP(3)) .field("f2", DataTypes.TIMESTAMP(3))
.primaryKey("f0") .primaryKey("f0")
.build(); .build();
final MockSourceContext sourceContext1 = MockSourceContext.getInstance(this.conf, schema1, "f2"); final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createTableSource(sourceContext1); final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
final Configuration conf1 = tableSource1.getConf(); final Configuration conf1 = tableSource1.getConf();
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0")); assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass")); assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass"));
@@ -115,8 +114,8 @@ public class TestHoodieTableFactory {
.field("f2", DataTypes.TIMESTAMP(3)) .field("f2", DataTypes.TIMESTAMP(3))
.primaryKey("f0", "f1") .primaryKey("f0", "f1")
.build(); .build();
final MockSourceContext sourceContext2 = MockSourceContext.getInstance(this.conf, schema2, "f2"); final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema2, "f2");
final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createTableSource(sourceContext2); final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2);
final Configuration conf2 = tableSource2.getConf(); final Configuration conf2 = tableSource2.getConf();
assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1")); assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS), is(ComplexAvroKeyGenerator.class.getName())); assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS), is(ComplexAvroKeyGenerator.class.getName()));
@@ -126,14 +125,14 @@ public class TestHoodieTableFactory {
void testInferAvroSchemaForSink() { void testInferAvroSchemaForSink() {
// infer the schema if not specified // infer the schema if not specified
final HoodieTableSink tableSink1 = final HoodieTableSink tableSink1 =
(HoodieTableSink) new HoodieTableFactory().createTableSink(MockSinkContext.getInstance(this.conf)); (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf));
final Configuration conf1 = tableSink1.getConf(); final Configuration conf1 = tableSink1.getConf();
assertThat(conf1.get(FlinkOptions.READ_AVRO_SCHEMA), is(INFERRED_SCHEMA)); assertThat(conf1.get(FlinkOptions.READ_AVRO_SCHEMA), is(INFERRED_SCHEMA));
// set up the explicit schema using the file path // set up the explicit schema using the file path
this.conf.setString(FlinkOptions.READ_AVRO_SCHEMA_PATH, AVRO_SCHEMA_FILE_PATH); this.conf.setString(FlinkOptions.READ_AVRO_SCHEMA_PATH, AVRO_SCHEMA_FILE_PATH);
HoodieTableSink tableSink2 = HoodieTableSink tableSink2 =
(HoodieTableSink) new HoodieTableFactory().createTableSink(MockSinkContext.getInstance(this.conf)); (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf));
Configuration conf2 = tableSink2.getConf(); Configuration conf2 = tableSink2.getConf();
assertNull(conf2.get(FlinkOptions.READ_AVRO_SCHEMA), "expect schema string as null"); assertNull(conf2.get(FlinkOptions.READ_AVRO_SCHEMA), "expect schema string as null");
} }
@@ -149,8 +148,8 @@ public class TestHoodieTableFactory {
.field("f2", DataTypes.TIMESTAMP(3)) .field("f2", DataTypes.TIMESTAMP(3))
.primaryKey("f0") .primaryKey("f0")
.build(); .build();
final MockSinkContext sinkContext1 = MockSinkContext.getInstance(this.conf, schema1, "f2"); final MockContext sinkContext1 = MockContext.getInstance(this.conf, schema1, "f2");
final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createTableSink(sinkContext1); final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext1);
final Configuration conf1 = tableSink1.getConf(); final Configuration conf1 = tableSink1.getConf();
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0")); assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass")); assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass"));
@@ -163,8 +162,8 @@ public class TestHoodieTableFactory {
.field("f2", DataTypes.TIMESTAMP(3)) .field("f2", DataTypes.TIMESTAMP(3))
.primaryKey("f0", "f1") .primaryKey("f0", "f1")
.build(); .build();
final MockSinkContext sinkContext2 = MockSinkContext.getInstance(this.conf, schema2, "f2"); final MockContext sinkContext2 = MockContext.getInstance(this.conf, schema2, "f2");
final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createTableSink(sinkContext2); final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext2);
final Configuration conf2 = tableSink2.getConf(); final Configuration conf2 = tableSink2.getConf();
assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1")); assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS), is(ComplexAvroKeyGenerator.class.getName())); assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS), is(ComplexAvroKeyGenerator.class.getName()));
@@ -175,29 +174,29 @@ public class TestHoodieTableFactory {
// ------------------------------------------------------------------------- // -------------------------------------------------------------------------
/** /**
* Mock context for table source. * Mock dynamic table factory context.
*/ */
private static class MockSourceContext implements TableSourceFactory.Context { private static class MockContext implements DynamicTableFactory.Context {
private final Configuration conf; private final Configuration conf;
private final TableSchema schema; private final TableSchema schema;
private final List<String> partitions; private final List<String> partitions;
private MockSourceContext(Configuration conf, TableSchema schema, List<String> partitions) { private MockContext(Configuration conf, TableSchema schema, List<String> partitions) {
this.conf = conf; this.conf = conf;
this.schema = schema; this.schema = schema;
this.partitions = partitions; this.partitions = partitions;
} }
static MockSourceContext getInstance(Configuration conf) { static MockContext getInstance(Configuration conf) {
return getInstance(conf, TestConfigurations.TABLE_SCHEMA, Collections.singletonList("partition")); return getInstance(conf, TestConfigurations.TABLE_SCHEMA, Collections.singletonList("partition"));
} }
static MockSourceContext getInstance(Configuration conf, TableSchema schema, String partition) { static MockContext getInstance(Configuration conf, TableSchema schema, String partition) {
return getInstance(conf, schema, Collections.singletonList(partition)); return getInstance(conf, schema, Collections.singletonList(partition));
} }
static MockSourceContext getInstance(Configuration conf, TableSchema schema, List<String> partitions) { static MockContext getInstance(Configuration conf, TableSchema schema, List<String> partitions) {
return new MockSourceContext(conf, schema, partitions); return new MockContext(conf, schema, partitions);
} }
@Override @Override
@@ -206,7 +205,7 @@ public class TestHoodieTableFactory {
} }
@Override @Override
public CatalogTable getTable() { public CatalogTable getCatalogTable() {
return new CatalogTableImpl(schema, partitions, conf.toMap(), "mock source table"); return new CatalogTableImpl(schema, partitions, conf.toMap(), "mock source table");
} }
@@ -214,51 +213,14 @@ public class TestHoodieTableFactory {
public ReadableConfig getConfiguration() { public ReadableConfig getConfiguration() {
return conf; return conf;
} }
}
/** @Override
* Mock context for table sink. public ClassLoader getClassLoader() {
*/ return null;
private static class MockSinkContext implements TableSinkFactory.Context {
private final Configuration conf;
private final TableSchema schema;
private final List<String> partitions;
private MockSinkContext(Configuration conf, TableSchema schema, List<String> partitions) {
this.conf = conf;
this.schema = schema;
this.partitions = partitions;
}
static MockSinkContext getInstance(Configuration conf) {
return getInstance(conf, TestConfigurations.TABLE_SCHEMA, "partition");
}
static MockSinkContext getInstance(Configuration conf, TableSchema schema, String partition) {
return getInstance(conf, schema, Collections.singletonList(partition));
}
static MockSinkContext getInstance(Configuration conf, TableSchema schema, List<String> partitions) {
return new MockSinkContext(conf, schema, partitions);
} }
@Override @Override
public ObjectIdentifier getObjectIdentifier() { public boolean isTemporary() {
return ObjectIdentifier.of("hudi", "default", "t1");
}
@Override
public CatalogTable getTable() {
return new CatalogTableImpl(this.schema, this.partitions, conf.toMap(), "mock sink table");
}
@Override
public ReadableConfig getConfiguration() {
return conf;
}
@Override
public boolean isBounded() {
return false; return false;
} }
} }

View File

@@ -88,8 +88,7 @@ public class TestHoodieTableSource {
Map<String, String> partitions = new HashMap<>(); Map<String, String> partitions = new HashMap<>();
partitions.put("partition", "par1"); partitions.put("partition", "par1");
tableSource = (HoodieTableSource) tableSource tableSource.applyPartitions(Collections.singletonList(partitions));
.applyPartitionPruning(Collections.singletonList(partitions));
Path[] paths2 = tableSource.getReadPaths(); Path[] paths2 = tableSource.getReadPaths();
assertNotNull(paths2); assertNotNull(paths2);

View File

@@ -161,9 +161,8 @@ public class TestInputFormat {
Map<String, String> prunedPartitions = new HashMap<>(); Map<String, String> prunedPartitions = new HashMap<>();
prunedPartitions.put("partition", "par1"); prunedPartitions.put("partition", "par1");
// prune to only be with partition 'par1' // prune to only be with partition 'par1'
HoodieTableSource newSource = (HoodieTableSource) tableSource tableSource.applyPartitions(Collections.singletonList(prunedPartitions));
.applyPartitionPruning(Collections.singletonList(prunedPartitions)); InputFormat<RowData, ?> inputFormat = tableSource.getInputFormat();
InputFormat<RowData, ?> inputFormat = newSource.getInputFormat();
List<RowData> result = readData(inputFormat); List<RowData> result = readData(inputFormat);

View File

@@ -23,7 +23,6 @@ import org.apache.hudi.streamer.FlinkStreamerConfig;
import org.apache.hudi.utils.factory.CollectSinkTableFactory; import org.apache.hudi.utils.factory.CollectSinkTableFactory;
import org.apache.hudi.utils.factory.ContinuousFileSourceFactory; import org.apache.hudi.utils.factory.ContinuousFileSourceFactory;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
@@ -119,7 +118,7 @@ public class TestConfigurations {
+ ")"; + ")";
} }
public static final RowDataSerializer SERIALIZER = new RowDataSerializer(new ExecutionConfig(), ROW_TYPE); public static final RowDataSerializer SERIALIZER = new RowDataSerializer(ROW_TYPE);
public static Configuration getDefaultConf(String tablePath) { public static Configuration getDefaultConf(String tablePath) {
Configuration conf = new Configuration(); Configuration conf = new Configuration();

View File

@@ -41,7 +41,7 @@ import org.apache.flink.table.data.conversion.DataStructureConverter;
import org.apache.flink.table.data.conversion.DataStructureConverters; import org.apache.flink.table.data.conversion.DataStructureConverters;
import org.apache.flink.table.data.writer.BinaryRowWriter; import org.apache.flink.table.data.writer.BinaryRowWriter;
import org.apache.flink.table.data.writer.BinaryWriter; import org.apache.flink.table.data.writer.BinaryWriter;
import org.apache.flink.table.runtime.types.InternalSerializers; import org.apache.flink.table.runtime.typeutils.InternalSerializers;
import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.types.Row; import org.apache.flink.types.Row;

View File

@@ -103,7 +103,11 @@ public class CollectSinkTableFactory implements DynamicTableSinkFactory {
@Override @Override
public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
return ChangelogMode.insertOnly(); return ChangelogMode.newBuilder()
.addContainedKind(RowKind.INSERT)
.addContainedKind(RowKind.DELETE)
.addContainedKind(RowKind.UPDATE_AFTER)
.build();
} }
@Override @Override

View File

@@ -21,42 +21,46 @@ package org.apache.hudi.utils.factory;
import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.utils.source.ContinuousFileSource; import org.apache.hudi.utils.source.ContinuousFileSource;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.Path;
import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.data.RowData; import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.factories.DynamicTableSourceFactory;
import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.FactoryUtil;
import org.apache.flink.table.factories.TableSourceFactory;
import org.apache.flink.table.sources.TableSource;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.Set;
import java.util.List;
import java.util.Map;
/** /**
* Factory for ContinuousFileSource. * Factory for ContinuousFileSource.
*/ */
public class ContinuousFileSourceFactory implements TableSourceFactory<RowData> { public class ContinuousFileSourceFactory implements DynamicTableSourceFactory {
public static final String FACTORY_ID = "continuous-file-source"; public static final String FACTORY_ID = "continuous-file-source";
@Override @Override
public TableSource<RowData> createTableSource(Context context) { public DynamicTableSource createDynamicTableSource(Context context) {
Configuration conf = FlinkOptions.fromMap(context.getTable().getOptions()); FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
helper.validate();
Configuration conf = (Configuration) helper.getOptions();
Path path = new Path(conf.getOptional(FlinkOptions.PATH).orElseThrow(() -> Path path = new Path(conf.getOptional(FlinkOptions.PATH).orElseThrow(() ->
new ValidationException("Option [path] should be not empty."))); new ValidationException("Option [path] should be not empty.")));
return new ContinuousFileSource(context.getTable().getSchema(), path, conf); return new ContinuousFileSource(context.getCatalogTable().getSchema(), path, conf);
} }
@Override @Override
public Map<String, String> requiredContext() { public String factoryIdentifier() {
Map<String, String> context = new HashMap<>(); return FACTORY_ID;
context.put(FactoryUtil.CONNECTOR.key(), FACTORY_ID);
return context;
} }
@Override @Override
public List<String> supportedProperties() { public Set<ConfigOption<?>> requiredOptions() {
return Collections.singletonList("*"); return Collections.emptySet();
}
@Override
public Set<ConfigOption<?>> optionalOptions() {
return Collections.singleton(FlinkOptions.PATH);
} }
} }

View File

@@ -27,10 +27,12 @@ import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.RowData;
import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.sources.StreamTableSource;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.RowType;
import java.io.IOException; import java.io.IOException;
@@ -53,7 +55,7 @@ import java.util.concurrent.atomic.AtomicInteger;
* *
* <p>If all the data are flushed out, it waits for the next checkpoint to finish and tear down the source. * <p>If all the data are flushed out, it waits for the next checkpoint to finish and tear down the source.
*/ */
public class ContinuousFileSource implements StreamTableSource<RowData> { public class ContinuousFileSource implements ScanTableSource {
private final TableSchema tableSchema; private final TableSchema tableSchema;
private final Path path; private final Path path;
@@ -69,30 +71,46 @@ public class ContinuousFileSource implements StreamTableSource<RowData> {
} }
@Override @Override
public DataStream<RowData> getDataStream(StreamExecutionEnvironment execEnv) { public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
final RowType rowType = (RowType) this.tableSchema.toRowDataType().getLogicalType(); return new DataStreamScanProvider() {
@Override
public boolean isBounded() {
return false;
}
@Override
public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv) {
final RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema( JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
rowType, rowType,
new RowDataTypeInfo(rowType), InternalTypeInfo.of(rowType),
false, false,
true, true,
TimestampFormat.ISO_8601); TimestampFormat.ISO_8601);
return execEnv.addSource(new BoundedSourceFunction(this.path, 2)) return execEnv.addSource(new BoundedSourceFunction(path, 2))
.name("continuous_file_source") .name("continuous_file_source")
.setParallelism(1) .setParallelism(1)
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)), .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)),
new RowDataTypeInfo(rowType)); InternalTypeInfo.of(rowType));
}
};
} }
@Override @Override
public TableSchema getTableSchema() { public ChangelogMode getChangelogMode() {
return this.tableSchema; return ChangelogMode.insertOnly();
} }
@Override @Override
public DataType getProducedDataType() { public DynamicTableSource copy() {
return this.tableSchema.toRowDataType().bridgedTo(RowData.class); return new ContinuousFileSource(this.tableSchema, this.path, this.conf);
}
@Override
public String asSummaryString() {
return "ContinuousFileSource";
} }
/** /**

View File

@@ -14,4 +14,5 @@
# See the License for the specific language governing permissions and # See the License for the specific language governing permissions and
# limitations under the License. # limitations under the License.
org.apache.hudi.utils.factory.ContinuousFileSourceFactory
org.apache.hudi.utils.factory.CollectSinkTableFactory org.apache.hudi.utils.factory.CollectSinkTableFactory

View File

@@ -1,18 +0,0 @@
# 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.
org.apache.hudi.table.HoodieTableFactory
org.apache.hudi.utils.factory.ContinuousFileSourceFactory

View File

@@ -33,6 +33,8 @@
<main.basedir>${project.parent.basedir}</main.basedir> <main.basedir>${project.parent.basedir}</main.basedir>
<flink.bundle.hive.shade.prefix>org.apache.hudi.</flink.bundle.hive.shade.prefix> <flink.bundle.hive.shade.prefix>org.apache.hudi.</flink.bundle.hive.shade.prefix>
<javax.servlet.version>3.1.0</javax.servlet.version> <javax.servlet.version>3.1.0</javax.servlet.version>
<!-- override to be same with flink 1.12.2 -->
<parquet.version>1.11.1</parquet.version>
</properties> </properties>
<build> <build>
@@ -89,8 +91,9 @@
<include>org.apache.parquet:parquet-hadoop</include> <include>org.apache.parquet:parquet-hadoop</include>
<include>org.apache.parquet:parquet-column</include> <include>org.apache.parquet:parquet-column</include>
<include>org.apache.parquet:parquet-common</include> <include>org.apache.parquet:parquet-common</include>
<include>org.apache.parquet:parquet-format</include> <include>org.apache.parquet:parquet-format-structures</include>
<include>org.apache.parquet:parquet-encoding</include> <include>org.apache.parquet:parquet-encoding</include>
<include>org.apache.parquet:parquet-jackson</include>
<include>org.apache.avro:avro</include> <include>org.apache.avro:avro</include>
<include>joda-time:joda-time</include> <include>joda-time:joda-time</include>
@@ -100,11 +103,6 @@
<include>com.twitter:bijection-avro_${scala.binary.version}</include> <include>com.twitter:bijection-avro_${scala.binary.version}</include>
<include>com.twitter:bijection-core_${scala.binary.version}</include> <include>com.twitter:bijection-core_${scala.binary.version}</include>
<include>io.confluent:kafka-avro-serializer</include>
<include>io.confluent:common-config</include>
<include>io.confluent:common-utils</include>
<include>io.confluent:kafka-schema-registry-client</include>
<include>org.apache.kafka:kafka-clients</include>
<include>io.dropwizard.metrics:metrics-core</include> <include>io.dropwizard.metrics:metrics-core</include>
<include>io.dropwizard.metrics:metrics-graphite</include> <include>io.dropwizard.metrics:metrics-graphite</include>
<include>io.prometheus:simpleclient</include> <include>io.prometheus:simpleclient</include>
@@ -113,16 +111,12 @@
<include>io.prometheus:simpleclient_pushgateway</include> <include>io.prometheus:simpleclient_pushgateway</include>
<include>io.prometheus:simpleclient_common</include> <include>io.prometheus:simpleclient_common</include>
<include>com.yammer.metrics:metrics-core</include> <include>com.yammer.metrics:metrics-core</include>
<include>org.apache.kafka:kafka_${scala.binary.version}</include>
<include>com.101tec:zkclient</include>
<!-- Used for HUDI TimelineService --> <!-- Used for HUDI TimelineService -->
<include>org.eclipse.jetty:*</include> <include>org.eclipse.jetty:*</include>
<include>org.eclipse.jetty.websocket:*</include> <include>org.eclipse.jetty.websocket:*</include>
<include>javax.servlet:javax.servlet-api</include> <include>javax.servlet:javax.servlet-api</include>
<include>org.apache.flink:flink-connector-kafka_${scala.binary.version}</include>
<include>org.apache.flink:flink-connector-kafka-base_${scala.binary.version}</include>
<include>org.apache.flink:flink-hadoop-compatibility_${scala.binary.version}</include> <include>org.apache.flink:flink-hadoop-compatibility_${scala.binary.version}</include>
<include>org.apache.flink:flink-avro</include> <include>org.apache.flink:flink-avro</include>
<include>org.apache.flink:flink-json</include> <include>org.apache.flink:flink-json</include>
@@ -133,14 +127,18 @@
<include>org.apache.hive:hive-metastore</include> <include>org.apache.hive:hive-metastore</include>
<include>org.apache.hive:hive-jdbc</include> <include>org.apache.hive:hive-jdbc</include>
<include>org.apache.hbase:hbase-client</include>
<include>org.apache.hbase:hbase-common</include> <include>org.apache.hbase:hbase-common</include>
<include>org.apache.hbase:hbase-protocol</include>
<include>org.apache.hbase:hbase-server</include>
<include>org.apache.htrace:htrace-core</include>
</includes> </includes>
</artifactSet> </artifactSet>
<relocations> <relocations>
<relocation>
<pattern>org.apache.avro.</pattern>
<shadedPattern>${flink.bundle.hive.shade.prefix}org.apache.avro.</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.parquet.</pattern>
<shadedPattern>${flink.bundle.hive.shade.prefix}org.apache.parquet.</shadedPattern>
</relocation>
<relocation> <relocation>
<pattern>com.yammer.metrics.</pattern> <pattern>com.yammer.metrics.</pattern>
<shadedPattern>org.apache.hudi.com.yammer.metrics.</shadedPattern> <shadedPattern>org.apache.hudi.com.yammer.metrics.</shadedPattern>
@@ -198,8 +196,6 @@
<exclude>META-INF/*.DSA</exclude> <exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude> <exclude>META-INF/*.RSA</exclude>
<exclude>META-INF/services/javax.*</exclude> <exclude>META-INF/services/javax.*</exclude>
<!-- exclude the test dynamic factory -->
<exclude>META-INF/services/org.apache.flink.table.factories.Factory</exclude>
</excludes> </excludes>
</filter> </filter>
</filters> </filters>
@@ -258,17 +254,6 @@
</dependency> </dependency>
<!-- Flink --> <!-- Flink -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-kafka_${scala.binary.version}</artifactId>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-kafka-base_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>compile</scope>
</dependency>
<dependency> <dependency>
<groupId>org.apache.flink</groupId> <groupId>org.apache.flink</groupId>
<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId> <artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
@@ -295,9 +280,62 @@
</dependency> </dependency>
<!-- Parquet --> <!-- Parquet -->
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-hadoop</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope>
<exclusions>
<exclusion>
<groupId>org.xerial.snappy</groupId>
<artifactId>snappy-java</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency> <dependency>
<groupId>org.apache.parquet</groupId> <groupId>org.apache.parquet</groupId>
<artifactId>parquet-avro</artifactId> <artifactId>parquet-avro</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope>
<exclusions>
<exclusion>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
</exclusion>
<exclusion>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-column</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-common</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-encoding</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-format-structures</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-jackson</artifactId>
<version>${parquet.version}</version>
<scope>compile</scope> <scope>compile</scope>
</dependency> </dependency>
@@ -391,52 +429,11 @@
<scope>compile</scope> <scope>compile</scope>
</dependency> </dependency>
<dependency>
<groupId>org.apache.htrace</groupId>
<artifactId>htrace-core</artifactId>
<version>${htrace.version}</version>
<scope>compile</scope>
</dependency>
<!-- Hbase --> <!-- Hbase -->
<dependency> <dependency>
<groupId>org.apache.hbase</groupId> <groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId> <artifactId>hbase-common</artifactId>
<version>${hbase.version}</version> <version>${hbase.version}</version>
</dependency> </dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-server</artifactId>
<version>${hbase.version}</version>
<scope>compile</scope>
<exclusions>
<exclusion>
<groupId>javax.servlet</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>org.codehaus.jackson</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>org.mortbay.jetty</groupId>
<artifactId>*</artifactId>
</exclusion>
<exclusion>
<groupId>tomcat</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
<version>${hbase.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-protocol</artifactId>
<version>${hbase.version}</version>
</dependency>
</dependencies> </dependencies>
</project> </project>

View File

@@ -107,7 +107,7 @@
<http.version>4.4.1</http.version> <http.version>4.4.1</http.version>
<spark.version>${spark2.version}</spark.version> <spark.version>${spark2.version}</spark.version>
<sparkbundle.version>${spark2bundle.version}</sparkbundle.version> <sparkbundle.version>${spark2bundle.version}</sparkbundle.version>
<flink.version>1.11.2</flink.version> <flink.version>1.12.2</flink.version>
<spark2.version>2.4.4</spark2.version> <spark2.version>2.4.4</spark2.version>
<spark3.version>3.0.0</spark3.version> <spark3.version>3.0.0</spark3.version>
<spark2bundle.version></spark2bundle.version> <spark2bundle.version></spark2bundle.version>

View File

@@ -28,5 +28,6 @@
<suppress checks="IllegalImport" files="Option.java" /> <suppress checks="IllegalImport" files="Option.java" />
<!-- java.util.Optional part of DataSource V2 API signature --> <!-- java.util.Optional part of DataSource V2 API signature -->
<suppress checks="IllegalImport" files="DefaultSource.java" /> <suppress checks="IllegalImport" files="DefaultSource.java" />
<suppress checks="IllegalImport" files="HoodieTableSource.java" />
<suppress checks="IllegalTokenText" files="FilePathUtils.java"/> <suppress checks="IllegalTokenText" files="FilePathUtils.java"/>
</suppressions> </suppressions>