tillrohrmann commented on a change in pull request #8117: [FLINK-12115] 
[filesystems]: Add support for AzureFS
URL: https://github.com/apache/flink/pull/8117#discussion_r280414648
 
 

 ##########
 File path: 
flink-filesystems/flink-azure-fs-hadoop/src/main/java/org/apache/flink/fs/azurefs/AbstractAzureFSFactory.java
 ##########
 @@ -48,6 +55,33 @@ public void configure(Configuration config) {
        public FileSystem create(URI fsUri) throws IOException {
                checkNotNull(fsUri, "passed file system URI object should not 
be null");
                LOG.info("Trying to load and instantiate Azure File System");
-               return new AzureFileSystem(fsUri, flinkConfig);
+               return new HadoopFileSystem(createInitializedAzureFS(fsUri, 
flinkConfig));
+       }
+
+       // uri is of the form: 
wasb(s)://[email protected]/testDir
+       private org.apache.hadoop.fs.FileSystem createInitializedAzureFS(URI 
fsUri, Configuration flinkConfig) throws IOException {
+               org.apache.hadoop.conf.Configuration hadoopConfig = 
HadoopUtils.getHadoopConfiguration(flinkConfig);
 
 Review comment:
   Looking at the details of the `HadoopConfigLoader` and the `HadoopUtils`, it 
actually makes a bigger difference than thought. `HadoopUtils` loads the Hadoop 
configuration from a set of specified locations (assuming that Hadoop is 
available). In contrast to that `HadoopConfigLoader` creates a vanilla 
`HadoopConfiguration` and populates it with the contents of the Flink 
configuration. So the `HadoopConfigLoader` is actually used in the S3 
FileSystem implementations to completely encapsulate the Hadoop dependencies so 
that the S3 filesystem implementation is completely self-contained (see 
FLINK-10362). I think we should do the same for the Azure FS since it is not 
relevant for the user which implementation we use here (Hadoop) and one should 
also be able to use it without having Hadoop installed.
   
   I would propose to move the `HadoopConfigLoader` to the `flink-hadoop-fs` 
module since all implementations which use a Hadoop based implementation will 
depend on this one. I think the only filesystem not doing it correctly at the 
moment is the `SwiftFileSystem`. Once we have moved the utility, we should 
create a JIRA issue to update this file system as a follow up.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to