Description
Java tool currently does not work with HDFS paths:
$ mvn clean package -DskipTests $ export UBERJAR=tools/target/orc-tools-1.3.0-SNAPSHOT-uber.jar $ java -jar $UBERJAR meta hdfs://nn-host/foobar.snappy.orc log4j:WARN No appenders could be found for logger (org.apache.hadoop.metrics2.lib.MutableMetricsFactory). log4j:WARN Please initialize the log4j system properly. log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more info. Exception in thread "main" java.io.IOException: No FileSystem for scheme: hdfs at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2584) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2591) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:91) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2630) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2612) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:370) at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296) at org.apache.orc.tools.FileDump.getAllFilesInPath(FileDump.java:248) at org.apache.orc.tools.FileDump.main(FileDump.java:121) at org.apache.orc.tools.Driver.main(Driver.java:100)
This is because DistributedFileSystem is not registered as a subclass of FileSystem SPI.
$ unzip -p tools/target/orc-tools-1.3.0-SNAPSHOT-uber.jar META-INF/services/org.apache.hadoop.fs.FileSystem # ... org.apache.hadoop.fs.LocalFileSystem org.apache.hadoop.fs.viewfs.ViewFileSystem org.apache.hadoop.fs.ftp.FTPFileSystem org.apache.hadoop.fs.HarFileSystem
We can fix this by simply adding containerDescriptorHandler to the assembly descriptor. Now the same command gives:
org.apache.hadoop.fs.LocalFileSystem org.apache.hadoop.fs.viewfs.ViewFileSystem org.apache.hadoop.fs.ftp.FTPFileSystem org.apache.hadoop.fs.HarFileSystem org.apache.hadoop.hdfs.DistributedFileSystem org.apache.hadoop.hdfs.web.HftpFileSystem org.apache.hadoop.hdfs.web.HsftpFileSystem org.apache.hadoop.hdfs.web.WebHdfsFileSystem org.apache.hadoop.hdfs.web.SWebHdfsFileSystem
and we can use the uberjar with HDFS paths.