Skip to content

[FLINK-20291][table][filesystem] Optimize the exception message of FileSystemTableSink when missing format dependencies#14198

Merged
wuchong merged 1 commit into
apache:masterfrom
wuchong:improve-err-msg-filesystem
Nov 25, 2020
Merged

[FLINK-20291][table][filesystem] Optimize the exception message of FileSystemTableSink when missing format dependencies#14198
wuchong merged 1 commit into
apache:masterfrom
wuchong:improve-err-msg-filesystem

Conversation

@wuchong
Copy link
Copy Markdown
Member

@wuchong wuchong commented Nov 24, 2020

What is the purpose of the change

Current when the format factory failed to load, the following exception would be thrown:

Exception in thread "main" org.apache.flink.table.api.ValidationException: Unable to create a sink for writing table 'default_catalog.default_database.sink'.

Table options are:

'auto-compaction'='true'
'connector'='filesystem'
'format'='csv'
'path'='file:///tmp/compaction'
  at org.apache.flink.table.factories.FactoryUtil.createTableSink(FactoryUtil.java:166)
  at org.apache.flink.table.planner.delegation.PlannerBase.getTableSink(PlannerBase.scala:362)
  at org.apache.flink.table.planner.delegation.PlannerBase.translateToRel(PlannerBase.scala:220)
  at org.apache.flink.table.planner.delegation.PlannerBase$$anonfun$1.apply(PlannerBase.scala:164)
  at org.apache.flink.table.planner.delegation.PlannerBase$$anonfun$1.apply(PlannerBase.scala:164)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.Iterator$class.foreach(Iterator.scala:891)
  at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
  at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
  at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
  at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
  at scala.collection.AbstractTraversable.map(Traversable.scala:104)
  at org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:164)
  at org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1261)
  at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:674)
  at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeOperation(TableEnvironmentImpl.java:757)
  at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:664)
  at FileCompactionTest.main(FileCompactionTest.java:147)
Caused by: org.apache.flink.table.api.ValidationException: Please implement at least one of the following formats: BulkWriter.Factory, SerializationSchema, FileSystemFormatFactory.
  at org.apache.flink.table.filesystem.FileSystemTableSink.<init>(FileSystemTableSink.java:124)
  at org.apache.flink.table.filesystem.FileSystemTableFactory.createDynamicTableSink(FileSystemTableFactory.java:83)
  at org.apache.flink.table.factories.FactoryUtil.createTableSink(FactoryUtil.java:163)
  ... 18 more

We might directly advice users to check if the format dependency is added or if there are package conflicts so that users would fix this issue faster.

Brief change log

Currently, we swallow all the exceptions when discovering format factories. Fix this problem by discovering format factories with exception throwing when the format factory exists. And throw a new exception if all the format factories are not exist.

Verifying this change

  • Added unit tests

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (yes / no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (yes / no)
  • The serializers: (yes / no / don't know)
  • The runtime per-record code paths (performance sensitive): (yes / no / don't know)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / no / don't know)
  • The S3 file system connector: (yes / no / don't know)

Documentation

  • Does this pull request introduce a new feature? (yes / no)
  • If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)

…leSystemTableSink when missing format dependencies
@wuchong wuchong requested a review from JingsongLi November 24, 2020 10:27
@flinkbot
Copy link
Copy Markdown
Collaborator

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Automated Checks

Last check on commit 9d29148 (Tue Nov 24 10:28:49 UTC 2020)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!

Mention the bot in a comment to re-run the automated checks.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❓ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.

Details
The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

@flinkbot
Copy link
Copy Markdown
Collaborator

flinkbot commented Nov 24, 2020

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run travis re-run the last Travis build
  • @flinkbot run azure re-run the last Azure build

Copy link
Copy Markdown
Contributor

@JingsongLi JingsongLi left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, thanks for the contribution.

@wuchong wuchong merged commit 264f2e9 into apache:master Nov 25, 2020
@wuchong wuchong deleted the improve-err-msg-filesystem branch November 25, 2020 03:58
@cy258
Copy link
Copy Markdown

cy258 commented Mar 19, 2024

source sink select 在flink客户端可以正常运行 使用java代码运行报错:Exception in thread "main" org.apache.flink.table.api.ValidationException: Unable to create a sink for writing table 'default_catalog.default_database.sink'.

Table options are:

'connector'='kafka'
'fields-include'='ALL'
'format'='json'
'properties.bootstrap.servers'='192.168.10.31:9092,192.168.10.32:9092'
'properties.max.block.ms'='60000'
'properties.request.timeout.ms'='30000'
'topic'='thirdDataTest'
怎么解决呀

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants