Nick Dimiduk created PHOENIX-1995:
-------------------------------------
Summary: client uberjar doesn't support dfs
Key: PHOENIX-1995
URL: https://issues.apache.org/jira/browse/PHOENIX-1995
Project: Phoenix
Issue Type: Bug
Affects Versions: 4.4.0
Reporter: Nick Dimiduk
Assignee: Nick Dimiduk
After UDF, the client uberjar needs hadoop dfs class on classpath in order to
use the dynamic classload. Without it, you get the following stacktrace
{noformat}
$ ./bin/sqlline.py localhost
Setting property: [isolation, TRANSACTION_READ_COMMITTED]
issuing: !connect jdbc:phoenix:localhost none none
org.apache.phoenix.jdbc.PhoenixDriver
Connecting to jdbc:phoenix:localhost
SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
SLF4J: Defaulting to no-operation (NOP) logger implementation
SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
details.
15/05/20 12:04:11 WARN util.NativeCodeLoader: Unable to load native-hadoop
library for your platform... using builtin-java classes where applicable
15/05/20 12:04:12 WARN util.DynamicClassLoader: Failed to identify the fs of
dir hdfs://localhost:9000/hbase/lib, ignored
java.io.IOException: No FileSystem for scheme: hdfs
at
org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2579)
at
org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2586)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:89)
at
org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2625)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2607)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:368)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296)
at
org.apache.hadoop.hbase.util.DynamicClassLoader.<init>(DynamicClassLoader.java:104)
at
org.apache.hadoop.hbase.protobuf.ProtobufUtil.<clinit>(ProtobufUtil.java:238)
at org.apache.hadoop.hbase.ClusterId.parseFrom(ClusterId.java:64)
at
org.apache.hadoop.hbase.zookeeper.ZKClusterId.readClusterIdZNode(ZKClusterId.java:75)
at
org.apache.hadoop.hbase.client.ZooKeeperRegistry.getClusterId(ZooKeeperRegistry.java:105)
at
org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.retrieveClusterId(ConnectionManager.java:879)
at
org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.<init>(ConnectionManager.java:635)
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
at
org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:238)
at
org.apache.hadoop.hbase.client.ConnectionManager.createConnection(ConnectionManager.java:420)
at
org.apache.hadoop.hbase.client.ConnectionManager.createConnectionInternal(ConnectionManager.java:329)
at
org.apache.hadoop.hbase.client.HConnectionManager.createConnection(HConnectionManager.java:144)
at
org.apache.phoenix.query.HConnectionFactory$HConnectionFactoryImpl.createConnection(HConnectionFactory.java:47)
at
org.apache.phoenix.query.ConnectionQueryServicesImpl.openConnection(ConnectionQueryServicesImpl.java:286)
at
org.apache.phoenix.query.ConnectionQueryServicesImpl.access$300(ConnectionQueryServicesImpl.java:171)
at
org.apache.phoenix.query.ConnectionQueryServicesImpl$12.call(ConnectionQueryServicesImpl.java:1881)
at
org.apache.phoenix.query.ConnectionQueryServicesImpl$12.call(ConnectionQueryServicesImpl.java:1860)
at
org.apache.phoenix.util.PhoenixContextExecutor.call(PhoenixContextExecutor.java:77)
at
org.apache.phoenix.query.ConnectionQueryServicesImpl.init(ConnectionQueryServicesImpl.java:1860)
at
org.apache.phoenix.jdbc.PhoenixDriver.getConnectionQueryServices(PhoenixDriver.java:180)
at
org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.connect(PhoenixEmbeddedDriver.java:131)
at org.apache.phoenix.jdbc.PhoenixDriver.connect(PhoenixDriver.java:151)
at sqlline.DatabaseConnection.connect(DatabaseConnection.java:157)
at sqlline.DatabaseConnection.getConnection(DatabaseConnection.java:203)
at sqlline.Commands.connect(Commands.java:1064)
at sqlline.Commands.connect(Commands.java:996)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at
sqlline.ReflectiveCommandHandler.execute(ReflectiveCommandHandler.java:36)
at sqlline.SqlLine.dispatch(SqlLine.java:804)
at sqlline.SqlLine.initArgs(SqlLine.java:588)
at sqlline.SqlLine.begin(SqlLine.java:656)
at sqlline.SqlLine.start(SqlLine.java:398)
at sqlline.SqlLine.main(SqlLine.java:292)
{noformat}
This is because hadoop's FileSystem class uses
[SPI|http://docs.oracle.com/javase/tutorial/sound/SPI-intro.html] to load
available file system drivers at runtime. Our uberjar is bringing in jars
containing multiple SPI definitions for the FileSystem class, and the one
containing DistributeFileSystem is losing. You can see from what's actually in
the uberjar:
{noformat}
$ unzip -p phoenix-assembly/target/phoenix-4.4.0-HBase-1.0-client.jar
META-INF/services/org.apache.hadoop.fs.FileSystem
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.hadoop.fs.LocalFileSystem
org.apache.hadoop.fs.viewfs.ViewFileSystem
org.apache.hadoop.fs.s3.S3FileSystem
org.apache.hadoop.fs.s3native.NativeS3FileSystem
org.apache.hadoop.fs.ftp.FTPFileSystem
org.apache.hadoop.fs.HarFileSystem
{noformat}
This is despite the DistributedFileSystem class being in phoenix-client.jar
We can fix this with the assembly plugin's
[containerDescriptorHandler|https://maven.apache.org/plugins/maven-assembly-plugin/assembly.html#class_containerDescriptorHandler]
setting.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)