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-8439] Add Flink shading to AWS credential provider s3 hadoop c…
…onfig
- Loading branch information
Showing
6 changed files
with
342 additions
and
181 deletions.
There are no files selected for viewing
65 changes: 65 additions & 0 deletions
65
...k-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/AbstractFileSystemFactory.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,65 @@ | ||
/* | ||
* 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 | ||
* | ||
* http://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.runtime.fs.hdfs; | ||
|
||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.core.fs.FileSystem; | ||
import org.apache.flink.core.fs.FileSystemFactory; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.io.IOException; | ||
import java.net.URI; | ||
|
||
/** Base class for Hadoop file system factories. */ | ||
public abstract class AbstractFileSystemFactory implements FileSystemFactory { | ||
private static final Logger LOG = LoggerFactory.getLogger(AbstractFileSystemFactory.class); | ||
|
||
/** Name of this factory for logging. */ | ||
private final String name; | ||
|
||
private final HadoopConfigLoader hadoopConfigLoader; | ||
|
||
protected AbstractFileSystemFactory(String name, HadoopConfigLoader hadoopConfigLoader) { | ||
this.name = name; | ||
this.hadoopConfigLoader = hadoopConfigLoader; | ||
} | ||
|
||
@Override | ||
public void configure(Configuration config) { | ||
hadoopConfigLoader.setFlinkConfig(config); | ||
} | ||
|
||
@Override | ||
public FileSystem create(URI fsUri) throws IOException { | ||
LOG.debug("Creating Hadoop file system (backed by " + name + ")"); | ||
LOG.debug("Loading Hadoop configuration for " + name); | ||
org.apache.hadoop.conf.Configuration hadoopConfig = hadoopConfigLoader.getOrLoadHadoopConfig(); | ||
org.apache.hadoop.fs.FileSystem fs = createHadoopFileSystem(); | ||
fs.initialize(getInitURI(fsUri, hadoopConfig), hadoopConfig); | ||
return new HadoopFileSystem(fs); | ||
} | ||
|
||
protected abstract org.apache.hadoop.fs.FileSystem createHadoopFileSystem(); | ||
|
||
protected abstract URI getInitURI( | ||
URI fsUri, org.apache.hadoop.conf.Configuration hadoopConfig); | ||
} | ||
|
132 changes: 132 additions & 0 deletions
132
...ms/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopConfigLoader.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,132 @@ | ||
/* | ||
* 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 | ||
* | ||
* http://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.runtime.fs.hdfs; | ||
|
||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.runtime.util.HadoopUtils; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import javax.annotation.Nonnull; | ||
|
||
import java.util.Set; | ||
|
||
/** This class lazily loads hadoop configuration from resettable Flink's configuration. */ | ||
public class HadoopConfigLoader { | ||
private static final Logger LOG = LoggerFactory.getLogger(HadoopConfigLoader.class); | ||
|
||
/** The prefixes that Flink adds to the Hadoop fs config. */ | ||
private final String[] flinkConfigPrefixes; | ||
|
||
/** Keys that are replaced (after prefix replacement, to give a more uniform experience | ||
* across different file system implementations. */ | ||
private final String[][] mirroredConfigKeys; | ||
|
||
/** Hadoop config prefix to replace Flink prefix. */ | ||
private final String hadoopConfigPrefix; | ||
|
||
private final Set<String> packagePrefixesToShade; | ||
private final Set<String> configKeysToShade; | ||
private final String flinkShadingPrefix; | ||
|
||
/** Flink's configuration object. */ | ||
private Configuration flinkConfig; | ||
|
||
/** Hadoop's configuration for the file systems, lazily initialized. */ | ||
private org.apache.hadoop.conf.Configuration hadoopConfig; | ||
|
||
public HadoopConfigLoader( | ||
@Nonnull String[] flinkConfigPrefixes, | ||
@Nonnull String[][] mirroredConfigKeys, | ||
@Nonnull String hadoopConfigPrefix, | ||
Set<String> packagePrefixesToShade, | ||
@Nonnull Set<String> configKeysToShade, | ||
@Nonnull String flinkShadingPrefix) { | ||
this.flinkConfigPrefixes = flinkConfigPrefixes; | ||
this.mirroredConfigKeys = mirroredConfigKeys; | ||
this.hadoopConfigPrefix = hadoopConfigPrefix; | ||
this.packagePrefixesToShade = packagePrefixesToShade; | ||
this.configKeysToShade = configKeysToShade; | ||
this.flinkShadingPrefix = flinkShadingPrefix; | ||
} | ||
|
||
public void setFlinkConfig(Configuration config) { | ||
flinkConfig = config; | ||
hadoopConfig = null; | ||
} | ||
|
||
/** get the loaded Hadoop config (or fall back to one loaded from the classpath). */ | ||
public org.apache.hadoop.conf.Configuration getOrLoadHadoopConfig() { | ||
org.apache.hadoop.conf.Configuration hadoopConfig = this.hadoopConfig; | ||
if (hadoopConfig == null) { | ||
if (flinkConfig != null) { | ||
hadoopConfig = mirrorCertainHadoopConfig(loadHadoopConfigFromFlink()); | ||
} | ||
else { | ||
LOG.warn("Flink configuration is not set prior to loading this configuration." | ||
+ " Using Hadoop configuration from the classpath."); | ||
hadoopConfig = new org.apache.hadoop.conf.Configuration(); | ||
} | ||
} | ||
this.hadoopConfig = hadoopConfig; | ||
return hadoopConfig; | ||
} | ||
|
||
// add additional config entries from the Flink config to the Hadoop config | ||
private org.apache.hadoop.conf.Configuration loadHadoopConfigFromFlink() { | ||
org.apache.hadoop.conf.Configuration hadoopConfig = HadoopUtils.getHadoopConfiguration(flinkConfig); | ||
for (String key : flinkConfig.keySet()) { | ||
for (String prefix : flinkConfigPrefixes) { | ||
if (key.startsWith(prefix)) { | ||
String value = flinkConfig.getString(key, null); | ||
String newKey = hadoopConfigPrefix + key.substring(prefix.length()); | ||
String newValue = fixHadoopConfig(key, flinkConfig.getString(key, null)); | ||
hadoopConfig.set(newKey, newValue); | ||
|
||
LOG.debug("Adding Flink config entry for {} as {}={} to Hadoop config", key, newKey, value); | ||
} | ||
} | ||
} | ||
return hadoopConfig; | ||
} | ||
|
||
// mirror certain keys to make use more uniform across implementations | ||
// with different keys | ||
private org.apache.hadoop.conf.Configuration mirrorCertainHadoopConfig( | ||
org.apache.hadoop.conf.Configuration hadoopConfig) { | ||
for (String[] mirrored : mirroredConfigKeys) { | ||
String value = hadoopConfig.get(mirrored[0], null); | ||
if (value != null) { | ||
hadoopConfig.set(mirrored[1], value); | ||
} | ||
} | ||
return hadoopConfig; | ||
} | ||
|
||
private String fixHadoopConfig(String key, String value) { | ||
return key != null && configKeysToShade.contains(key) ? | ||
shadeClassConfig(value) : value; | ||
} | ||
|
||
private String shadeClassConfig(String classConfig) { | ||
return packagePrefixesToShade.stream().anyMatch(classConfig::startsWith) ? | ||
flinkShadingPrefix + classConfig : classConfig; | ||
} | ||
} |
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
Oops, something went wrong.