User-defined Sources & Sinks

Dynamic tables are the core concept of Flink’s Table & SQL API for processing both bounded and unbounded data in a unified fashion.

Because dynamic tables are only a logical concept, Flink does not own the data itself. Instead, the content of a dynamic table is stored in external systems (such as databases, key-value stores, message queues) or files.

Dynamic sources and dynamic sinks can be used to read and write data from and to an external system. In the documentation, sources and sinks are often summarized under the term connector.

Flink provides pre-defined connectors for Kafka, Hive, and different file systems. See the connector section for more information about built-in table sources and sinks.

This page focuses on how to develop a custom, user-defined connector.

Overview

In many cases, implementers don’t need to create a new connector from scratch but would like to slightly modify existing connectors or hook into the existing stack. In other cases, implementers would like to create specialized connectors.

This section helps for both kinds of use cases. It explains the general architecture of table connectors from pure declaration in the API to runtime code that will be executed on the cluster.

The filled arrows show how objects are transformed to other objects from one stage to the next stage during the translation process.

Translation of table connectors

Metadata

Both Table API and SQL are declarative APIs. This includes the declaration of tables. Thus, executing a CREATE TABLE statement results in updated metadata in the target catalog.

For most catalog implementations, physical data in the external system is not modified for such an operation. Connector-specific dependencies don’t have to be present in the classpath yet. The options declared in the WITH clause are neither validated nor otherwise interpreted.

The metadata for dynamic tables (created via DDL or provided by the catalog) is represented as instances of CatalogTable. A table name will be resolved into a CatalogTable internally when necessary.

Planning

When it comes to planning and optimization of the table program, a CatalogTable needs to be resolved into a DynamicTableSource (for reading in a SELECT query) and DynamicTableSink (for writing in an INSERT INTO statement).

DynamicTableSourceFactory and DynamicTableSinkFactory provide connector-specific logic for translating the metadata of a CatalogTable into instances of DynamicTableSource and DynamicTableSink. In most of the cases, a factory’s purpose is to validate options (such as 'port' = '5022' in the example), configure encoding/decoding formats (if required), and create a parameterized instance of the table connector.

By default, instances of DynamicTableSourceFactory and DynamicTableSinkFactory are discovered using Java’s Service Provider Interfaces (SPI). The connector option (such as 'connector' = 'custom' in the example) must correspond to a valid factory identifier.

Although it might not be apparent in the class naming, DynamicTableSource and DynamicTableSink can also be seen as stateful factories that eventually produce concrete runtime implementation for reading/writing the actual data.

The planner uses the source and sink instances to perform connector-specific bidirectional communication until an optimal logical plan could be found. Depending on the optionally declared ability interfaces (e.g. SupportsProjectionPushDown or SupportsOverwrite), the planner might apply changes to an instance and thus mutate the produced runtime implementation.

Runtime

Once the logical planning is complete, the planner will obtain the runtime implementation from the table connector. Runtime logic is implemented in Flink’s core connector interfaces such as InputFormat or SourceFunction.

Those interfaces are grouped by another level of abstraction as subclasses of ScanRuntimeProvider, LookupRuntimeProvider, and SinkRuntimeProvider.

For example, both OutputFormatProvider (providing org.apache.flink.api.common.io.OutputFormat) and SinkFunctionProvider (providing org.apache.flink.streaming.api.functions.sink.SinkFunction) are concrete instances of SinkRuntimeProvider that the planner can handle.

Extension Points

This section explains the available interfaces for extending Flink’s table connectors.

Dynamic Table Factories

Dynamic table factories are used to configure a dynamic table connector for an external storage system from catalog and session information.

org.apache.flink.table.factories.DynamicTableSourceFactory can be implemented to construct a DynamicTableSource.

org.apache.flink.table.factories.DynamicTableSinkFactory can be implemented to construct a DynamicTableSink.

By default, the factory is discovered using the value of the connector option as the factory identifier and Java’s Service Provider Interface.

In JAR files, references to new implementations can be added to the service file:

META-INF/services/org.apache.flink.table.factories.Factory

The framework will check for a single matching factory that is uniquely identified by factory identifier and requested base class (e.g. DynamicTableSourceFactory).

The factory discovery process can be bypassed by the catalog implementation if necessary. For this, a catalog needs to return an instance that implements the requested base class in org.apache.flink.table.catalog.Catalog#getFactory.

Dynamic Table Source

By definition, a dynamic table can change over time.

When reading a dynamic table, the content can either be considered as:

  • A changelog (finite or infinite) for which all changes are consumed continuously until the changelog is exhausted. This is represented by the ScanTableSource interface.
  • A continuously changing or very large external table whose content is usually never read entirely but queried for individual values when necessary. This is represented by the LookupTableSource interface.

A class can implement both of these interfaces at the same time. The planner decides about their usage depending on the specified query.

Scan Table Source

A ScanTableSource scans all rows from an external storage system during runtime.

The scanned rows don’t have to contain only insertions but can also contain updates and deletions. Thus, the table source can be used to read a (finite or infinite) changelog. The returned changelog mode indicates the set of changes that the planner can expect during runtime.

For regular batch scenarios, the source can emit a bounded stream of insert-only rows.

For regular streaming scenarios, the source can emit an unbounded stream of insert-only rows.

For change data capture (CDC) scenarios, the source can emit bounded or unbounded streams with insert, update, and delete rows.

A table source can implement further ability interfaces such as SupportsProjectionPushDown that might mutate an instance during planning. All abilities can be found in the org.apache.flink.table.connector.source.abilities package and are listed in the source abilities table.

The runtime implementation of a ScanTableSource must produce internal data structures. Thus, records must be emitted as org.apache.flink.table.data.RowData. The framework provides runtime converters such that a source can still work on common data structures and perform a conversion at the end.

Lookup Table Source

A LookupTableSource looks up rows of an external storage system by one or more keys during runtime.

Compared to ScanTableSource, the source does not have to read the entire table and can lazily fetch individual values from a (possibly continuously changing) external table when necessary.

Compared to ScanTableSource, a LookupTableSource does only support emitting insert-only changes currently.

Further abilities are not supported. See the documentation of org.apache.flink.table.connector.source.LookupTableSource for more information.

The runtime implementation of a LookupTableSource is a TableFunction or AsyncTableFunction. The function will be called with values for the given lookup keys during runtime.

Source Abilities

InterfaceDescription
SupportsFilterPushDownEnables to push down the filter into the DynamicTableSource. For efficiency, a source can push filters further down in order to be close to the actual data generation.
SupportsLimitPushDownEnables to push down a limit (the expected maximum number of produced records) into a DynamicTableSource.
SupportsPartitionPushDownEnables to pass available partitions to the planner and push down partitions into a DynamicTableSource. During the runtime, the source will only read data from the passed partition list for efficiency.
SupportsProjectionPushDownEnables to push down a (possibly nested) projection into a DynamicTableSource. For efficiency, a source can push a projection further down in order to be close to the actual data generation. If the source also implements SupportsReadingMetadata, the source will also read the required metadata only.
SupportsReadingMetadataEnables to read metadata columns from a DynamicTableSource. The source is responsible to add the required metadata at the end of the produced rows. This includes potentially forwarding metadata column from contained formats.
SupportsWatermarkPushDownEnables to push down a watermark strategy into a DynamicTableSource. The watermark strategy is a builder/factory for timestamp extraction and watermark generation. During the runtime, the watermark generator is located inside the source and is able to generate per-partition watermarks.
SupportsSourceWatermarkEnables to fully rely on the watermark strategy provided by the ScanTableSource itself. Thus, a CREATE TABLE DDL is able to use SOURCE_WATERMARK() which is a built-in marker function that will be detected by the planner and translated into a call to this interface if available.

Attention The interfaces above are currently only available for ScanTableSource, not for LookupTableSource.

Dynamic Table Sink

By definition, a dynamic table can change over time.

When writing a dynamic table, the content can always be considered as a changelog (finite or infinite) for which all changes are written out continuously until the changelog is exhausted. The returned changelog mode indicates the set of changes that the sink accepts during runtime.

For regular batch scenarios, the sink can solely accept insert-only rows and write out bounded streams.

For regular streaming scenarios, the sink can solely accept insert-only rows and can write out unbounded streams.

For change data capture (CDC) scenarios, the sink can write out bounded or unbounded streams with insert, update, and delete rows.

A table sink can implement further ability interfaces such as SupportsOverwrite that might mutate an instance during planning. All abilities can be found in the org.apache.flink.table.connector.sink.abilities package and are listed in the sink abilities table.

The runtime implementation of a DynamicTableSink must consume internal data structures. Thus, records must be accepted as org.apache.flink.table.data.RowData. The framework provides runtime converters such that a sink can still work on common data structures and perform a conversion at the beginning.

Sink Abilities

InterfaceDescription
SupportsOverwriteEnables to overwrite existing data in a DynamicTableSink. By default, if this interface is not implemented, existing tables or partitions cannot be overwritten using e.g. the SQL INSERT OVERWRITE clause.
SupportsPartitioningEnables to write partitioned data in a DynamicTableSink.
SupportsWritingMetadataEnables to write metadata columns into a DynamicTableSource. A table sink is responsible for accepting requested metadata columns at the end of consumed rows and persist them. This includes potentially forwarding metadata columns to contained formats.

Encoding / Decoding Formats

Some table connectors accept different formats that encode and decode keys and/or values.

Formats work similar to the pattern DynamicTableSourceFactory -> DynamicTableSource -> ScanRuntimeProvider, where the factory is responsible for translating options and the source is responsible for creating runtime logic.

Because formats might be located in different modules, they are discovered using Java’s Service Provider Interface similar to table factories. In order to discover a format factory, the dynamic table factory searches for a factory that corresponds to a factory identifier and connector-specific base class.

For example, the Kafka table source requires a DeserializationSchema as runtime interface for a decoding format. Therefore, the Kafka table source factory uses the value of the value.format option to discover a DeserializationFormatFactory.

The following format factories are currently supported:

  1. org.apache.flink.table.factories.DeserializationFormatFactory
  2. org.apache.flink.table.factories.SerializationFormatFactory

The format factory translates the options into an EncodingFormat or a DecodingFormat. Those interfaces are another kind of factory that produce specialized format runtime logic for the given data type.

For example, for a Kafka table source factory, the DeserializationFormatFactory would return an EncodingFormat<DeserializationSchema> that can be passed into the Kafka table source.

Full Stack Example

This section sketches how to implement a scan table source with a decoding format that supports changelog semantics. The example illustrates how all of the mentioned components play together. It can serve as a reference implementation.

In particular, it shows how to

  • create factories that parse and validate options,
  • implement table connectors,
  • implement and discover custom formats,
  • and use provided utilities such as data structure converters and the FactoryUtil.

The table source uses a simple single-threaded SourceFunction to open a socket that listens for incoming bytes. The raw bytes are decoded into rows by a pluggable format. The format expects a changelog flag as the first column.

We will use most of the interfaces mentioned above to enable the following DDL:

  1. CREATE TABLE UserScores (name STRING, score INT)
  2. WITH (
  3. 'connector' = 'socket',
  4. 'hostname' = 'localhost',
  5. 'port' = '9999',
  6. 'byte-delimiter' = '10',
  7. 'format' = 'changelog-csv',
  8. 'changelog-csv.column-delimiter' = '|'
  9. );

Because the format supports changelog semantics, we are able to ingest updates during runtime and create an updating view that can continuously evaluate changing data:

  1. SELECT name, SUM(score) FROM UserScores GROUP BY name;

Use the following command to ingest data in a terminal:

  1. > nc -lk 9999
  2. INSERT|Alice|12
  3. INSERT|Bob|5
  4. DELETE|Alice|12
  5. INSERT|Alice|18

Factories

This section illustrates how to translate metadata coming from the catalog to concrete connector instances.

Both factories have been added to the META-INF/services directory.

SocketDynamicTableFactory

The SocketDynamicTableFactory translates the catalog table to a table source. Because the table source requires a decoding format, we are discovering the format using the provided FactoryUtil for convenience.

  1. import org.apache.flink.api.common.serialization.DeserializationSchema;
  2. import org.apache.flink.configuration.ConfigOption;
  3. import org.apache.flink.configuration.ConfigOptions;
  4. import org.apache.flink.configuration.ReadableConfig;
  5. import org.apache.flink.table.connector.format.DecodingFormat;
  6. import org.apache.flink.table.connector.source.DynamicTableSource;
  7. import org.apache.flink.table.data.RowData;
  8. import org.apache.flink.table.factories.DeserializationFormatFactory;
  9. import org.apache.flink.table.factories.DynamicTableSourceFactory;
  10. import org.apache.flink.table.factories.FactoryUtil;
  11. import org.apache.flink.table.types.DataType;
  12. public class SocketDynamicTableFactory implements DynamicTableSourceFactory {
  13. // define all options statically
  14. public static final ConfigOption<String> HOSTNAME = ConfigOptions.key("hostname")
  15. .stringType()
  16. .noDefaultValue();
  17. public static final ConfigOption<Integer> PORT = ConfigOptions.key("port")
  18. .intType()
  19. .noDefaultValue();
  20. public static final ConfigOption<Integer> BYTE_DELIMITER = ConfigOptions.key("byte-delimiter")
  21. .intType()
  22. .defaultValue(10); // corresponds to '\n'
  23. @Override
  24. public String factoryIdentifier() {
  25. return "socket"; // used for matching to `connector = '...'`
  26. }
  27. @Override
  28. public Set<ConfigOption<?>> requiredOptions() {
  29. final Set<ConfigOption<?>> options = new HashSet<>();
  30. options.add(HOSTNAME);
  31. options.add(PORT);
  32. options.add(FactoryUtil.FORMAT); // use pre-defined option for format
  33. return options;
  34. }
  35. @Override
  36. public Set<ConfigOption<?>> optionalOptions() {
  37. final Set<ConfigOption<?>> options = new HashSet<>();
  38. options.add(BYTE_DELIMITER);
  39. return options;
  40. }
  41. @Override
  42. public DynamicTableSource createDynamicTableSource(Context context) {
  43. // either implement your custom validation logic here ...
  44. // or use the provided helper utility
  45. final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context);
  46. // discover a suitable decoding format
  47. final DecodingFormat<DeserializationSchema<RowData>> decodingFormat = helper.discoverDecodingFormat(
  48. DeserializationFormatFactory.class,
  49. FactoryUtil.FORMAT);
  50. // validate all options
  51. helper.validate();
  52. // get the validated options
  53. final ReadableConfig options = helper.getOptions();
  54. final String hostname = options.get(HOSTNAME);
  55. final int port = options.get(PORT);
  56. final byte byteDelimiter = (byte) (int) options.get(BYTE_DELIMITER);
  57. // derive the produced data type (excluding computed columns) from the catalog table
  58. final DataType producedDataType =
  59. context.getCatalogTable().getResolvedSchema().toPhysicalRowDataType();
  60. // create and return dynamic table source
  61. return new SocketDynamicTableSource(hostname, port, byteDelimiter, decodingFormat, producedDataType);
  62. }
  63. }

ChangelogCsvFormatFactory

The ChangelogCsvFormatFactory translates format-specific options to a format. The FactoryUtil in SocketDynamicTableFactory takes care of adapting the option keys accordingly and handles the prefixing like changelog-csv.column-delimiter.

Because this factory implements DeserializationFormatFactory, it could also be used for other connectors that support deserialization formats such as the Kafka connector.

  1. import org.apache.flink.api.common.serialization.DeserializationSchema;
  2. import org.apache.flink.configuration.ConfigOption;
  3. import org.apache.flink.configuration.ConfigOptions;
  4. import org.apache.flink.configuration.ReadableConfig;
  5. import org.apache.flink.table.connector.format.DecodingFormat;
  6. import org.apache.flink.table.data.RowData;
  7. import org.apache.flink.table.factories.FactoryUtil;
  8. import org.apache.flink.table.factories.DeserializationFormatFactory;
  9. import org.apache.flink.table.factories.DynamicTableFactory;
  10. public class ChangelogCsvFormatFactory implements DeserializationFormatFactory {
  11. // define all options statically
  12. public static final ConfigOption<String> COLUMN_DELIMITER = ConfigOptions.key("column-delimiter")
  13. .stringType()
  14. .defaultValue("|");
  15. @Override
  16. public String factoryIdentifier() {
  17. return "changelog-csv";
  18. }
  19. @Override
  20. public Set<ConfigOption<?>> requiredOptions() {
  21. return Collections.emptySet();
  22. }
  23. @Override
  24. public Set<ConfigOption<?>> optionalOptions() {
  25. final Set<ConfigOption<?>> options = new HashSet<>();
  26. options.add(COLUMN_DELIMITER);
  27. return options;
  28. }
  29. @Override
  30. public DecodingFormat<DeserializationSchema<RowData>> createDecodingFormat(
  31. DynamicTableFactory.Context context,
  32. ReadableConfig formatOptions) {
  33. // either implement your custom validation logic here ...
  34. // or use the provided helper method
  35. FactoryUtil.validateFactoryOptions(this, formatOptions);
  36. // get the validated options
  37. final String columnDelimiter = formatOptions.get(COLUMN_DELIMITER);
  38. // create and return the format
  39. return new ChangelogCsvFormat(columnDelimiter);
  40. }
  41. }

Table Source and Decoding Format

This section illustrates how to translate from instances of the planning layer to runtime instances that are shipped to the cluster.

SocketDynamicTableSource

The SocketDynamicTableSource is used during planning. In our example, we don’t implement any of the available ability interfaces. Therefore, the main logic can be found in getScanRuntimeProvider(...) where we instantiate the required SourceFunction and its DeserializationSchema for runtime. Both instances are parameterized to return internal data structures (i.e. RowData).

  1. import org.apache.flink.api.common.serialization.DeserializationSchema;
  2. import org.apache.flink.streaming.api.functions.source.SourceFunction;
  3. import org.apache.flink.table.connector.ChangelogMode;
  4. import org.apache.flink.table.connector.format.DecodingFormat;
  5. import org.apache.flink.table.connector.source.DynamicTableSource;
  6. import org.apache.flink.table.connector.source.ScanTableSource;
  7. import org.apache.flink.table.connector.source.SourceFunctionProvider;
  8. import org.apache.flink.table.data.RowData;
  9. import org.apache.flink.table.types.DataType;
  10. public class SocketDynamicTableSource implements ScanTableSource {
  11. private final String hostname;
  12. private final int port;
  13. private final byte byteDelimiter;
  14. private final DecodingFormat<DeserializationSchema<RowData>> decodingFormat;
  15. private final DataType producedDataType;
  16. public SocketDynamicTableSource(
  17. String hostname,
  18. int port,
  19. byte byteDelimiter,
  20. DecodingFormat<DeserializationSchema<RowData>> decodingFormat,
  21. DataType producedDataType) {
  22. this.hostname = hostname;
  23. this.port = port;
  24. this.byteDelimiter = byteDelimiter;
  25. this.decodingFormat = decodingFormat;
  26. this.producedDataType = producedDataType;
  27. }
  28. @Override
  29. public ChangelogMode getChangelogMode() {
  30. // in our example the format decides about the changelog mode
  31. // but it could also be the source itself
  32. return decodingFormat.getChangelogMode();
  33. }
  34. @Override
  35. public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
  36. // create runtime classes that are shipped to the cluster
  37. final DeserializationSchema<RowData> deserializer = decodingFormat.createRuntimeDecoder(
  38. runtimeProviderContext,
  39. producedDataType);
  40. final SourceFunction<RowData> sourceFunction = new SocketSourceFunction(
  41. hostname,
  42. port,
  43. byteDelimiter,
  44. deserializer);
  45. return SourceFunctionProvider.of(sourceFunction, false);
  46. }
  47. @Override
  48. public DynamicTableSource copy() {
  49. return new SocketDynamicTableSource(hostname, port, byteDelimiter, decodingFormat, producedDataType);
  50. }
  51. @Override
  52. public String asSummaryString() {
  53. return "Socket Table Source";
  54. }
  55. }

ChangelogCsvFormat

The ChangelogCsvFormat is a decoding format that uses a DeserializationSchema during runtime. It supports emitting INSERT and DELETE changes.

  1. import org.apache.flink.api.common.serialization.DeserializationSchema;
  2. import org.apache.flink.api.common.typeinfo.TypeInformation;
  3. import org.apache.flink.table.connector.ChangelogMode;
  4. import org.apache.flink.table.connector.format.DecodingFormat;
  5. import org.apache.flink.table.connector.source.DynamicTableSource;
  6. import org.apache.flink.table.connector.source.DynamicTableSource.DataStructureConverter;
  7. import org.apache.flink.table.data.RowData;
  8. import org.apache.flink.table.types.DataType;
  9. import org.apache.flink.table.types.logical.LogicalType;
  10. import org.apache.flink.types.RowKind;
  11. public class ChangelogCsvFormat implements DecodingFormat<DeserializationSchema<RowData>> {
  12. private final String columnDelimiter;
  13. public ChangelogCsvFormat(String columnDelimiter) {
  14. this.columnDelimiter = columnDelimiter;
  15. }
  16. @Override
  17. @SuppressWarnings("unchecked")
  18. public DeserializationSchema<RowData> createRuntimeDecoder(
  19. DynamicTableSource.Context context,
  20. DataType producedDataType) {
  21. // create type information for the DeserializationSchema
  22. final TypeInformation<RowData> producedTypeInfo = (TypeInformation<RowData>) context.createTypeInformation(
  23. producedDataType);
  24. // most of the code in DeserializationSchema will not work on internal data structures
  25. // create a converter for conversion at the end
  26. final DataStructureConverter converter = context.createDataStructureConverter(producedDataType);
  27. // use logical types during runtime for parsing
  28. final List<LogicalType> parsingTypes = producedDataType.getLogicalType().getChildren();
  29. // create runtime class
  30. return new ChangelogCsvDeserializer(parsingTypes, converter, producedTypeInfo, columnDelimiter);
  31. }
  32. @Override
  33. public ChangelogMode getChangelogMode() {
  34. // define that this format can produce INSERT and DELETE rows
  35. return ChangelogMode.newBuilder()
  36. .addContainedKind(RowKind.INSERT)
  37. .addContainedKind(RowKind.DELETE)
  38. .build();
  39. }
  40. }

Runtime

For completeness, this section illustrates the runtime logic for both SourceFunction and DeserializationSchema.

ChangelogCsvDeserializer

The ChangelogCsvDeserializer contains a simple parsing logic for converting bytes into Row of Integer and String with a row kind. The final conversion step converts those into internal data structures.

  1. import org.apache.flink.api.common.serialization.DeserializationSchema;
  2. import org.apache.flink.api.common.typeinfo.TypeInformation;
  3. import org.apache.flink.table.connector.RuntimeConverter.Context;
  4. import org.apache.flink.table.connector.source.DynamicTableSource.DataStructureConverter;
  5. import org.apache.flink.table.data.RowData;
  6. import org.apache.flink.table.types.logical.LogicalType;
  7. import org.apache.flink.table.types.logical.LogicalTypeRoot;
  8. import org.apache.flink.types.Row;
  9. import org.apache.flink.types.RowKind;
  10. public class ChangelogCsvDeserializer implements DeserializationSchema<RowData> {
  11. private final List<LogicalType> parsingTypes;
  12. private final DataStructureConverter converter;
  13. private final TypeInformation<RowData> producedTypeInfo;
  14. private final String columnDelimiter;
  15. public ChangelogCsvDeserializer(
  16. List<LogicalType> parsingTypes,
  17. DataStructureConverter converter,
  18. TypeInformation<RowData> producedTypeInfo,
  19. String columnDelimiter) {
  20. this.parsingTypes = parsingTypes;
  21. this.converter = converter;
  22. this.producedTypeInfo = producedTypeInfo;
  23. this.columnDelimiter = columnDelimiter;
  24. }
  25. @Override
  26. public TypeInformation<RowData> getProducedType() {
  27. // return the type information required by Flink's core interfaces
  28. return producedTypeInfo;
  29. }
  30. @Override
  31. public void open(InitializationContext context) {
  32. // converters must be open
  33. converter.open(Context.create(ChangelogCsvDeserializer.class.getClassLoader()));
  34. }
  35. @Override
  36. public RowData deserialize(byte[] message) {
  37. // parse the columns including a changelog flag
  38. final String[] columns = new String(message).split(Pattern.quote(columnDelimiter));
  39. final RowKind kind = RowKind.valueOf(columns[0]);
  40. final Row row = new Row(kind, parsingTypes.size());
  41. for (int i = 0; i < parsingTypes.size(); i++) {
  42. row.setField(i, parse(parsingTypes.get(i).getTypeRoot(), columns[i + 1]));
  43. }
  44. // convert to internal data structure
  45. return (RowData) converter.toInternal(row);
  46. }
  47. private static Object parse(LogicalTypeRoot root, String value) {
  48. switch (root) {
  49. case INTEGER:
  50. return Integer.parseInt(value);
  51. case VARCHAR:
  52. return value;
  53. default:
  54. throw new IllegalArgumentException();
  55. }
  56. }
  57. @Override
  58. public boolean isEndOfStream(RowData nextElement) {
  59. return false;
  60. }
  61. }

SocketSourceFunction

The SocketSourceFunction opens a socket and consumes bytes. It splits records by the given byte delimiter (\n by default) and delegates the decoding to a pluggable DeserializationSchema. The source function can only work with a parallelism of 1.

  1. import org.apache.flink.api.common.serialization.DeserializationSchema;
  2. import org.apache.flink.api.common.typeinfo.TypeInformation;
  3. import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
  4. import org.apache.flink.configuration.Configuration;
  5. import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
  6. import org.apache.flink.table.data.RowData;
  7. public class SocketSourceFunction extends RichSourceFunction<RowData> implements ResultTypeQueryable<RowData> {
  8. private final String hostname;
  9. private final int port;
  10. private final byte byteDelimiter;
  11. private final DeserializationSchema<RowData> deserializer;
  12. private volatile boolean isRunning = true;
  13. private Socket currentSocket;
  14. public SocketSourceFunction(String hostname, int port, byte byteDelimiter, DeserializationSchema<RowData> deserializer) {
  15. this.hostname = hostname;
  16. this.port = port;
  17. this.byteDelimiter = byteDelimiter;
  18. this.deserializer = deserializer;
  19. }
  20. @Override
  21. public TypeInformation<RowData> getProducedType() {
  22. return deserializer.getProducedType();
  23. }
  24. @Override
  25. public void open(Configuration parameters) throws Exception {
  26. deserializer.open(() -> getRuntimeContext().getMetricGroup());
  27. }
  28. @Override
  29. public void run(SourceContext<RowData> ctx) throws Exception {
  30. while (isRunning) {
  31. // open and consume from socket
  32. try (final Socket socket = new Socket()) {
  33. currentSocket = socket;
  34. socket.connect(new InetSocketAddress(hostname, port), 0);
  35. try (InputStream stream = socket.getInputStream()) {
  36. ByteArrayOutputStream buffer = new ByteArrayOutputStream();
  37. int b;
  38. while ((b = stream.read()) >= 0) {
  39. // buffer until delimiter
  40. if (b != byteDelimiter) {
  41. buffer.write(b);
  42. }
  43. // decode and emit record
  44. else {
  45. ctx.collect(deserializer.deserialize(buffer.toByteArray()));
  46. buffer.reset();
  47. }
  48. }
  49. }
  50. } catch (Throwable t) {
  51. t.printStackTrace(); // print and continue
  52. }
  53. Thread.sleep(1000);
  54. }
  55. }
  56. @Override
  57. public void cancel() {
  58. isRunning = false;
  59. try {
  60. currentSocket.close();
  61. } catch (Throwable t) {
  62. // ignore
  63. }
  64. }
  65. }