[
https://issues.apache.org/jira/browse/FLINK-15167?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16992350#comment-16992350
]
Rui Li commented on FLINK-15167:
--------------------------------
Identified the following issues:
# Both creating and registering catalogs should be done with user class loader
as the context loader.
# Need to set the dependent jars via {{PipelineOptions.JARS}} in the
configuration, so that our job graph carries these dependencies.
After fixing the above issues, reading Hive tables works. But when I try to
write to Hive tables, got the following problem:
{noformat}
Caused by: java.lang.NoClassDefFoundError: org/apache/hadoop/fs/FileSystem
at
org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.<init>(HadoopFileSystem.java:54)
at
org.apache.flink.connectors.hive.HadoopFileSystemFactory.create(HadoopFileSystemFactory.java:46)
at
org.apache.flink.table.filesystem.PartitionTempFileManager.<init>(PartitionTempFileManager.java:73)
at
org.apache.flink.table.filesystem.FileSystemOutputFormat.open(FileSystemOutputFormat.java:104)
at
org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction.open(OutputFormatSinkFunction.java:65)
at
org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:36)
at
org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:102)
at
org.apache.flink.streaming.api.operators.StreamSink.open(StreamSink.java:48)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeStateAndOpen(StreamTask.java:986)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:453)
at
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$SynchronizedStreamTaskActionExecutor.runThrowing(StreamTaskActionExecutor.java:94)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:448)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:460)
at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:702)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:527)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.fs.FileSystem
at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:349)
at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
... 16 more
{noformat}
This is because {{HadoopFileSystem}} is loaded by parent loader (in flink-dist
jar), while {{org.apache.hadoop.fs.FileSystem}} (a field of
{{HadoopFileSystem}}) needs to be loaded by child loader.
I think one option is to let Hive connector implement its own
{{HadoopFileSystem}}. Alternatively, we can remove {{HadoopFileSystem}} from
{{flink-dist}}. Since {{flink-dist}} doesn't contain
{{org.apache.hadoop.fs.FileSystem}}, I'm not sure what's the point to pack
{{HadoopFileSystem}} in the jar.
cc [~lzljs3620320] [~ykt836] What do you think?
> SQL CLI library option doesn't work for Hive connector
> ------------------------------------------------------
>
> Key: FLINK-15167
> URL: https://issues.apache.org/jira/browse/FLINK-15167
> Project: Flink
> Issue Type: Bug
> Components: Connectors / Hive, Table SQL / Client
> Reporter: Rui Li
> Priority: Major
> Fix For: 1.10.0
>
>
> Put all Hive connector dependency jars in a folder and start sql cli like:
> {{sql-client.sh embedded -l <folder>}}. Hit the following exception:
> {noformat}
> at
> org.apache.flink.table.catalog.hive.HiveCatalog.open(HiveCatalog.java:186)
> at
> org.apache.flink.table.catalog.CatalogManager.registerCatalog(CatalogManager.java:102)
> at
> org.apache.flink.table.api.internal.TableEnvironmentImpl.registerCatalog(TableEnvironmentImpl.java:233)
> at java.util.LinkedHashMap.forEach(LinkedHashMap.java:684)
> at
> org.apache.flink.table.client.gateway.local.ExecutionContext.initializeCatalogs(ExecutionContext.java:519)
> at
> org.apache.flink.table.client.gateway.local.ExecutionContext.initializeTableEnvironment(ExecutionContext.java:463)
> at
> org.apache.flink.table.client.gateway.local.ExecutionContext.<init>(ExecutionContext.java:156)
> at
> org.apache.flink.table.client.gateway.local.ExecutionContext.<init>(ExecutionContext.java:115)
> at
> org.apache.flink.table.client.gateway.local.ExecutionContext$Builder.build(ExecutionContext.java:724)
> ... 3 more
> Caused by: java.lang.reflect.InvocationTargetException
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at
> org.apache.flink.table.catalog.hive.client.HiveShimV230.getHiveMetastoreClient(HiveShimV230.java:55)
> ... 15 more
> Caused by:
> MetaException(message:org.apache.hadoop.hive.metastore.HiveMetaStoreClient
> class not found)
> at
> org.apache.hadoop.hive.metastore.MetaStoreUtils.getClass(MetaStoreUtils.java:1676)
> at
> org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:131)
> at
> org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:89)
> ... 20 more
> {noformat}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)