[
https://issues.apache.org/jira/browse/FLINK-15992?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Robert Metzger reopened FLINK-15992:
------------------------------------
I reopened this ticket, because the end to end test are still failing because
of this: [https://travis-ci.org/apache/flink/builds/651101849]
{code:java}
16:17:45.788 [INFO] Running
org.apache.flink.tests.util.kafka.SQLClientKafkaITCase2021216:19:12.123 [ERROR]
Tests run: 3, Failures: 0, Errors: 3, Skipped: 0, Time elapsed: 86.32 s <<<
FAILURE! - in
org.apache.flink.tests.util.kafka.SQLClientKafkaITCase2021316:19:12.123 [ERROR]
testKafka[0: kafka-version:0.10
kafka-sql-version:.*kafka-0.10.jar](org.apache.flink.tests.util.kafka.SQLClientKafkaITCase)
Time elapsed: 27.975 s <<< ERROR!20214java.io.IOException: 20215Process
execution failed due error. Error output:Exception in thread "main"
org.apache.flink.table.client.SqlClientException: Unexpected exception. This is
a bug. Please consider filing an issue.20216 at
org.apache.flink.table.client.SqlClient.main(SqlClient.java:190)20217Caused by:
org.apache.flink.table.client.gateway.SqlExecutionException: Could not create
execution context.20218 at
org.apache.flink.table.client.gateway.local.ExecutionContext$Builder.build(ExecutionContext.java:761)20219
at
org.apache.flink.table.client.gateway.local.LocalExecutor.openSession(LocalExecutor.java:228)20220
at org.apache.flink.table.client.SqlClient.start(SqlClient.java:98)20221
at
org.apache.flink.table.client.SqlClient.main(SqlClient.java:178)20222Caused by:
org.apache.flink.table.api.NoMatchingTableFactoryException: Could not find a
suitable table factory for
'org.apache.flink.table.factories.DeserializationSchemaFactory' in20223the
classpath.2022420225Reason: No factory implements
'org.apache.flink.table.factories.DeserializationSchemaFactory'.2022620227
{code}
> Incorrect classloader when finding TableFactory
> -----------------------------------------------
>
> Key: FLINK-15992
> URL: https://issues.apache.org/jira/browse/FLINK-15992
> Project: Flink
> Issue Type: Bug
> Components: Connectors / Kafka, Table SQL / API
> Reporter: Victor Wong
> Assignee: Victor Wong
> Priority: Major
> Labels: pull-request-available
> Fix For: 1.11.0
>
> Time Spent: 20m
> Remaining Estimate: 0h
>
> *Background*
> As a streaming service maintainer in our company, to ensure our users depend
> on the correct version of Kafka and flink-kafka, we add
> "flink-connector-kafka" into "fink-dist/lib" directory.
> *Problem*
> When submitting flink-sql jobs, we encountered below exceptions:
> {code:java}
> Caused by: org.apache.flink.table.api.NoMatchingTableFactoryException: Could
> not find a suitable table factory for
> 'org.apache.flink.table.factories.DeserializationSchemaFactory' in
> the classpath.
> {code}
> But we have add "org.apache.flink.formats.json.JsonRowFormatFactory" in
> "META-INF/services/org.apache.flink.table.factories.TableFactory", which
> implements DeserializationSchemaFactory.
> *Debug*
> We find that it was caused by this:
> {code:java}
> //
> org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactoryBase#getSerializationSchema
> final SerializationSchemaFactory<Row> formatFactory =
> TableFactoryService.find(
> SerializationSchemaFactory.class,
> properties,
> this.getClass().getClassLoader());
> {code}
> It uses `this.getClass().getClassLoader()`, which will be
> BootStrapClassLoader of flink.
> I think we could replace it with
> `Thread.currentThread().getContextClassLoader()` to solve this.
> There is a related issue: https://issues.apache.org/jira/browse/FLINK-15552
--
This message was sent by Atlassian Jira
(v8.3.4#803005)