aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorChris Bannister <chris.bannister@swiftkey.com>2015-11-17 10:03:46 -0800
committerMarcelo Vanzin <vanzin@cloudera.com>2015-11-17 10:03:46 -0800
commitcc567b6634c3142125526f4875795c1b1e862838 (patch)
treed5de6f972f4554d51401eb2ac2877784e7036531
parent6fc2740ebb59aca1aa0ee1e93658a7e4e69de33c (diff)
downloadspark-cc567b6634c3142125526f4875795c1b1e862838.tar.gz
spark-cc567b6634c3142125526f4875795c1b1e862838.tar.bz2
spark-cc567b6634c3142125526f4875795c1b1e862838.zip
[SPARK-11695][CORE] Set s3a credentials
Set s3a credentials when creating a new default hadoop configuration. Author: Chris Bannister <chris.bannister@swiftkey.com> Closes #9663 from Zariel/set-s3a-creds.
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala13
1 files changed, 9 insertions, 4 deletions
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index d606b80c03..59e90564b3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -92,10 +92,15 @@ class SparkHadoopUtil extends Logging {
// Explicitly check for S3 environment variables
if (System.getenv("AWS_ACCESS_KEY_ID") != null &&
System.getenv("AWS_SECRET_ACCESS_KEY") != null) {
- hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
- hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
- hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
- hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
+ val keyId = System.getenv("AWS_ACCESS_KEY_ID")
+ val accessKey = System.getenv("AWS_SECRET_ACCESS_KEY")
+
+ hadoopConf.set("fs.s3.awsAccessKeyId", keyId)
+ hadoopConf.set("fs.s3n.awsAccessKeyId", keyId)
+ hadoopConf.set("fs.s3a.access.key", keyId)
+ hadoopConf.set("fs.s3.awsSecretAccessKey", accessKey)
+ hadoopConf.set("fs.s3n.awsSecretAccessKey", accessKey)
+ hadoopConf.set("fs.s3a.secret.key", accessKey)
}
// Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar"
conf.getAll.foreach { case (key, value) =>