[
https://issues.apache.org/jira/browse/HIVE-21683?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16832242#comment-16832242
]
Todd Lipcon commented on HIVE-21683:
------------------------------------
Attached is one approach that uses javassist to dynamically wrap any newly
added methods for ProxyFileSystem23. I verified this fixes the issue.
I'll also momentarily attach a more "straightforward" approach which just adds
the new method. The problem with this latter approach is that it won't compile
against Hadoop 3.1, since the new methods are in Hadoop 3.3 (not yet released).
We could just wait until that releases before committing if we want to go with
the simpler approach, though.
> ProxyFileSystem breaks with Hadoop trunk
> ----------------------------------------
>
> Key: HIVE-21683
> URL: https://issues.apache.org/jira/browse/HIVE-21683
> Project: Hive
> Issue Type: Bug
> Reporter: Todd Lipcon
> Assignee: Todd Lipcon
> Priority: Major
> Attachments: hive-21683-javassist.patch
>
>
> When trying to run with a recent build of Hadoop which includes HADOOP-15229
> I ran into the following stack:
> {code}
> Caused by: java.lang.IllegalArgumentException: Wrong FS:
> pfile:/src/hive/itests/qtest/target/warehouse/src/kv1.txt, expected: file:///
> at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:793)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:86)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:636)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:930)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:631)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:456)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.<init>(ChecksumFileSystem.java:153)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSystem.java:354)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.ChecksumFileSystem.lambda$openFileWithOptions$0(ChecksumFileSystem.java:846)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at org.apache.hadoop.util.LambdaUtils.eval(LambdaUtils.java:52)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.ChecksumFileSystem.openFileWithOptions(ChecksumFileSystem.java:845)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.fs.FileSystem$FSDataInputStreamBuilder.build(FileSystem.java:4522)
> ~[hadoop-common-3.1.1.6.0.99.0-135.jar:?]
> at
> org.apache.hadoop.mapred.LineRecordReader.<init>(LineRecordReader.java:115)
> ~[hadoop-mapreduce-client-core-3.1.1.6.0.99.0-135.jar:?]{code}
> We need to add appropriate path-swizzling wrappers for the new APIs in
> ProxyFileSystem23
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)