forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-7643] [core] Rework FileSystem loading to use factories
This makes sure that configurations are loaded once and file system instances are properly reused by scheme and authority. This also factors out a lot of the special treatment of Hadoop file systems and simply makes the Hadoop File System factory the default fallback factory.
- Loading branch information
1 parent
a5ef09b
commit 536675b
Showing
26 changed files
with
936 additions
and
799 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
304 changes: 126 additions & 178 deletions
304
flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
Large diffs are not rendered by default.
Oops, something went wrong.
56 changes: 56 additions & 0 deletions
56
flink-core/src/main/java/org/apache/flink/core/fs/FileSystemFactory.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,56 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.core.fs; | ||
|
||
import org.apache.flink.annotation.PublicEvolving; | ||
import org.apache.flink.configuration.Configuration; | ||
|
||
import java.io.IOException; | ||
import java.net.URI; | ||
|
||
/** | ||
* A factory to create file systems. | ||
* | ||
* <p>The factory is typically configured via {@link #configure(Configuration)} before | ||
* creating file systems via {@link #create(URI)}. | ||
*/ | ||
@PublicEvolving | ||
public interface FileSystemFactory { | ||
|
||
/** | ||
* Applies the given configuration to this factory. All future file system | ||
* instantiations via {@link #create(URI)} should take the configuration into | ||
* account. | ||
* | ||
* @param config The configuration to apply. | ||
*/ | ||
void configure(Configuration config); | ||
|
||
/** | ||
* Creates a new file system for the given file system URI. | ||
* The URI describes the type of file system (via its scheme) and optionally the | ||
* authority (for example the host) of the file system. | ||
* | ||
* @param fsUri The URI that describes the file system. | ||
* @return A new instance of the specified file system. | ||
* | ||
* @throws IOException Thrown if the file system could not be instantiated. | ||
*/ | ||
FileSystem create(URI fsUri) throws IOException; | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
48 changes: 48 additions & 0 deletions
48
flink-core/src/main/java/org/apache/flink/core/fs/UnsupportedFileSystemSchemeException.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,48 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.core.fs; | ||
|
||
import java.io.IOException; | ||
|
||
/** | ||
* An exception to indicate that a specific file system scheme is not supported. | ||
*/ | ||
public class UnsupportedFileSystemSchemeException extends IOException { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
/** | ||
* Creates a new exception with the given message. | ||
* | ||
* @param message The exception message | ||
*/ | ||
public UnsupportedFileSystemSchemeException(String message) { | ||
super(message); | ||
} | ||
|
||
/** | ||
* Creates a new exception with the given message and cause. | ||
* | ||
* @param message The exception message | ||
* @param cause The exception cause | ||
*/ | ||
public UnsupportedFileSystemSchemeException(String message, Throwable cause) { | ||
super(message, cause); | ||
} | ||
} |
81 changes: 81 additions & 0 deletions
81
...-core/src/main/java/org/apache/flink/core/fs/factories/HadoopFileSystemFactoryLoader.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,81 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.core.fs.factories; | ||
|
||
import org.apache.flink.core.fs.FileSystemFactory; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
/** | ||
* A utility class to check and reflectively load the Hadoop file system factory. | ||
*/ | ||
public class HadoopFileSystemFactoryLoader { | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(HadoopFileSystemFactoryLoader.class); | ||
|
||
private static final String FACTORY_CLASS = "org.apache.flink.runtime.fs.hdfs.HadoopFsFactory"; | ||
|
||
private static final String HADOOP_CONFIG_CLASS = "org.apache.hadoop.conf.Configuration"; | ||
|
||
private static final String HADOOP_FS_CLASS = "org.apache.hadoop.fs.FileSystem"; | ||
|
||
|
||
/** | ||
* Loads the FileSystemFactory for the Hadoop-backed file systems. | ||
*/ | ||
public static FileSystemFactory loadFactory() { | ||
final ClassLoader cl = HadoopFileSystemFactoryLoader.class.getClassLoader(); | ||
|
||
// first, see if the Flink runtime classes are available | ||
final Class<? extends FileSystemFactory> factoryClass; | ||
try { | ||
factoryClass = Class.forName(FACTORY_CLASS, false, cl).asSubclass(FileSystemFactory.class); | ||
} | ||
catch (ClassNotFoundException e) { | ||
LOG.info("No Flink runtime dependency present - the extended set of supported File Systems " + | ||
"via Hadoop is not available."); | ||
return new UnsupportedSchemeFactory("Flink runtime classes missing in classpath/dependencies."); | ||
} | ||
catch (Exception | LinkageError e) { | ||
LOG.warn("Flink's Hadoop file system factory could not be loaded", e); | ||
return new UnsupportedSchemeFactory("Flink's Hadoop file system factory could not be loaded", e); | ||
} | ||
|
||
// check (for eager and better exception messages) if the Hadoop classes are available here | ||
try { | ||
Class.forName(HADOOP_CONFIG_CLASS, false, cl); | ||
Class.forName(HADOOP_FS_CLASS, false, cl); | ||
} | ||
catch (ClassNotFoundException e) { | ||
LOG.info("Hadoop is not in the classpath/dependencies - the extended set of supported File Systems " + | ||
"via Hadoop is not available."); | ||
return new UnsupportedSchemeFactory("Hadoop is not in the classpath/dependencies."); | ||
} | ||
|
||
// Create the factory. | ||
try { | ||
return factoryClass.newInstance(); | ||
} | ||
catch (Exception | LinkageError e) { | ||
LOG.warn("Flink's Hadoop file system factory could not be created", e); | ||
return new UnsupportedSchemeFactory("Flink's Hadoop file system factory could not be created", e); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
75 changes: 75 additions & 0 deletions
75
flink-core/src/main/java/org/apache/flink/core/fs/factories/MapRFsFactory.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,75 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.core.fs.factories; | ||
|
||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.core.fs.FileSystem; | ||
import org.apache.flink.core.fs.FileSystemFactory; | ||
|
||
import java.io.IOException; | ||
import java.lang.reflect.Constructor; | ||
import java.lang.reflect.InvocationTargetException; | ||
import java.net.URI; | ||
|
||
/** | ||
* A factory for the MapR file system. | ||
* | ||
* <p>This factory tries to reflectively instantiate the MapR file system. It can only be | ||
* used when the MapR FS libraries are in the classpath. | ||
*/ | ||
public class MapRFsFactory implements FileSystemFactory { | ||
|
||
private static final String MAPR_FILESYSTEM_CLASS = "org.apache.flink.runtime.fs.maprfs.MapRFileSystem"; | ||
|
||
@Override | ||
public void configure(Configuration config) { | ||
// nothing to configure based on the configuration here | ||
} | ||
|
||
@Override | ||
public FileSystem create(URI fsUri) throws IOException { | ||
try { | ||
Class<? extends FileSystem> fsClass = Class.forName( | ||
MAPR_FILESYSTEM_CLASS, false, getClass().getClassLoader()).asSubclass(FileSystem.class); | ||
|
||
Constructor<? extends FileSystem> constructor = fsClass.getConstructor(URI.class); | ||
|
||
try { | ||
return constructor.newInstance(fsUri); | ||
} | ||
catch (InvocationTargetException e) { | ||
throw e.getTargetException(); | ||
} | ||
} | ||
catch (ClassNotFoundException e) { | ||
throw new IOException("Could not load MapR file system class '" + MAPR_FILESYSTEM_CLASS + | ||
"\'. Please make sure the Flink runtime classes are part of the classpath or dependencies.", e); | ||
} | ||
catch (LinkageError e) { | ||
throw new IOException("Some of the MapR FS or required Hadoop classes seem to be missing or incompatible. " | ||
+ "Please check that a compatible version of the MapR Hadoop libraries is in the classpath.", e); | ||
} | ||
catch (IOException e) { | ||
throw e; | ||
} | ||
catch (Throwable t) { | ||
throw new IOException("Could not instantiate MapR file system class '" + MAPR_FILESYSTEM_CLASS + "'.", t); | ||
} | ||
} | ||
} |
Oops, something went wrong.