...
The flink-connector-redshift module will leverage and use already existing connectors. When a user specifies the read mode as "jdbc" in the source configuration, the flink-connector-redshift module internally integrates and uses the capabilities of the flink-connector-jdbcuses redshift jdbc driver. This integration enables seamless interaction with Redshift using the JDBC protocol. Leveraging this connector, the flink-connector-redshift can efficiently retrieve data from the Redshift database, providing a reliable and performant data source for further processing within the Flink framework.
...
- Directly writing to redshift using JDBC driver.
- Flink stream write to a specified s3 path in the format and schema accepted by Redshift then use
COPY
command to write data into the redshift table.
Flink connector redshift will provide users with the ability to configure different modes for the connector sink by utilizing thesink.write.mode
option option in the source configuration. This configuration flexibility allows users to tailor the behaviour of the connector sink to their specific needs. Internally, the flink-connector-redshift module will intelligently select and integrate with the appropriate connectors based on the chosen write mode. If the user specifies the write mode as "jdbc" in the source configuration, the flink-connector-redshift module will internally leverage the capabilities of the flink-connector-jdbcwill use custom redshift JDBC driver. This integration will enable seamless interaction with Redshift using the JDBC protocol, ensuring efficient data transfer from Flink to the Redshift database. Similarly, when the write mode is selected as a file-based operation, the flink-connector-redshift module will utilize the flink-connector-filesystem. This connector will facilitate writing data to an S3 bucket, adhering to the specific format and schema requirements outlined by Redshift's COPY command. By utilizing this connector, the flink-connector-redshift module will ensure that data is written in a manner compatible with Redshift's expectations. To provide a streamlined sink solution for Flink and Redshift integration, the flink-connector-redshift module will orchestrate and utilize these two connectors, use flink-connector-jdbc and flink-filesystem, behind the scenes. It preprocesses the data and seamlessly wraps these connectors to offer a unified sink interface for transferring data from Flink to Redshift.
In summary, the flink-connector-redshift module offers users the flexibility to configure different modes for the connector sink. By intelligently utilizing the appropriate connectors, such as flink-connector-jdbc for direct JDBC-based interaction and flink-connector-filesystem for file-based operations, the module ensures efficient data transfer to Redshift. Through its preprocessing capabilities and seamless integration, the flink-connector-redshift module acts as a comprehensive sink solution, facilitating the smooth transfer of data from Flink to Redshift.
i) Sink configurations(
mandatory)
i) Sink configurations(
mandatory)
Option | Value | Description |
---|---|---|
host | <redshifthost> | |
database | database_name | |
table-name | table_name | |
sink.write.mode | jdbc/ copy | |
aws-iam-role | arn:aws:iam::123456789010:role/redshift_iam_role | |
username | <username> | |
password | <password> |
...
- Flink to Redshift: The Flink connects to Redshift via JDBC using a username and password.
Redshift does not support the use of IAM roles to authenticate this connection.
This connection can be secured using SSL; for more details, see the Encryption section below. - Flink to S3: S3 acts as a middleman to store bulk data when reading from or writing to Redshift.
Flink connects to S3 using both the Hadoop FileSystem interfaces and directly using the Amazon Java SDK's S3 client.
This connection can be authenticated using either AWS keys or IAM roles. - Default Authentication: Flink-connector-aws provides different modes of Authentication redshift connectors will use CredentialProvider.
AWS credentials will automatically be retrieved through the DefaultAWSCredentialsProviderChain.
Users can use IAM instance roles to authenticate to S3 (e.g. on EMR, or EC2). or EC2).
- Flink to Redshift: The Flink connects to Redshift via JDBC using a username and password.
Proof Of Concept : https://github.com/Samrat002/flink-connector-aws/tree/redshift-connector
Limitations
- Parallelism in flink-connector-redshift will be limited by Quotas configured for the job in Redshift Connections Quotas and limits in Amazon Redshift - Amazon Redshift.
- Speed and latency in source and sink will be regulated by the latency and data transfer rate of
UNLOAD
andCOPY
feature from redshift - Flink connector redshift sink will only support append-only tables. (no changelog mode support)
...
- There will be Unit Tests cases to test methods in the redshift connector.
- E2E test suit to be added in flink-connector-aws-e2e-tests.
Rejected Alternatives
...
- Initially , Idea was to use flink-connector-jdbc to support jdbc mode in redshift connector but rejected because it adds dependency of flink-connector-aws on flink-connector-jdbc creating dependency on release cycles of different externalised connectors.