Khanh Vu created FLINK-34076:
--------------------------------
Summary: flink-connector-base missing fails kinesis table sink to
create
Key: FLINK-34076
URL: https://issues.apache.org/jira/browse/FLINK-34076
Project: Flink
Issue Type: Bug
Components: Connectors / Kinesis
Affects Versions: aws-connector-4.2.0
Reporter: Khanh Vu
The
[commit|https://github.com/apache/flink-connector-aws/commit/01f112bd5a69f95cd5d2a4bc7e08d1ba9a81d56a]
which stops bundling `flink-connector-base` with `flink-connector-kinesis` has
caused kinesis sink failing to create when using Table API as required classes
from `flink-connector-base` are not loaded in execution.
E.g. with following depenency only in pom.xml
```
{{Caused by: org.apache.flink.table.api.ValidationException: Connector
'kinesis' can only be used as a source. It cannot be used as a sink.
at
org.apache.flink.table.factories.FactoryUtil.enrichNoMatchingConnectorError(FactoryUtil.java:756)
at
org.apache.flink.table.factories.FactoryUtil.discoverTableFactory(FactoryUtil.java:710)
at
org.apache.flink.table.factories.FactoryUtil.createDynamicTableSink(FactoryUtil.java:265)
... 20 more}}
```
following exception will be thrown:
```
{{Caused by: org.apache.flink.table.api.ValidationException: Connector
'kinesis' can only be used as a source. It cannot be used as a sink.
at
org.apache.flink.table.factories.FactoryUtil.enrichNoMatchingConnectorError(FactoryUtil.java:756)
at
org.apache.flink.table.factories.FactoryUtil.discoverTableFactory(FactoryUtil.java:710)
at
org.apache.flink.table.factories.FactoryUtil.createDynamicTableSink(FactoryUtil.java:265)
... 20 more}}
```
Workaround is to explicitly specify `flink-connector-base` as dependency of the
project:
```
{{<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-kinesis</artifactId>
<version>${flink.connector.kinesis.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-base</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>}}
```
In general, `flink-connector-base` should be pulled in by default when pulling
in the connector, the current separation adds unnecessary hassle to use the
connector.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)