[ https://issues.apache.org/jira/browse/HBASE-17878?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15976016#comment-15976016 ]
Xiang Li commented on HBASE-17878: ---------------------------------- [-stack] We are using HBase 1.2.x with Hadoop 2.7.x. If we can have a fix on HBase 1.2.x, it is good us. Currently, we have 2 ways to fix/bypass it 1. Very tricky one : declare joda-time as a (direct) dependency of hbase-server(or whatever needed). It results in that 2 sets of joda-time classes are in the classpath when runtime, one is from joda-time jar we just declared, the others is from jruby-complete jar. It could work only because that joda-time is in front of jruby-complete in the classpath due to lexicographical order. It is very tricky, as If I remember it correctly, it is not in the Java Language Specification that the one in the front of the classpath takes precedence. 2. Move the dir of Hadoop(hadoop-mapreduce) to be in front of hbase lib in HBase's classpath, which is also very tricky. Both of them have 2 sets of joda-time classes in the classpath. Regarding bq. If hbase2 is your target, then jruby is different now. I see in master branch, jruby-complete 1.6.8 is still used. Do you mean a JIRA is working on it? > java.lang.NoSuchMethodError: > org.joda.time.format.DateTimeFormatter.withZoneUTC()Lorg/joda/time/format/DateTimeFormatter > when starting HBase with hbase.rootdir on S3 > --------------------------------------------------------------------------------------------------------------------------------------------------------------------- > > Key: HBASE-17878 > URL: https://issues.apache.org/jira/browse/HBASE-17878 > Project: HBase > Issue Type: Bug > Components: master > Reporter: Xiang Li > Assignee: Xiang Li > Priority: Minor > Fix For: 2.0.0 > > Attachments: HBASE-17878.master.000.patch, jruby-core-dep-tree.txt > > > When setting up HBASE-17437 (Support specifying a WAL directory outside of > the root directory), we specify > (1) hbase.rootdir on s3a > (2) hbase.wal.dir on HDFS > When starting HBase, the following exception is thrown: > {code} > Caused by: java.lang.NoSuchMethodError: > org.joda.time.format.DateTimeFormatter.withZoneUTC()Lorg/joda/time/format/DateTimeFormatter; > at > com.amazonaws.auth.internal.AWS4SignerUtils.<clinit>(AWS4SignerUtils.java:26) > at > com.amazonaws.auth.internal.AWS4SignerRequestParams.<init>(AWS4SignerRequestParams.java:85) > at com.amazonaws.auth.AWS4Signer.sign(AWS4Signer.java:184) > at > com.amazonaws.http.AmazonHttpClient.executeOneRequest(AmazonHttpClient.java:709) > at > com.amazonaws.http.AmazonHttpClient.executeHelper(AmazonHttpClient.java:489) > at > com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:310) > at > com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:3785) > at > com.amazonaws.services.s3.AmazonS3Client.headBucket(AmazonS3Client.java:1107) > at > com.amazonaws.services.s3.AmazonS3Client.doesBucketExist(AmazonS3Client.java:1070) > at > org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:232) > at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2669) > at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:94) > at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2703) > at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2685) > at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:373) > at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295) > at org.apache.hadoop.hbase.util.FSUtils.getRootDir(FSUtils.java:1007) > at > org.apache.hadoop.hbase.util.FSUtils.isValidWALRootDir(FSUtils.java:1050) > at > org.apache.hadoop.hbase.util.FSUtils.getWALRootDir(FSUtils.java:1032) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.initializeFileSystem(HRegionServer.java:627) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.<init>(HRegionServer.java:570) > at org.apache.hadoop.hbase.master.HMaster.<init>(HMaster.java:393) > at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native > Method) > at > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) > at > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) > at java.lang.reflect.Constructor.newInstance(Constructor.java:423) > at > org.apache.hadoop.hbase.master.HMaster.constructMaster(HMaster.java:2456) > ... 5 more > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346)