[HUDI-1495] Bump Flink version to 1.12.2 (#2718)
This commit is contained in:
@@ -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.functions.source.FileProcessingMode;
|
||||
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.util.TestLogger;
|
||||
import org.junit.jupiter.api.Test;
|
||||
@@ -104,7 +104,7 @@ public class StreamWriteITCase extends TestLogger {
|
||||
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
new RowDataTypeInfo(rowType),
|
||||
InternalTypeInfo.of(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601
|
||||
@@ -135,7 +135,7 @@ public class StreamWriteITCase extends TestLogger {
|
||||
|
||||
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
|
||||
// wait for the streaming job to finish
|
||||
client.getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
|
||||
client.getJobExecutionResult().get();
|
||||
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED);
|
||||
}
|
||||
@@ -159,7 +159,7 @@ public class StreamWriteITCase extends TestLogger {
|
||||
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
new RowDataTypeInfo(rowType),
|
||||
InternalTypeInfo.of(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601
|
||||
@@ -204,7 +204,7 @@ public class StreamWriteITCase extends TestLogger {
|
||||
|
||||
JobClient client = execEnv.executeAsync(execEnv.getStreamGraph(conf.getString(FlinkOptions.TABLE_NAME)));
|
||||
// wait for the streaming job to finish
|
||||
client.getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
|
||||
client.getJobExecutionResult().get();
|
||||
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED);
|
||||
}
|
||||
@@ -230,7 +230,7 @@ public class StreamWriteITCase extends TestLogger {
|
||||
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
new RowDataTypeInfo(rowType),
|
||||
InternalTypeInfo.of(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601
|
||||
|
||||
@@ -94,7 +94,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
coordinator.handleEventFromOperator(0, event0);
|
||||
coordinator.handleEventFromOperator(1, event1);
|
||||
|
||||
coordinator.checkpointComplete(1);
|
||||
coordinator.notifyCheckpointComplete(1);
|
||||
String inflight = coordinator.getWriteClient()
|
||||
.getInflightAndRequestedInstant("COPY_ON_WRITE");
|
||||
String lastCompleted = coordinator.getWriteClient().getLastCompletedInstant("COPY_ON_WRITE");
|
||||
@@ -116,7 +116,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
public void testCheckpointAndRestore() throws Exception {
|
||||
CompletableFuture<byte[]> future = new CompletableFuture<>();
|
||||
coordinator.checkpointCoordinator(1, future);
|
||||
coordinator.resetToCheckpoint(future.get());
|
||||
coordinator.resetToCheckpoint(1, future.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -145,7 +145,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
.build();
|
||||
coordinator.handleEventFromOperator(0, event);
|
||||
assertThrows(HoodieException.class,
|
||||
() -> coordinator.checkpointComplete(1),
|
||||
() -> coordinator.notifyCheckpointComplete(1),
|
||||
"Try 3 to commit instant");
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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.AggregatingStateDescriptor;
|
||||
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.ListState;
|
||||
import org.apache.flink.api.common.state.ListStateDescriptor;
|
||||
@@ -89,11 +87,6 @@ public class MockOperatorStateStore implements KeyedStateStore, OperatorStateSto
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T, A> FoldingState<T, A> getFoldingState(FoldingStateDescriptor<T, A> foldingStateDescriptor) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public <K, V> MapState<K, V> getMapState(MapStateDescriptor<K, V> mapStateDescriptor) {
|
||||
|
||||
@@ -156,7 +156,7 @@ public class StreamWriteFunctionWrapper<I> {
|
||||
|
||||
public void checkpointComplete(long checkpointId) {
|
||||
functionInitializationContext.getOperatorStateStore().checkpointSuccess(checkpointId);
|
||||
coordinator.checkpointComplete(checkpointId);
|
||||
coordinator.notifyCheckpointComplete(checkpointId);
|
||||
this.bucketAssignerFunction.notifyCheckpointComplete(checkpointId);
|
||||
this.writeFunction.notifyCheckpointComplete(checkpointId);
|
||||
if (conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED)) {
|
||||
|
||||
@@ -218,8 +218,7 @@ public class HoodieDataSourceITCase extends AbstractTestBase {
|
||||
TableResult tableResult = tEnv.executeSql(insert);
|
||||
// wait to finish
|
||||
try {
|
||||
tableResult.getJobClient().get()
|
||||
.getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
|
||||
tableResult.getJobClient().get().getJobExecutionResult().get();
|
||||
} catch (InterruptedException | ExecutionException ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
|
||||
@@ -30,8 +30,7 @@ import org.apache.flink.table.api.TableSchema;
|
||||
import org.apache.flink.table.catalog.CatalogTable;
|
||||
import org.apache.flink.table.catalog.CatalogTableImpl;
|
||||
import org.apache.flink.table.catalog.ObjectIdentifier;
|
||||
import org.apache.flink.table.factories.TableSinkFactory;
|
||||
import org.apache.flink.table.factories.TableSourceFactory;
|
||||
import org.apache.flink.table.factories.DynamicTableFactory;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
@@ -78,14 +77,14 @@ public class TestHoodieTableFactory {
|
||||
void testInferAvroSchemaForSource() {
|
||||
// infer the schema if not specified
|
||||
final HoodieTableSource tableSource1 =
|
||||
(HoodieTableSource) new HoodieTableFactory().createTableSource(MockSourceContext.getInstance(this.conf));
|
||||
(HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(MockContext.getInstance(this.conf));
|
||||
final Configuration conf1 = tableSource1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.READ_AVRO_SCHEMA), is(INFERRED_SCHEMA));
|
||||
|
||||
// set up the explicit schema using the file path
|
||||
this.conf.setString(FlinkOptions.READ_AVRO_SCHEMA_PATH, AVRO_SCHEMA_FILE_PATH);
|
||||
HoodieTableSource tableSource2 =
|
||||
(HoodieTableSource) new HoodieTableFactory().createTableSource(MockSourceContext.getInstance(this.conf));
|
||||
(HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(MockContext.getInstance(this.conf));
|
||||
Configuration conf2 = tableSource2.getConf();
|
||||
assertNull(conf2.get(FlinkOptions.READ_AVRO_SCHEMA), "expect schema string as null");
|
||||
}
|
||||
@@ -101,8 +100,8 @@ public class TestHoodieTableFactory {
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
final MockSourceContext sourceContext1 = MockSourceContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createTableSource(sourceContext1);
|
||||
final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
|
||||
final Configuration conf1 = tableSource1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
|
||||
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass"));
|
||||
@@ -115,8 +114,8 @@ public class TestHoodieTableFactory {
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0", "f1")
|
||||
.build();
|
||||
final MockSourceContext sourceContext2 = MockSourceContext.getInstance(this.conf, schema2, "f2");
|
||||
final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createTableSource(sourceContext2);
|
||||
final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema2, "f2");
|
||||
final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2);
|
||||
final Configuration conf2 = tableSource2.getConf();
|
||||
assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
|
||||
assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS), is(ComplexAvroKeyGenerator.class.getName()));
|
||||
@@ -126,14 +125,14 @@ public class TestHoodieTableFactory {
|
||||
void testInferAvroSchemaForSink() {
|
||||
// infer the schema if not specified
|
||||
final HoodieTableSink tableSink1 =
|
||||
(HoodieTableSink) new HoodieTableFactory().createTableSink(MockSinkContext.getInstance(this.conf));
|
||||
(HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf));
|
||||
final Configuration conf1 = tableSink1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.READ_AVRO_SCHEMA), is(INFERRED_SCHEMA));
|
||||
|
||||
// set up the explicit schema using the file path
|
||||
this.conf.setString(FlinkOptions.READ_AVRO_SCHEMA_PATH, AVRO_SCHEMA_FILE_PATH);
|
||||
HoodieTableSink tableSink2 =
|
||||
(HoodieTableSink) new HoodieTableFactory().createTableSink(MockSinkContext.getInstance(this.conf));
|
||||
(HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(MockContext.getInstance(this.conf));
|
||||
Configuration conf2 = tableSink2.getConf();
|
||||
assertNull(conf2.get(FlinkOptions.READ_AVRO_SCHEMA), "expect schema string as null");
|
||||
}
|
||||
@@ -149,8 +148,8 @@ public class TestHoodieTableFactory {
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0")
|
||||
.build();
|
||||
final MockSinkContext sinkContext1 = MockSinkContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createTableSink(sinkContext1);
|
||||
final MockContext sinkContext1 = MockContext.getInstance(this.conf, schema1, "f2");
|
||||
final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext1);
|
||||
final Configuration conf1 = tableSink1.getConf();
|
||||
assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
|
||||
assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass"));
|
||||
@@ -163,8 +162,8 @@ public class TestHoodieTableFactory {
|
||||
.field("f2", DataTypes.TIMESTAMP(3))
|
||||
.primaryKey("f0", "f1")
|
||||
.build();
|
||||
final MockSinkContext sinkContext2 = MockSinkContext.getInstance(this.conf, schema2, "f2");
|
||||
final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createTableSink(sinkContext2);
|
||||
final MockContext sinkContext2 = MockContext.getInstance(this.conf, schema2, "f2");
|
||||
final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext2);
|
||||
final Configuration conf2 = tableSink2.getConf();
|
||||
assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
|
||||
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 TableSchema schema;
|
||||
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.schema = schema;
|
||||
this.partitions = partitions;
|
||||
}
|
||||
|
||||
static MockSourceContext getInstance(Configuration conf) {
|
||||
static MockContext getInstance(Configuration conf) {
|
||||
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));
|
||||
}
|
||||
|
||||
static MockSourceContext getInstance(Configuration conf, TableSchema schema, List<String> partitions) {
|
||||
return new MockSourceContext(conf, schema, partitions);
|
||||
static MockContext getInstance(Configuration conf, TableSchema schema, List<String> partitions) {
|
||||
return new MockContext(conf, schema, partitions);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -206,7 +205,7 @@ public class TestHoodieTableFactory {
|
||||
}
|
||||
|
||||
@Override
|
||||
public CatalogTable getTable() {
|
||||
public CatalogTable getCatalogTable() {
|
||||
return new CatalogTableImpl(schema, partitions, conf.toMap(), "mock source table");
|
||||
}
|
||||
|
||||
@@ -214,51 +213,14 @@ public class TestHoodieTableFactory {
|
||||
public ReadableConfig getConfiguration() {
|
||||
return conf;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Mock context for table sink.
|
||||
*/
|
||||
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
|
||||
public ClassLoader getClassLoader() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ObjectIdentifier getObjectIdentifier() {
|
||||
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() {
|
||||
public boolean isTemporary() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -88,8 +88,7 @@ public class TestHoodieTableSource {
|
||||
Map<String, String> partitions = new HashMap<>();
|
||||
partitions.put("partition", "par1");
|
||||
|
||||
tableSource = (HoodieTableSource) tableSource
|
||||
.applyPartitionPruning(Collections.singletonList(partitions));
|
||||
tableSource.applyPartitions(Collections.singletonList(partitions));
|
||||
|
||||
Path[] paths2 = tableSource.getReadPaths();
|
||||
assertNotNull(paths2);
|
||||
|
||||
@@ -161,9 +161,8 @@ public class TestInputFormat {
|
||||
Map<String, String> prunedPartitions = new HashMap<>();
|
||||
prunedPartitions.put("partition", "par1");
|
||||
// prune to only be with partition 'par1'
|
||||
HoodieTableSource newSource = (HoodieTableSource) tableSource
|
||||
.applyPartitionPruning(Collections.singletonList(prunedPartitions));
|
||||
InputFormat<RowData, ?> inputFormat = newSource.getInputFormat();
|
||||
tableSource.applyPartitions(Collections.singletonList(prunedPartitions));
|
||||
InputFormat<RowData, ?> inputFormat = tableSource.getInputFormat();
|
||||
|
||||
List<RowData> result = readData(inputFormat);
|
||||
|
||||
|
||||
@@ -23,7 +23,6 @@ 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.ExecutionConfig;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.api.common.typeinfo.Types;
|
||||
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) {
|
||||
Configuration conf = new Configuration();
|
||||
|
||||
@@ -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.writer.BinaryRowWriter;
|
||||
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.RowType;
|
||||
import org.apache.flink.types.Row;
|
||||
|
||||
@@ -103,7 +103,11 @@ public class CollectSinkTableFactory implements DynamicTableSinkFactory {
|
||||
|
||||
@Override
|
||||
public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
|
||||
return ChangelogMode.insertOnly();
|
||||
return ChangelogMode.newBuilder()
|
||||
.addContainedKind(RowKind.INSERT)
|
||||
.addContainedKind(RowKind.DELETE)
|
||||
.addContainedKind(RowKind.UPDATE_AFTER)
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -21,42 +21,46 @@ package org.apache.hudi.utils.factory;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.utils.source.ContinuousFileSource;
|
||||
|
||||
import org.apache.flink.configuration.ConfigOption;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.core.fs.Path;
|
||||
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.TableSourceFactory;
|
||||
import org.apache.flink.table.sources.TableSource;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Factory for ContinuousFileSource.
|
||||
*/
|
||||
public class ContinuousFileSourceFactory implements TableSourceFactory<RowData> {
|
||||
public class ContinuousFileSourceFactory implements DynamicTableSourceFactory {
|
||||
public static final String FACTORY_ID = "continuous-file-source";
|
||||
|
||||
@Override
|
||||
public TableSource<RowData> createTableSource(Context context) {
|
||||
Configuration conf = FlinkOptions.fromMap(context.getTable().getOptions());
|
||||
public DynamicTableSource createDynamicTableSource(Context context) {
|
||||
FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
|
||||
helper.validate();
|
||||
|
||||
Configuration conf = (Configuration) helper.getOptions();
|
||||
Path path = new Path(conf.getOptional(FlinkOptions.PATH).orElseThrow(() ->
|
||||
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
|
||||
public Map<String, String> requiredContext() {
|
||||
Map<String, String> context = new HashMap<>();
|
||||
context.put(FactoryUtil.CONNECTOR.key(), FACTORY_ID);
|
||||
return context;
|
||||
public String factoryIdentifier() {
|
||||
return FACTORY_ID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> supportedProperties() {
|
||||
return Collections.singletonList("*");
|
||||
public Set<ConfigOption<?>> requiredOptions() {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<ConfigOption<?>> optionalOptions() {
|
||||
return Collections.singleton(FlinkOptions.PATH);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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.functions.source.SourceFunction;
|
||||
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.runtime.typeutils.RowDataTypeInfo;
|
||||
import org.apache.flink.table.sources.StreamTableSource;
|
||||
import org.apache.flink.table.types.DataType;
|
||||
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
|
||||
import org.apache.flink.table.types.logical.RowType;
|
||||
|
||||
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.
|
||||
*/
|
||||
public class ContinuousFileSource implements StreamTableSource<RowData> {
|
||||
public class ContinuousFileSource implements ScanTableSource {
|
||||
|
||||
private final TableSchema tableSchema;
|
||||
private final Path path;
|
||||
@@ -69,30 +71,46 @@ public class ContinuousFileSource implements StreamTableSource<RowData> {
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataStream<RowData> getDataStream(StreamExecutionEnvironment execEnv) {
|
||||
final RowType rowType = (RowType) this.tableSchema.toRowDataType().getLogicalType();
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
new RowDataTypeInfo(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601);
|
||||
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
|
||||
return new DataStreamScanProvider() {
|
||||
|
||||
return execEnv.addSource(new BoundedSourceFunction(this.path, 2))
|
||||
.name("continuous_file_source")
|
||||
.setParallelism(1)
|
||||
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)),
|
||||
new RowDataTypeInfo(rowType));
|
||||
@Override
|
||||
public boolean isBounded() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataStream<RowData> produceDataStream(StreamExecutionEnvironment execEnv) {
|
||||
final RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
|
||||
JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
|
||||
rowType,
|
||||
InternalTypeInfo.of(rowType),
|
||||
false,
|
||||
true,
|
||||
TimestampFormat.ISO_8601);
|
||||
|
||||
return execEnv.addSource(new BoundedSourceFunction(path, 2))
|
||||
.name("continuous_file_source")
|
||||
.setParallelism(1)
|
||||
.map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)),
|
||||
InternalTypeInfo.of(rowType));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public TableSchema getTableSchema() {
|
||||
return this.tableSchema;
|
||||
public ChangelogMode getChangelogMode() {
|
||||
return ChangelogMode.insertOnly();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataType getProducedDataType() {
|
||||
return this.tableSchema.toRowDataType().bridgedTo(RowData.class);
|
||||
public DynamicTableSource copy() {
|
||||
return new ContinuousFileSource(this.tableSchema, this.path, this.conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String asSummaryString() {
|
||||
return "ContinuousFileSource";
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -14,4 +14,5 @@
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
org.apache.hudi.utils.factory.ContinuousFileSourceFactory
|
||||
org.apache.hudi.utils.factory.CollectSinkTableFactory
|
||||
|
||||
@@ -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
|
||||
Reference in New Issue
Block a user