You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by li...@apache.org on 2018/09/12 00:26:24 UTC

hadoop git commit: HADOOP-14833. Remove s3a user:secret authentication. Contributed by Steve Loughran

Repository: hadoop
Updated Branches:
  refs/heads/trunk 9c238ffc3 -> 87f63b647


HADOOP-14833. Remove s3a user:secret authentication. Contributed by Steve Loughran


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/87f63b64
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/87f63b64
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/87f63b64

Branch: refs/heads/trunk
Commit: 87f63b6479330840e9d708a729355948bb91fd4d
Parents: 9c238ff
Author: Mingliang Liu <li...@apache.org>
Authored: Tue Sep 11 17:18:42 2018 -0700
Committer: Mingliang Liu <li...@apache.org>
Committed: Tue Sep 11 17:18:42 2018 -0700

----------------------------------------------------------------------
 .../src/main/resources/core-default.xml         |   5 +-
 .../fs/s3a/BasicAWSCredentialsProvider.java     |  62 ----------
 .../org/apache/hadoop/fs/s3a/Constants.java     |   7 --
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  17 +--
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  49 ++++----
 .../fs/s3a/SimpleAWSCredentialsProvider.java    |  27 ++---
 .../fs/s3a/TemporaryAWSCredentialsProvider.java |  24 ++--
 .../hadoop/fs/s3native/S3xLoginHelper.java      | 121 +++++++++----------
 .../src/site/markdown/tools/hadoop-aws/index.md |  32 +++--
 .../tools/hadoop-aws/troubleshooting_s3a.md     |  44 ++-----
 .../fs/s3a/ITestS3AAWSCredentialsProvider.java  |  20 +--
 .../hadoop/fs/s3a/ITestS3AConfiguration.java    |  51 +-------
 .../fs/s3a/TestS3AAWSCredentialsProvider.java   |  50 +++++---
 .../hadoop/fs/s3a/TestSSEConfiguration.java     |  31 +----
 .../hadoop/fs/s3native/TestS3xLoginHelper.java  |  70 ++++-------
 15 files changed, 206 insertions(+), 404 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index 7a0e1a8..fdd7a87 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -977,9 +977,8 @@
 
     If unspecified, then the default list of credential provider classes,
     queried in sequence, is:
-    1. org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider: supports static
-        configuration of AWS access key ID and secret access key.  See also
-        fs.s3a.access.key and fs.s3a.secret.key.
+    1. org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider:
+       Uses the values of fs.s3a.access.key and fs.s3a.secret.key.
     2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
         configuration of AWS access key ID and secret access key in
         environment variables named AWS_ACCESS_KEY_ID and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
deleted file mode 100644
index 01bcc6a..0000000
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/BasicAWSCredentialsProvider.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * 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.hadoop.fs.s3a;
-
-import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.BasicAWSCredentials;
-import com.amazonaws.auth.AWSCredentials;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * BasicAWSCredentialsProvider supports static configuration of access key ID
- * and secret access key for use with the AWS SDK.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class BasicAWSCredentialsProvider implements AWSCredentialsProvider {
-  public static final String NAME
-      = "org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider";
-  private final String accessKey;
-  private final String secretKey;
-
-
-  public BasicAWSCredentialsProvider(String accessKey, String secretKey) {
-    this.accessKey = accessKey;
-    this.secretKey = secretKey;
-  }
-
-  public AWSCredentials getCredentials() {
-    if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
-      return new BasicAWSCredentials(accessKey, secretKey);
-    }
-    throw new CredentialInitializationException(
-        "Access key or secret key is null");
-  }
-
-  public void refresh() {}
-
-  @Override
-  public String toString() {
-    return getClass().getSimpleName();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index 8beefff..a714e92 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -305,13 +305,6 @@ public final class Constants {
   public static final String SERVER_SIDE_ENCRYPTION_KEY =
       "fs.s3a.server-side-encryption.key";
 
-  /**
-   * The original key name. Never used in ASF releases,
-   * but did get into downstream products.
-   */
-  static final String OLD_S3A_SERVER_SIDE_ENCRYPTION_KEY
-      = "fs.s3a.server-side-encryption-key";
-
   //override signature algorithm used for signing requests
   public static final String SIGNING_ALGORITHM = "fs.s3a.signing-algorithm";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index 72a5fde..fcbcbcc 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -208,14 +208,15 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
   /** Add any deprecated keys. */
   @SuppressWarnings("deprecation")
   private static void addDeprecatedKeys() {
-    Configuration.addDeprecations(
-        new Configuration.DeprecationDelta[]{
-            // never shipped in an ASF release, but did get into the wild.
-            new Configuration.DeprecationDelta(
-                OLD_S3A_SERVER_SIDE_ENCRYPTION_KEY,
-                SERVER_SIDE_ENCRYPTION_KEY)
-        });
-    Configuration.reloadExistingConfigurations();
+    // this is retained as a placeholder for when new deprecated keys
+    // need to be added.
+    Configuration.DeprecationDelta[] deltas = {
+    };
+
+    if (deltas.length > 0) {
+      Configuration.addDeprecations(deltas);
+      Configuration.reloadExistingConfigurations();
+    }
   }
 
   static {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
index 9908fd1..c3f1e93 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java
@@ -592,9 +592,7 @@ public final class S3AUtils {
     Class<?>[] awsClasses = loadAWSProviderClasses(conf,
         AWS_CREDENTIALS_PROVIDER);
     if (awsClasses.length == 0) {
-      S3xLoginHelper.Login creds = getAWSAccessKeys(binding, conf);
-      credentials.add(new BasicAWSCredentialsProvider(
-              creds.getUser(), creds.getPassword()));
+      credentials.add(new SimpleAWSCredentialsProvider(binding, conf));
       credentials.add(new EnvironmentVariableCredentialsProvider());
       credentials.add(InstanceProfileCredentialsProvider.getInstance());
     } else {
@@ -725,7 +723,6 @@ public final class S3AUtils {
 
   /**
    * Return the access key and secret for S3 API use.
-   * Credentials may exist in configuration, within credential providers
    * or indicated in the UserInfo of the name URI param.
    * @param name the URI for which we need the access keys; may be null
    * @param conf the Configuration object to interrogate for keys.
@@ -734,25 +731,19 @@ public final class S3AUtils {
    */
   public static S3xLoginHelper.Login getAWSAccessKeys(URI name,
       Configuration conf) throws IOException {
-    S3xLoginHelper.Login login =
-        S3xLoginHelper.extractLoginDetailsWithWarnings(name);
+    S3xLoginHelper.rejectSecretsInURIs(name);
     Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
         conf, S3AFileSystem.class);
     String bucket = name != null ? name.getHost() : "";
 
-    // build the secrets. as getPassword() uses the last arg as
-    // the return value if non-null, the ordering of
-    // login -> bucket -> base is critical
+    // get the secrets from the configuration
 
-    // get the bucket values
-    String accessKey = lookupPassword(bucket, c, ACCESS_KEY,
-        login.getUser());
+    // get the access key
+    String accessKey = lookupPassword(bucket, c, ACCESS_KEY);
 
-    // finally the base
-    String secretKey = lookupPassword(bucket, c, SECRET_KEY,
-        login.getPassword());
+    // and the secret
+    String secretKey = lookupPassword(bucket, c, SECRET_KEY);
 
-    // and override with any per bucket values
     return new S3xLoginHelper.Login(accessKey, secretKey);
   }
 
@@ -768,6 +759,7 @@ public final class S3AUtils {
    * @throws IOException on any IO problem
    * @throws IllegalArgumentException bad arguments
    */
+  @Deprecated
   public static String lookupPassword(
       String bucket,
       Configuration conf,
@@ -783,6 +775,24 @@ public final class S3AUtils {
    * @param bucket bucket or "" if none known
    * @param conf configuration
    * @param baseKey base key to look up, e.g "fs.s3a.secret.key"
+   * @return a password or "".
+   * @throws IOException on any IO problem
+   * @throws IllegalArgumentException bad arguments
+   */
+  public static String lookupPassword(
+      String bucket,
+      Configuration conf,
+      String baseKey)
+      throws IOException {
+    return lookupPassword(bucket, conf, baseKey, null, "");
+  }
+
+  /**
+   * Get a password from a configuration, including JCEKS files, handling both
+   * the absolute key and bucket override.
+   * @param bucket bucket or "" if none known
+   * @param conf configuration
+   * @param baseKey base key to look up, e.g "fs.s3a.secret.key"
    * @param overrideVal override value: if non empty this is used instead of
    * querying the configuration.
    * @param defVal value to return if there is no password
@@ -1387,10 +1397,7 @@ public final class S3AUtils {
   static String getServerSideEncryptionKey(String bucket,
       Configuration conf) {
     try {
-      return lookupPassword(bucket, conf,
-          SERVER_SIDE_ENCRYPTION_KEY,
-          getPassword(conf, OLD_S3A_SERVER_SIDE_ENCRYPTION_KEY,
-              null, null));
+      return lookupPassword(bucket, conf, SERVER_SIDE_ENCRYPTION_KEY);
     } catch (IOException e) {
       LOG.error("Cannot retrieve " + SERVER_SIDE_ENCRYPTION_KEY, e);
       return "";
@@ -1412,7 +1419,7 @@ public final class S3AUtils {
       Configuration conf) throws IOException {
     S3AEncryptionMethods sse = S3AEncryptionMethods.getMethod(
         lookupPassword(bucket, conf,
-            SERVER_SIDE_ENCRYPTION_ALGORITHM, null));
+            SERVER_SIDE_ENCRYPTION_ALGORITHM));
     String sseKey = getServerSideEncryptionKey(bucket, conf);
     int sseKeyLen = StringUtils.isBlank(sseKey) ? 0 : sseKey.length();
     String diagnostics = passwordDiagnostics(sseKey, "key");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java
index 7f9e57e..b31b72a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/SimpleAWSCredentialsProvider.java
@@ -21,10 +21,12 @@ package org.apache.hadoop.fs.s3a;
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.BasicAWSCredentials;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.security.ProviderUtils;
 
 import java.io.IOException;
@@ -32,10 +34,10 @@ import java.net.URI;
 
 import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
+import static org.apache.hadoop.fs.s3a.S3AUtils.getAWSAccessKeys;
 
 /**
  * Support simple credentials for authenticating with AWS.
- * Keys generated in URLs are not supported.
  *
  * Please note that users may reference this class name from configuration
  * property fs.s3a.aws.credentials.provider.  Therefore, changing the class name
@@ -49,26 +51,17 @@ public class SimpleAWSCredentialsProvider implements AWSCredentialsProvider {
       = "org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider";
   private String accessKey;
   private String secretKey;
-  private IOException lookupIOE;
 
-  public SimpleAWSCredentialsProvider(URI uri, Configuration conf) {
-    try {
-      String bucket = uri != null ? uri.getHost() : "";
-      Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
-          conf, S3AFileSystem.class);
-      this.accessKey = S3AUtils.lookupPassword(bucket, c, ACCESS_KEY, null);
-      this.secretKey = S3AUtils.lookupPassword(bucket, c, SECRET_KEY, null);
-    } catch (IOException e) {
-      lookupIOE = e;
-    }
+  public SimpleAWSCredentialsProvider(URI uri, Configuration conf)
+      throws IOException {
+
+      S3xLoginHelper.Login login = getAWSAccessKeys(uri, conf);
+      this.accessKey = login.getUser();
+      this.secretKey = login.getPassword();
   }
 
+  @Override
   public AWSCredentials getCredentials() {
-    if (lookupIOE != null) {
-      // propagate any initialization problem
-      throw new CredentialInitializationException(lookupIOE.toString(),
-          lookupIOE);
-    }
     if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) {
       return new BasicAWSCredentials(accessKey, secretKey);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
index 3b89bde..d42f68e 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/TemporaryAWSCredentialsProvider.java
@@ -50,32 +50,26 @@ public class TemporaryAWSCredentialsProvider implements AWSCredentialsProvider {
   private String accessKey;
   private String secretKey;
   private String sessionToken;
-  private IOException lookupIOE;
 
-  public TemporaryAWSCredentialsProvider(Configuration conf) {
+  public TemporaryAWSCredentialsProvider(Configuration conf)
+      throws IOException {
     this(null, conf);
   }
 
-  public TemporaryAWSCredentialsProvider(URI uri, Configuration conf) {
-    try {
+  public TemporaryAWSCredentialsProvider(URI uri, Configuration conf)
+      throws IOException {
+
       // determine the bucket
       String bucket = uri != null ? uri.getHost():  "";
       Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
           conf, S3AFileSystem.class);
-      this.accessKey = lookupPassword(bucket, c, ACCESS_KEY, null);
-      this.secretKey = lookupPassword(bucket, c, SECRET_KEY, null);
-      this.sessionToken = lookupPassword(bucket, c, SESSION_TOKEN, null);
-    } catch (IOException e) {
-      lookupIOE = e;
-    }
+      this.accessKey = lookupPassword(bucket, c, ACCESS_KEY);
+      this.secretKey = lookupPassword(bucket, c, SECRET_KEY);
+      this.sessionToken = lookupPassword(bucket, c, SESSION_TOKEN);
   }
 
+  @Override
   public AWSCredentials getCredentials() {
-    if (lookupIOE != null) {
-      // propagate any initialization problem
-      throw new CredentialInitializationException(lookupIOE.toString(),
-          lookupIOE);
-    }
     if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)
         && !StringUtils.isEmpty(sessionToken)) {
       return new BasicSessionCredentials(accessKey, secretKey, sessionToken);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
index 9e2f34d..60d4b76 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
@@ -18,56 +18,57 @@
 
 package org.apache.hadoop.fs.s3native;
 
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.UnsupportedEncodingException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URLDecoder;
-import java.util.Objects;
 
 import static org.apache.commons.lang3.StringUtils.equalsIgnoreCase;
 
 /**
  * Class to aid logging in to S3 endpoints.
  * It is in S3N so that it can be used across all S3 filesystems.
+ *
+ * The core function of this class was the extraction and decoding of user:secret
+ * information from filesystems URIs. As this is no longer supported,
+ * its role has been reduced to checking for secrets in the URI and rejecting
+ * them where found.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
 public final class S3xLoginHelper {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(S3xLoginHelper.class);
 
   private S3xLoginHelper() {
   }
 
   public static final String LOGIN_WARNING =
       "The Filesystem URI contains login details."
-      +" This is insecure and may be unsupported in future.";
-
-  public static final String PLUS_WARNING =
-      "Secret key contains a special character that should be URL encoded! " +
-          "Attempting to resolve...";
-
-  public static final String PLUS_UNENCODED = "+";
-  public static final String PLUS_ENCODED = "%2B";
+      +" This authentication mechanism is no longer supported.";
 
   /**
-   * Build the filesystem URI. This can include stripping down of part
-   * of the URI.
+   * Build the filesystem URI.
    * @param uri filesystem uri
    * @return the URI to use as the basis for FS operation and qualifying paths.
    * @throws IllegalArgumentException if the URI is in some way invalid.
    */
   public static URI buildFSURI(URI uri) {
+    // look for login secrets and fail if they are present.
+    rejectSecretsInURIs(uri);
     Objects.requireNonNull(uri, "null uri");
     Objects.requireNonNull(uri.getScheme(), "null uri.getScheme()");
     if (uri.getHost() == null && uri.getAuthority() != null) {
-      Objects.requireNonNull(uri.getHost(), "null uri host." +
-          " This can be caused by unencoded / in the password string");
+      Objects.requireNonNull(uri.getHost(), "null uri host.");
     }
     Objects.requireNonNull(uri.getHost(), "null uri host.");
     return URI.create(uri.getScheme() + "://" + uri.getHost());
@@ -86,17 +87,14 @@ public final class S3xLoginHelper {
   }
 
   /**
-   * Extract the login details from a URI, logging a warning if
-   * the URI contains these.
+   * Extract the login details from a URI, raising an exception if
+   * the URI contains them.
    * @param name URI of the filesystem, can be null
-   * @return a login tuple, possibly empty.
+   * @throws IllegalArgumentException if there is a secret in the URI.
    */
-  public static Login extractLoginDetailsWithWarnings(URI name) {
+  public static void rejectSecretsInURIs(URI name) {
     Login login = extractLoginDetails(name);
-    if (login.hasLogin()) {
-      LOG.warn(LOGIN_WARNING);
-    }
-    return login;
+    Preconditions.checkArgument(!login.hasLogin(), LOGIN_WARNING);
   }
 
   /**
@@ -104,43 +102,34 @@ public final class S3xLoginHelper {
    * @param name URI of the filesystem, may be null
    * @return a login tuple, possibly empty.
    */
-  public static Login extractLoginDetails(URI name) {
+  @VisibleForTesting
+  static Login extractLoginDetails(URI name) {
     if (name == null) {
       return Login.EMPTY;
     }
 
-    try {
-      String authority = name.getAuthority();
-      if (authority == null) {
-        return Login.EMPTY;
-      }
-      int loginIndex = authority.indexOf('@');
-      if (loginIndex < 0) {
-        // no login
-        return Login.EMPTY;
-      }
-      String login = authority.substring(0, loginIndex);
-      int loginSplit = login.indexOf(':');
-      if (loginSplit > 0) {
-        String user = login.substring(0, loginSplit);
-        String encodedPassword = login.substring(loginSplit + 1);
-        if (encodedPassword.contains(PLUS_UNENCODED)) {
-          LOG.warn(PLUS_WARNING);
-          encodedPassword = encodedPassword.replaceAll("\\" + PLUS_UNENCODED,
-              PLUS_ENCODED);
-        }
-        String password = URLDecoder.decode(encodedPassword,
-            "UTF-8");
-        return new Login(user, password);
-      } else if (loginSplit == 0) {
-        // there is no user, just a password. In this case, there's no login
-        return Login.EMPTY;
-      } else {
-        return new Login(login, "");
-      }
-    } catch (UnsupportedEncodingException e) {
-      // this should never happen; translate it if it does.
-      throw new RuntimeException(e);
+    String authority = name.getAuthority();
+    if (authority == null) {
+      return Login.EMPTY;
+    }
+    int loginIndex = authority.indexOf('@');
+    if (loginIndex < 0) {
+      // no login
+      return Login.EMPTY;
+    }
+    String login = authority.substring(0, loginIndex);
+    int loginSplit = login.indexOf(':');
+    if (loginSplit > 0) {
+      String user = login.substring(0, loginSplit);
+      String encodedPassword = login.substring(loginSplit + 1);
+      return new Login(user, encodedPassword.isEmpty()? "": "password removed");
+    } else if (loginSplit == 0) {
+      // there is no user, just a password. In this case, there's no login
+      return Login.EMPTY;
+    } else {
+      // loginSplit < 0: there is no ":".
+      // return a login with a null password
+      return new Login(login, "");
     }
   }
 
@@ -159,7 +148,7 @@ public final class S3xLoginHelper {
       // reconstruct the uri with the default port set
       try {
         uri = new URI(uri.getScheme(),
-            null,
+            uri.getUserInfo(),
             uri.getHost(),
             defaultPort,
             uri.getPath(),
@@ -262,10 +251,10 @@ public final class S3xLoginHelper {
 
     /**
      * Predicate to verify login details are defined.
-     * @return true if the username is defined (not null, not empty).
+     * @return true if the instance contains login information.
      */
     public boolean hasLogin() {
-      return StringUtils.isNotEmpty(user);
+      return StringUtils.isNotEmpty(password) || StringUtils.isNotEmpty(user);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
index 2dee10a..7ab3b0e 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md
@@ -197,7 +197,7 @@ to safely save the output of queries directly into S3 object stores
 through the S3A filesystem.
 
 
-### Warning #3: Object stores have differerent authorization models
+### Warning #3: Object stores have different authorization models
 
 The object authorization model of S3 is much different from the file
 authorization model of HDFS and traditional file systems.
@@ -222,13 +222,12 @@ Your AWS credentials not only pay for services, they offer read and write
 access to the data. Anyone with the credentials can not only read your datasets
 —they can delete them.
 
-Do not inadvertently share these credentials through means such as
+Do not inadvertently share these credentials through means such as:
 
 1. Checking in to SCM any configuration files containing the secrets.
 1. Logging them to a console, as they invariably end up being seen.
-1. Defining filesystem URIs with the credentials in the URL, such as
-`s3a://AK0010:secret@landsat-pds/`. They will end up in logs and error messages.
 1. Including the secrets in bug reports.
+1. Logging the `AWS_` environment variables.
 
 If you do any of these: change your credentials immediately!
 
@@ -242,6 +241,11 @@ The client supports multiple authentication mechanisms and can be configured as
 which mechanisms to use, and their order of use. Custom implementations
 of `com.amazonaws.auth.AWSCredentialsProvider` may also be used.
 
+*Important*: The S3A connector no longer supports username and secrets
+in URLs of the form `s3a://key:secret@bucket/`.
+It is near-impossible to stop those secrets being logged —which is why
+a warning has been printed since Hadoop 2.8 whenever such a URL was used.
+
 ### Authentication properties
 
 ```xml
@@ -281,9 +285,8 @@ of `com.amazonaws.auth.AWSCredentialsProvider` may also be used.
 
     If unspecified, then the default list of credential provider classes,
     queried in sequence, is:
-    1. org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider: supports
-        static configuration of AWS access key ID and secret access key.
-        See also fs.s3a.access.key and fs.s3a.secret.key.
+    1. org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider:
+       Uses the values of fs.s3a.access.key and fs.s3a.secret.key.
     2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
         configuration of AWS access key ID and secret access key in
         environment variables named AWS_ACCESS_KEY_ID and
@@ -340,8 +343,6 @@ properties in the configuration file.
 
 The S3A client follows the following authentication chain:
 
-1. If login details were provided in the filesystem URI, a warning is printed
-and then the username and password extracted for the AWS key and secret respectively.
 1. The `fs.s3a.access.key` and `fs.s3a.secret.key` are looked for in the Hadoop
 XML configuration.
 1. The [AWS environment variables](http://docs.aws.amazon.com/cli/latest/userguide/cli-chap-getting-started.html#cli-environment),
@@ -461,12 +462,11 @@ security and therefore is unsuitable for most use cases.
 then the Anonymous Credential provider *must* come last. If not, credential
 providers listed after it will be ignored.
 
-*Simple name/secret credentials with `SimpleAWSCredentialsProvider`*
+### <a name="auth_simple"></a> Simple name/secret credentials with `SimpleAWSCredentialsProvider`*
 
-This is is the standard credential provider, which
-supports the secret key in `fs.s3a.access.key` and token in `fs.s3a.secret.key`
-values. It does not support authentication with logins credentials declared
-in the URLs.
+This is is the standard credential provider, which supports the secret
+key in `fs.s3a.access.key` and token in `fs.s3a.secret.key`
+values.
 
 ```xml
 <property>
@@ -475,9 +475,7 @@ in the URLs.
 </property>
 ```
 
-Apart from its lack of support of user:password details being included in filesystem
-URLs (a dangerous practise that is strongly discouraged), this provider acts
-exactly at the basic authenticator used in the default authentication chain.
+This is the basic authenticator used in the default authentication chain.
 
 This means that the default S3A authentication chain can be defined as
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
index 66ee8a3..da3e0da 100644
--- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
+++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md
@@ -149,19 +149,22 @@ credentials, through a command such as:
 Note the trailing "/" here; without that the shell thinks you are trying to list
 your home directory under the bucket, which will only exist if explicitly created.
 
-Attempting to list a bucket using inline credentials is a
-means of verifying that the key and secret can access a bucket;
-
-    hadoop fs -ls s3a://key:secret@my-bucket/
-
-Do escape any `+` or `/` symbols in the secret, as discussed below, and never
-share the URL, logs generated using it, or use such an inline authentication
-mechanism in production.
-
 Finally, if you set the environment variables, you can take advantage of S3A's
 support of environment-variable authentication by attempting the same ls operation.
 That is: unset the `fs.s3a` secrets and rely on the environment variables.
 
+
+### Authentication failure "The Filesystem URI contains login details."
+
+```
+The Filesystem URI contains login details. This authentication mechanism is no longer supported.
+```
+
+The S3A connector no longer supports the dangerously insecure mechanism of
+passing login details within the S3A URLs.
+
+Fix: use a more secure mechanism to pass down the secrets.
+
 ### Authentication failure due to clock skew
 
 The timestamp is used in signing to S3, so as to
@@ -173,29 +176,6 @@ read requests are allowed, but operations which write to the bucket are denied.
 
 Check the system clock.
 
-### Authentication failure when using URLs with embedded secrets
-
-If using the (strongly discouraged) mechanism of including the
-AWS Key and secret in a URL, then both "+" and "/" symbols need
-to encoded in the URL. As many AWS secrets include these characters,
-encoding problems are not uncommon.
-
-| symbol | encoded  value|
-|-----------|-------------|
-| `+` | `%2B` |
-| `/` | `%2F` |
-
-
-As an example, a URL for `bucket` with AWS ID `user1` and secret `a+b/c` would
-be represented as
-
-```
-s3a://user1:a%2Bb%2Fc@bucket/
-```
-
-This technique is only needed when placing secrets in the URL. Again,
-this is something users are strongly advised against using.
-
 ### <a name="bad_request"></a> "Bad Request" exception when working with AWS S3 Frankfurt, Seoul, or other "V4" endpoint
 
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
index 6601233..267646c 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AAWSCredentialsProvider.java
@@ -33,9 +33,7 @@ import org.junit.rules.Timeout;
 
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
-import com.amazonaws.auth.AWSCredentialsProviderChain;
 import com.amazonaws.auth.BasicAWSCredentials;
-import com.amazonaws.auth.InstanceProfileCredentialsProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -147,23 +145,6 @@ public class ITestS3AAWSCredentialsProvider {
     }
   }
 
-  static class GoodCredentialsProvider extends AWSCredentialsProviderChain {
-
-    @SuppressWarnings("unused")
-    public GoodCredentialsProvider(Configuration conf) {
-      super(new BasicAWSCredentialsProvider(conf.get(ACCESS_KEY),
-          conf.get(SECRET_KEY)),
-          InstanceProfileCredentialsProvider.getInstance());
-    }
-  }
-
-  @Test
-  public void testGoodProvider() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(AWS_CREDENTIALS_PROVIDER, GoodCredentialsProvider.class.getName());
-    S3ATestUtils.createTestFileSystem(conf);
-  }
-
   @Test
   public void testAnonymousProvider() throws Exception {
     Configuration conf = new Configuration();
@@ -178,4 +159,5 @@ public class ITestS3AAWSCredentialsProvider {
     assertNotNull(stat);
     assertEquals(testFile, stat.getPath());
   }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
index 3214c76..423003b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
@@ -57,7 +57,7 @@ import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
 import static org.junit.Assert.*;
 
 /**
- * S3A tests for configuration.
+ * S3A tests for configuration, especially credentials.
  */
 public class ITestS3AConfiguration {
   private static final String EXAMPLE_ID = "AKASOMEACCESSKEY";
@@ -246,47 +246,6 @@ public class ITestS3AConfiguration {
   }
 
   @Test
-  public void testCredsFromUserInfo() throws Exception {
-    // set up conf to have a cred provider
-    final Configuration conf = new Configuration();
-    final File file = tempDir.newFile("test.jks");
-    final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
-        file.toURI());
-    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
-        jks.toString());
-
-    provisionAccessKeys(conf);
-
-    conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
-    URI uriWithUserInfo = new URI("s3a://123:456@foobar");
-    S3xLoginHelper.Login creds =
-        S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
-    assertEquals("AccessKey incorrect.", "123", creds.getUser());
-    assertEquals("SecretKey incorrect.", "456", creds.getPassword());
-  }
-
-  @Test
-  public void testIDFromUserInfoSecretFromCredentialProvider()
-      throws Exception {
-    // set up conf to have a cred provider
-    final Configuration conf = new Configuration();
-    final File file = tempDir.newFile("test.jks");
-    final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider(
-        file.toURI());
-    conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
-        jks.toString());
-
-    provisionAccessKeys(conf);
-
-    conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
-    URI uriWithUserInfo = new URI("s3a://123@foobar");
-    S3xLoginHelper.Login creds =
-        S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
-    assertEquals("AccessKey incorrect.", "123", creds.getUser());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
-  }
-
-  @Test
   public void testSecretFromCredentialProviderIDFromConfig() throws Exception {
     // set up conf to have a cred provider
     final Configuration conf = new Configuration();
@@ -358,11 +317,11 @@ public class ITestS3AConfiguration {
     provisionAccessKeys(c);
 
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
-    URI uriWithUserInfo = new URI("s3a://123:456@foobar");
+    URI uri2 = new URI("s3a://foobar");
     S3xLoginHelper.Login creds =
-        S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
-    assertEquals("AccessKey incorrect.", "123", creds.getUser());
-    assertEquals("SecretKey incorrect.", "456", creds.getPassword());
+        S3AUtils.getAWSAccessKeys(uri2, conf);
+    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
index b28925c..66f7e0a 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a;
 
 import java.io.IOException;
+import java.io.InterruptedIOException;
 import java.net.URI;
 import java.nio.file.AccessDeniedException;
 import java.util.Arrays;
@@ -37,7 +38,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider;
 import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
 import org.apache.hadoop.io.retry.RetryPolicy;
-import org.apache.hadoop.test.GenericTestUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3ATestConstants.*;
@@ -119,7 +119,7 @@ public class TestS3AAWSCredentialsProvider {
         uri2, conf);
     List<Class<? extends AWSCredentialsProvider>> expectedClasses =
         Arrays.asList(
-            BasicAWSCredentialsProvider.class,
+            SimpleAWSCredentialsProvider.class,
             EnvironmentVariableCredentialsProvider.class,
             InstanceProfileCredentialsProvider.class);
     assertCredentialProviders(expectedClasses, list1);
@@ -213,25 +213,13 @@ public class TestS3AAWSCredentialsProvider {
     }
   }
 
-  /**
-   * Declare what exception to raise, and the text which must be found
-   * in it.
-   * @param exceptionClass class of exception
-   * @param text text in exception
-   */
-  private void expectException(Class<? extends Throwable> exceptionClass,
-      String text) {
-    exception.expect(exceptionClass);
-    exception.expectMessage(text);
-  }
-
-  private void expectProviderInstantiationFailure(String option,
+  private IOException expectProviderInstantiationFailure(String option,
       String expectedErrorText) throws Exception {
     Configuration conf = new Configuration();
     conf.set(AWS_CREDENTIALS_PROVIDER, option);
     Path testFile = new Path(
         conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE));
-    intercept(IOException.class, expectedErrorText,
+    return intercept(IOException.class, expectedErrorText,
         () -> S3AUtils.createAWSCredentialProviderSet(testFile.toUri(), conf));
   }
 
@@ -355,5 +343,35 @@ public class TestS3AAWSCredentialsProvider {
         () -> providers.getCredentials());
   }
 
+  /**
+   * Verify that IOEs are passed up without being wrapped.
+   */
+  @Test
+  public void testIOEInConstructorPropagation() throws Throwable {
+    IOException expected = expectProviderInstantiationFailure(
+        IOERaisingProvider.class.getName(),
+        "expected");
+    if (!(expected instanceof InterruptedIOException)) {
+      throw expected;
+    }
+  }
+
+  private static class IOERaisingProvider implements AWSCredentialsProvider {
+
+    public IOERaisingProvider(URI uri, Configuration conf)
+        throws IOException {
+      throw new InterruptedIOException("expected");
+    }
+
+    @Override
+    public AWSCredentials getCredentials() {
+      return null;
+    }
+
+    @Override
+    public void refresh() {
+
+    }
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSSEConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSSEConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSSEConfiguration.java
index 20baacc..050f0a7 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSSEConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestSSEConfiguration.java
@@ -77,17 +77,6 @@ public class TestSSEConfiguration extends Assert {
   }
 
   @Test
-  public void testKMSGoodOldOptionName() throws Throwable {
-    Configuration conf = emptyConf();
-    conf.set(SERVER_SIDE_ENCRYPTION_ALGORITHM, SSE_KMS.getMethod());
-    conf.set(OLD_S3A_SERVER_SIDE_ENCRYPTION_KEY, "kmskeyID");
-    // verify key round trip
-    assertEquals("kmskeyID", getServerSideEncryptionKey(BUCKET, conf));
-    // and that KMS lookup finds it
-    assertEquals(SSE_KMS, getEncryptionAlgorithm(BUCKET, conf));
-  }
-
-  @Test
   public void testAESKeySet() throws Throwable {
     assertGetAlgorithmFails(SSE_S3_WITH_KEY_ERROR,
         SSE_S3.getMethod(), "setkey");
@@ -126,24 +115,6 @@ public class TestSSEConfiguration extends Assert {
   }
 
   /**
-   * Very that the old key is picked up via the properties.
-   * @throws Exception failure
-   */
-  @Test
-  public void testOldKeyFromCredentialProvider() throws Exception {
-    // set up conf to have a cred provider
-    final Configuration conf = confWithProvider();
-    String key = "provisioned";
-    setProviderOption(conf, OLD_S3A_SERVER_SIDE_ENCRYPTION_KEY, key);
-    // let's set the password in config and ensure that it uses the credential
-    // provider provisioned value instead.
-    //conf.set(OLD_S3A_SERVER_SIDE_ENCRYPTION_KEY, "oldKeyInConf");
-    String sseKey = getServerSideEncryptionKey(BUCKET, conf);
-    assertNotNull("Proxy password should not retrun null.", sseKey);
-    assertEquals("Proxy password override did NOT work.", key, sseKey);
-  }
-
-  /**
    * Add a temp file provider to the config.
    * @param conf config
    * @throws Exception failure
@@ -293,7 +264,7 @@ public class TestSSEConfiguration extends Assert {
       String bucket,
       String expected, String overrideVal) throws IOException {
     assertEquals(expected,
-        S3AUtils.lookupPassword(bucket, conf, SECRET_KEY, overrideVal));
+        S3AUtils.lookupPassword(bucket, conf, SECRET_KEY, overrideVal, null));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/87f63b64/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java
index 3761cb7..10409df 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java
@@ -18,12 +18,14 @@
 
 package org.apache.hadoop.fs.s3native;
 
-import org.apache.hadoop.fs.Path;
+import java.net.URI;
+import java.net.URISyntaxException;
+
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.net.URI;
-import java.net.URISyntaxException;
+
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
  * Test how URIs and login details are extracted from URIs.
@@ -35,14 +37,11 @@ public class TestS3xLoginHelper extends Assert {
   public static final String P = "%2b";
   public static final String P_RAW = "+";
   public static final String USER = "user";
-  public static final String PASS = "pass";
   public static final String PASLASHSLASH = "pa" + S + S;
   public static final String PAPLUS = "pa" + P;
   public static final String PAPLUS_RAW = "pa" + P_RAW;
 
   public static final URI WITH_USER_AND_PASS = uri("s3a://user:pass@bucket");
-  public static final Path PATH_WITH_LOGIN =
-      new Path(uri("s3a://user:pass@bucket/dest"));
 
   public static final URI WITH_SLASH_IN_PASS = uri(
       "s3a://user:" + PASLASHSLASH + "@bucket");
@@ -73,7 +72,7 @@ public class TestS3xLoginHelper extends Assert {
 
   /**
    * Assert that a built up FS URI matches the endpoint.
-   * @param uri URI to build the FS UIR from
+   * @param uri URI to build the FS URI from
    */
   private void assertMatchesEndpoint(URI uri) {
     assertEquals("Source " + uri,
@@ -81,6 +80,17 @@ public class TestS3xLoginHelper extends Assert {
   }
 
   /**
+   * Assert that the supplied FS URI is invalid as it contains
+   * username:password secrets.
+   * @param uri URI to build the FS URI from
+   */
+  private void assertInvalid(URI uri) throws Exception {
+    intercept(IllegalArgumentException.class,
+    S3xLoginHelper.LOGIN_WARNING,
+    () -> S3xLoginHelper.buildFSURI(uri));
+  }
+
+  /**
    * Assert that the login/pass details from a URI match that expected.
    * @param user username
    * @param pass password
@@ -89,10 +99,8 @@ public class TestS3xLoginHelper extends Assert {
    */
   private S3xLoginHelper.Login assertMatchesLogin(String user,
       String pass, URI uri) {
-    S3xLoginHelper.Login expected = new S3xLoginHelper.Login(user,
-        pass);
-    S3xLoginHelper.Login actual = S3xLoginHelper.extractLoginDetails(
-        uri);
+    S3xLoginHelper.Login expected = new S3xLoginHelper.Login(user, pass);
+    S3xLoginHelper.Login actual = S3xLoginHelper.extractLoginDetails(uri);
     if (!expected.equals(actual)) {
       Assert.fail("Source " + uri
           + " login expected=:" + toString(expected)
@@ -113,28 +121,6 @@ public class TestS3xLoginHelper extends Assert {
   }
 
   @Test
-  public void testLoginWithUserAndPass() throws Throwable {
-    S3xLoginHelper.Login login = assertMatchesLogin(USER, PASS,
-        WITH_USER_AND_PASS);
-    assertTrue("Login of " + login, login.hasLogin());
-  }
-
-  @Test
-  public void testLoginWithSlashInPass() throws Throwable {
-    assertMatchesLogin(USER, "pa//", WITH_SLASH_IN_PASS);
-  }
-
-  @Test
-  public void testLoginWithPlusInPass() throws Throwable {
-    assertMatchesLogin(USER, "pa+", WITH_PLUS_IN_PASS);
-  }
-
-  @Test
-  public void testLoginWithPlusRawInPass() throws Throwable {
-    assertMatchesLogin(USER, "pa+", WITH_PLUS_RAW_IN_PASS);
-  }
-
-  @Test
   public void testLoginWithUser() throws Throwable {
     assertMatchesLogin(USER, "", USER_NO_PASS);
   }
@@ -161,32 +147,32 @@ public class TestS3xLoginHelper extends Assert {
 
   @Test
   public void testFsUriWithUserAndPass() throws Throwable {
-    assertMatchesEndpoint(WITH_USER_AND_PASS);
+    assertInvalid(WITH_USER_AND_PASS);
   }
 
   @Test
   public void testFsUriWithSlashInPass() throws Throwable {
-    assertMatchesEndpoint(WITH_SLASH_IN_PASS);
+    assertInvalid(WITH_SLASH_IN_PASS);
   }
 
   @Test
   public void testFsUriWithPlusInPass() throws Throwable {
-    assertMatchesEndpoint(WITH_PLUS_IN_PASS);
+    assertInvalid(WITH_PLUS_IN_PASS);
   }
 
   @Test
   public void testFsUriWithPlusRawInPass() throws Throwable {
-    assertMatchesEndpoint(WITH_PLUS_RAW_IN_PASS);
+    assertInvalid(WITH_PLUS_RAW_IN_PASS);
   }
 
   @Test
   public void testFsUriWithUser() throws Throwable {
-    assertMatchesEndpoint(USER_NO_PASS);
+    assertInvalid(USER_NO_PASS);
   }
 
   @Test
   public void testFsUriWithUserAndColon() throws Throwable {
-    assertMatchesEndpoint(WITH_USER_AND_COLON);
+    assertInvalid(WITH_USER_AND_COLON);
   }
 
   @Test
@@ -204,12 +190,6 @@ public class TestS3xLoginHelper extends Assert {
     assertMatchesEndpoint(NO_USER_NO_PASS_TWO_COLON);
   }
 
-  @Test
-  public void testPathURIFixup() throws Throwable {
-
-  }
-
-
   /**
    * Stringifier. Kept in the code to avoid accidental logging in production
    * code.


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org