Repository: spark Updated Branches: refs/heads/master 6fc2740eb -> cc567b663
[SPARK-11695][CORE] Set s3a credentials Set s3a credentials when creating a new default hadoop configuration. Author: Chris Bannister <[email protected]> Closes #9663 from Zariel/set-s3a-creds. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cc567b66 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cc567b66 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cc567b66 Branch: refs/heads/master Commit: cc567b6634c3142125526f4875795c1b1e862838 Parents: 6fc2740 Author: Chris Bannister <[email protected]> Authored: Tue Nov 17 10:03:46 2015 -0800 Committer: Marcelo Vanzin <[email protected]> Committed: Tue Nov 17 10:03:46 2015 -0800 ---------------------------------------------------------------------- .../org/apache/spark/deploy/SparkHadoopUtil.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/spark/blob/cc567b66/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala ---------------------------------------------------------------------- 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 d606b80..59e9056 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) => --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
