[
https://issues.apache.org/jira/browse/FLINK-25635?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17477047#comment-17477047
]
Jing Zhang commented on FLINK-25635:
------------------------------------
[~airblader]
Thanks a lot for reply.
> Are you using the correct version of flink-connector-hive as well?
Yes, I'm pretty sure about this point.
> TableFactory and Factory are different interfaces – HiveParserFactory is no
> longer provided as a TableFactory, so I'm not sure why it is attempted to be
> loaded as such.
This is a point which i'm confused about.
Now all factories class in hive module are all list in [Factory metadata
file|https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-hive/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory].
When call
[TableFactoryService.java#L207|https://github.com/apache/flink/blob/b7fd63b41500dbfeadbf2ee48e1a83d77cd1259b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/TableFactoryService.java#L207],
it seems to load the file [Factory metadata
file|https://github.com/apache/flink/blob/master/flink-connectors/flink-connector-hive/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory].
And in ServiceLoader, it checks all lists in metadata file have to be the
TableFactory, otherwise it would throws the exception in the issue.
!screenshot-1.png!
WDYT? [~airblader] [~luoyuxia]
> Using sql client create hive catalog throw exception
> ----------------------------------------------------
>
> Key: FLINK-25635
> URL: https://issues.apache.org/jira/browse/FLINK-25635
> Project: Flink
> Issue Type: New Feature
> Components: Connectors / Hive, Table SQL / Ecosystem
> Affects Versions: 1.14.0, 1.15.0
> Reporter: dalongliu
> Priority: Blocker
> Fix For: 1.15.0
>
> Attachments: image-2022-01-17-16-09-28-364.png,
> image-2022-01-17-16-13-07-674.png, image-2022-01-17-16-59-25-147.png,
> image-2022-01-17-17-00-18-160.png, image-2022-01-17-17-00-41-223.png,
> screenshot-1.png
>
>
> CREATE CATALOG hive WITH('type' = 'hive',
> 'hive-conf-dir'='/usr/local/hive/conf');
> [ERROR] Could not execute SQL statement. Reason:
> java.util.ServiceConfigurationError:
> org.apache.flink.table.factories.TableFactory: Provider
> org.apache.flink.table.planner.delegation.hive.HiveParserFactory not a subtype
> at java.util.ServiceLoader.fail(ServiceLoader.java:239)
> at java.util.ServiceLoader.access$300(ServiceLoader.java:185)
> at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:376)
> at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
> at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
> at java.util.Iterator.forEachRemaining(Iterator.java:116)
> at
> org.apache.flink.table.factories.TableFactoryService.discoverFactories(TableFactoryService.java:208)
> at
> org.apache.flink.table.factories.TableFactoryService.findSingleInternal(TableFactoryService.java:153)
> at
> org.apache.flink.table.factories.TableFactoryService.find(TableFactoryService.java:123)
> at
> org.apache.flink.table.factories.FactoryUtil.createCatalog(FactoryUtil.java:264)
> at
> org.apache.flink.table.api.internal.TableEnvironmentImpl.createCatalog(TableEnvironmentImpl.java:1292)
> at
> org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:1122)
> at
> org.apache.flink.table.client.gateway.local.LocalExecutor.lambda$executeOperation$3(LocalExecutor.java:209)
> at
> org.apache.flink.table.client.gateway.context.ExecutionContext.wrapClassLoader(ExecutionContext.java:88)
> at
> org.apache.flink.table.client.gateway.local.LocalExecutor.executeOperation(LocalExecutor.java:209)
> at
> org.apache.flink.table.client.cli.CliClient.executeOperation(CliClient.java:625)
> at
> org.apache.flink.table.client.cli.CliClient.callOperation(CliClient.java:447)
> at
> org.apache.flink.table.client.cli.CliClient.lambda$executeStatement$1(CliClient.java:332)
> at java.util.Optional.ifPresent(Optional.java:159)
> at
> org.apache.flink.table.client.cli.CliClient.executeStatement(CliClient.java:325)
> at
> org.apache.flink.table.client.cli.CliClient.executeInteractive(CliClient.java:297)
> at
> org.apache.flink.table.client.cli.CliClient.executeInInteractiveMode(CliClient.java:221)
> at org.apache.flink.table.client.SqlClient.openCli(SqlClient.java:151)
> at org.apache.flink.table.client.SqlClient.start(SqlClient.java:95)
> at org.apache.flink.table.client.SqlClient.startClient(SqlClient.java:187)
> at org.apache.flink.table.client.SqlClient.main(SqlClient.java:161)
--
This message was sent by Atlassian Jira
(v8.20.1#820001)