Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
0.5.1
-
Spark version : 2.4.4
Hadoop version : 2.7.3
Databricks Runtime: 6.1
Description
Hi,
I'm trying to use hudi to write to one of the Azure storage container file systems, ADLS Gen 2 (abfs://). ABFS:// is one of the whitelisted file schemes. The issue I'm facing is that in HoodieROTablePathFilter it tries to get a file path passing in a blank hadoop configuration. This manifests as java.io.IOException: No FileSystem for scheme: abfss because it doesn't have any of the configuration in the environment.
The problematic line is
Stacktrace
java.io.IOException: No FileSystem for scheme: abfss
at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2660)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2667)
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.hudi.hadoop.HoodieROTablePathFilter.accept(HoodieROTablePathFilter.java:96)
at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$16.apply(InMemoryFileIndex.scala:349)