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 ae...@apache.org on 2016/06/19 00:33:03 UTC

[24/39] hadoop git commit: HADOOP-3733. "s3x:" URLs break when Secret Key contains a slash, even if encoded. Contributed by Steve Loughran.

HADOOP-3733. "s3x:" URLs break when Secret Key contains a slash, even if encoded. 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/4aefe119
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4aefe119
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4aefe119

Branch: refs/heads/HDFS-1312
Commit: 4aefe119a0203c03cdc893dcb3330fd37f26f0ee
Parents: e983eaf
Author: Ravi Prakash <ra...@altiscale.com>
Authored: Thu Jun 16 11:13:35 2016 -0700
Committer: Ravi Prakash <ra...@altiscale.com>
Committed: Thu Jun 16 11:13:35 2016 -0700

----------------------------------------------------------------------
 .../hadoop-aws/dev-support/findbugs-exclude.xml |   7 +
 .../org/apache/hadoop/fs/s3/S3Credentials.java  |  31 +-
 .../org/apache/hadoop/fs/s3/S3FileSystem.java   |  20 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  79 ++----
 .../java/org/apache/hadoop/fs/s3a/S3AUtils.java |  46 +++
 .../hadoop/fs/s3native/NativeS3FileSystem.java  |  19 +-
 .../hadoop/fs/s3native/S3xLoginHelper.java      | 283 +++++++++++++++++++
 .../src/site/markdown/tools/hadoop-aws/index.md |   5 +-
 .../apache/hadoop/fs/s3/TestS3FileSystem.java   |  24 +-
 .../hadoop/fs/s3a/TestS3AConfiguration.java     |  55 ++--
 .../hadoop/fs/s3a/TestS3ACredentialsInURL.java  | 153 ++++++++++
 .../hadoop/fs/s3native/TestS3xLoginHelper.java  | 197 +++++++++++++
 12 files changed, 801 insertions(+), 118 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
index 2b4160a..ffb0a79 100644
--- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
+++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml
@@ -15,6 +15,13 @@
    limitations under the License.
 -->
 <FindBugsFilter>
+
+  <!-- same code as in FileSystem is triggering the same warning. -->
+  <Match>
+    <Class name="org.apache.hadoop.fs.s3native.S3xLoginHelper" />
+    <Method name="checkPath" />
+    <Bug pattern="ES_COMPARING_STRINGS_WITH_EQ" />
+  </Match>
   <!-- S3n warnings about malicious code aren't that relevant given its limited future. -->
   <Match>
     <Class name="org.apache.hadoop.fs.s3.INode" />

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java
index 5ab352a..3951a08 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3Credentials.java
@@ -24,6 +24,7 @@ import java.net.URI;
 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;
 
 /**
  * <p>
@@ -49,18 +50,12 @@ public class S3Credentials {
     if (uri.getHost() == null) {
       throw new IllegalArgumentException("Invalid hostname in URI " + uri);
     }
-    
-    String userInfo = uri.getUserInfo();
-    if (userInfo != null) {
-      int index = userInfo.indexOf(':');
-      if (index != -1) {
-        accessKey = userInfo.substring(0, index);
-        secretAccessKey = userInfo.substring(index + 1);
-      } else {
-        accessKey = userInfo;
-      }
+    S3xLoginHelper.Login login =
+        S3xLoginHelper.extractLoginDetailsWithWarnings(uri);
+    if (login.hasLogin()) {
+      accessKey = login.getUser();
+      secretAccessKey = login.getPassword();
     }
-    
     String scheme = uri.getScheme();
     String accessKeyProperty = String.format("fs.%s.awsAccessKeyId", scheme);
     String secretAccessKeyProperty =
@@ -77,24 +72,20 @@ public class S3Credentials {
     if (accessKey == null && secretAccessKey == null) {
       throw new IllegalArgumentException("AWS " +
                                          "Access Key ID and Secret Access " +
-                                         "Key must be specified as the " +
-                                         "username or password " +
-                                         "(respectively) of a " + scheme +
-                                         " URL, or by setting the " +
-                                         accessKeyProperty + " or " +
+                                         "Key must be specified " +
+                                         "by setting the " +
+                                         accessKeyProperty + " and " +
                                          secretAccessKeyProperty +
                                          " properties (respectively).");
     } else if (accessKey == null) {
       throw new IllegalArgumentException("AWS " +
                                          "Access Key ID must be specified " +
-                                         "as the username of a " + scheme +
-                                         " URL, or by setting the " +
+                                         "by setting the " +
                                          accessKeyProperty + " property.");
     } else if (secretAccessKey == null) {
       throw new IllegalArgumentException("AWS " +
                                          "Secret Access Key must be " +
-                                         "specified as the password of a " +
-                                         scheme + " URL, or by setting the " +
+                                         "specified by setting the " +
                                          secretAccessKeyProperty +
                                          " property.");       
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
index e5147a3..6a49d1a 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.s3native.NativeS3FileSystem;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.io.retry.RetryPolicies;
 import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
@@ -91,7 +92,7 @@ public class S3FileSystem extends FileSystem {
     }
     store.initialize(uri, conf);
     setConf(conf);
-    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+    this.uri = S3xLoginHelper.buildFSURI(uri);
     this.workingDir =
       new Path("/user", System.getProperty("user.name")).makeQualified(this);
   }
@@ -135,6 +136,23 @@ public class S3FileSystem extends FileSystem {
   }
 
   /**
+   * Check that a Path belongs to this FileSystem.
+   * Unlike the superclass, this version does not look at authority,
+   * only hostnames.
+   * @param path to check
+   * @throws IllegalArgumentException if there is an FS mismatch
+   */
+  @Override
+  protected void checkPath(Path path) {
+    S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
+  }
+
+  @Override
+  protected URI canonicalizeUri(URI rawUri) {
+    return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
+  }
+
+  /**
    * @param permission Currently ignored.
    */
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/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 9af0a99..63bfb4f 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
@@ -78,10 +78,11 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.ProviderUtils;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.VersionInfo;
 
+import static org.apache.commons.lang.StringUtils.*;
 import static org.apache.hadoop.fs.s3a.Constants.*;
 import static org.apache.hadoop.fs.s3a.S3AUtils.*;
 import static org.apache.hadoop.fs.s3a.Statistic.*;
@@ -140,7 +141,7 @@ public class S3AFileSystem extends FileSystem {
     try {
       instrumentation = new S3AInstrumentation(name);
 
-      uri = URI.create(name.getScheme() + "://" + name.getAuthority());
+      uri = S3xLoginHelper.buildFSURI(name);
       workingDir = new Path("/user", System.getProperty("user.name"))
           .makeQualified(this.uri, this.getWorkingDirectory());
 
@@ -399,53 +400,6 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
-   * 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.
-   * @param conf the Configuration object to interogate for keys.
-   * @return AWSAccessKeys
-   */
-  AWSAccessKeys getAWSAccessKeys(URI name, Configuration conf)
-      throws IOException {
-    String accessKey = null;
-    String secretKey = null;
-    String userInfo = name.getUserInfo();
-    if (userInfo != null) {
-      int index = userInfo.indexOf(':');
-      if (index != -1) {
-        accessKey = userInfo.substring(0, index);
-        secretKey = userInfo.substring(index + 1);
-      } else {
-        accessKey = userInfo;
-      }
-    }
-    Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
-          conf, S3AFileSystem.class);
-    if (accessKey == null) {
-      try {
-        final char[] key = c.getPassword(ACCESS_KEY);
-        if (key != null) {
-          accessKey = (new String(key)).trim();
-        }
-      } catch(IOException ioe) {
-        throw new IOException("Cannot find AWS access key.", ioe);
-      }
-    }
-    if (secretKey == null) {
-      try {
-        final char[] pass = c.getPassword(SECRET_KEY);
-        if (pass != null) {
-          secretKey = (new String(pass)).trim();
-        }
-      } catch(IOException ioe) {
-        throw new IOException("Cannot find AWS secret key.", ioe);
-      }
-    }
-    return new AWSAccessKeys(accessKey, secretKey);
-  }
-
-  /**
    * Create the standard credential provider, or load in one explicitly
    * identified in the configuration.
    * @param binding the S3 binding/bucket.
@@ -460,10 +414,10 @@ public class S3AFileSystem extends FileSystem {
 
     String className = conf.getTrimmed(AWS_CREDENTIALS_PROVIDER);
     if (StringUtils.isEmpty(className)) {
-      AWSAccessKeys creds = getAWSAccessKeys(binding, conf);
+      S3xLoginHelper.Login creds = getAWSAccessKeys(binding, conf);
       credentials = new AWSCredentialsProviderChain(
           new BasicAWSCredentialsProvider(
-              creds.getAccessKey(), creds.getAccessSecret()),
+              creds.getUser(), creds.getPassword()),
           new InstanceProfileCredentialsProvider(),
           new EnvironmentVariableCredentialsProvider());
 
@@ -551,10 +505,27 @@ public class S3AFileSystem extends FileSystem {
   }
 
   /**
-   * Opens an FSDataInputStream at the indicated Path.
-   * @param f the file name to open
-   * @param bufferSize the size of the buffer to be used.
+   * Check that a Path belongs to this FileSystem.
+   * Unlike the superclass, this version does not look at authority,
+   * only hostnames.
+   * @param path to check
+   * @throws IllegalArgumentException if there is an FS mismatch
    */
+  @Override
+  public void checkPath(Path path) {
+    S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
+  }
+
+  @Override
+  protected URI canonicalizeUri(URI rawUri) {
+    return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
+  }
+
+  /**
+     * Opens an FSDataInputStream at the indicated Path.
+     * @param f the file name to open
+     * @param bufferSize the size of the buffer to be used.
+     */
   public FSDataInputStream open(Path f, int bufferSize)
       throws IOException {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/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 062fca4..8033ac3 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
@@ -22,18 +22,26 @@ import com.amazonaws.AmazonClientException;
 import com.amazonaws.AmazonServiceException;
 import com.amazonaws.services.s3.model.AmazonS3Exception;
 import com.amazonaws.services.s3.model.S3ObjectSummary;
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
+import org.apache.hadoop.security.ProviderUtils;
 
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.net.URI;
 import java.nio.file.AccessDeniedException;
 import java.util.Date;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
 
+import static org.apache.hadoop.fs.s3a.Constants.ACCESS_KEY;
+import static org.apache.hadoop.fs.s3a.Constants.SECRET_KEY;
+
 /**
  * Utility methods for S3A code.
  */
@@ -234,4 +242,42 @@ public final class S3AUtils {
 
     return date.getTime();
   }
+
+  /**
+   * 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.
+   * @param conf the Configuration object to interrogate for keys.
+   * @return AWSAccessKeys
+   * @throws IOException problems retrieving passwords from KMS.
+   */
+  public static S3xLoginHelper.Login getAWSAccessKeys(URI name,
+      Configuration conf) throws IOException {
+    S3xLoginHelper.Login login =
+        S3xLoginHelper.extractLoginDetailsWithWarnings(name);
+    Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders(
+        conf, S3AFileSystem.class);
+    String accessKey = getPassword(c, ACCESS_KEY, login.getUser());
+    String secretKey = getPassword(c, SECRET_KEY, login.getPassword());
+    return new S3xLoginHelper.Login(accessKey, secretKey);
+  }
+
+  private static String getPassword(Configuration conf, String key, String val)
+      throws IOException {
+    if (StringUtils.isEmpty(val)) {
+      try {
+        final char[] pass = conf.getPassword(key);
+        if (pass != null) {
+          return (new String(pass)).trim();
+        } else {
+          return "";
+        }
+      } catch (IOException ioe) {
+        throw new IOException("Cannot find password option " + key, ioe);
+      }
+    } else {
+      return val;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
index 0ad8e5f..d646726 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java
@@ -333,7 +333,7 @@ public class NativeS3FileSystem extends FileSystem {
     }
     store.initialize(uri, conf);
     setConf(conf);
-    this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority());
+    this.uri = S3xLoginHelper.buildFSURI(uri);
     this.workingDir =
       new Path("/user", System.getProperty("user.name")).makeQualified(this.uri, this.getWorkingDirectory());
   }
@@ -388,6 +388,23 @@ public class NativeS3FileSystem extends FileSystem {
     return new Path(workingDir, path);
   }
 
+  /**
+   * Check that a Path belongs to this FileSystem.
+   * Unlike the superclass, this version does not look at authority,
+   * only hostnames.
+   * @param path to check
+   * @throws IllegalArgumentException if there is an FS mismatch
+   */
+  @Override
+  protected void checkPath(Path path) {
+    S3xLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort());
+  }
+
+  @Override
+  protected URI canonicalizeUri(URI rawUri) {
+    return S3xLoginHelper.canonicalizeUri(rawUri, getDefaultPort());
+  }
+
   /** This optional operation is not yet supported. */
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/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
new file mode 100644
index 0000000..bc8c2e6
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/S3xLoginHelper.java
@@ -0,0 +1,283 @@
+/*
+ * 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.s3native;
+
+import org.apache.commons.lang.StringUtils;
+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.lang.StringUtils.equalsIgnoreCase;
+
+/**
+ * Class to aid logging in to S3 endpoints.
+ * It is in S3N so that it can be used across all S3 filesystems.
+ */
+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.";
+
+  /**
+   * Build the filesystem URI. This can include stripping down of part
+   * of the 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) {
+    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.");
+    return URI.create(uri.getScheme() + "://" + uri.getHost());
+  }
+
+  /**
+   * Create a stripped down string value for error messages.
+   * @param pathUri URI
+   * @return a shortened schema://host/path value
+   */
+  public static String toString(URI pathUri) {
+    return pathUri != null
+        ? String.format("%s://%s/%s",
+        pathUri.getScheme(), pathUri.getHost(), pathUri.getPath())
+        : "(null URI)";
+  }
+
+  /**
+   * Extract the login details from a URI, logging a warning if
+   * the URI contains these.
+   * @param name URI of the filesystem
+   * @return a login tuple, possibly empty.
+   */
+  public static Login extractLoginDetailsWithWarnings(URI name) {
+    Login login = extractLoginDetails(name);
+    if (login.hasLogin()) {
+      LOG.warn(LOGIN_WARNING);
+    }
+    return login;
+  }
+
+  /**
+   * Extract the login details from a URI.
+   * @param name URI of the filesystem
+   * @return a login tuple, possibly empty.
+   */
+  public static Login extractLoginDetails(URI name) {
+    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 password = URLDecoder.decode(login.substring(loginSplit + 1),
+            "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);
+    }
+  }
+
+  /**
+   * Canonicalize the given URI.
+   *
+   * This strips out login information.
+   *
+   * @return a new, canonicalized URI.
+   */
+  public static URI canonicalizeUri(URI uri, int defaultPort) {
+    if (uri.getPort() == -1 && defaultPort > 0) {
+      // reconstruct the uri with the default port set
+      try {
+        uri = new URI(uri.getScheme(),
+            null,
+            uri.getHost(),
+            defaultPort,
+            uri.getPath(),
+            uri.getQuery(),
+            uri.getFragment());
+      } catch (URISyntaxException e) {
+        // Should never happen!
+        throw new AssertionError("Valid URI became unparseable: " +
+            uri);
+      }
+    }
+
+    return uri;
+  }
+
+  /**
+   * Check the path, ignoring authentication details.
+   * See {@link FileSystem#checkPath(Path)} for the operation of this.
+   *
+   * Essentially
+   * <ol>
+   *   <li>The URI is canonicalized.</li>
+   *   <li>If the schemas match, the hosts are compared.</li>
+   *   <li>If there is a mismatch between null/non-null host, the default FS
+   *   values are used to patch in the host.</li>
+   * </ol>
+   * That all originates in the core FS; the sole change here being to use
+   * {@link URI#getHost()} over {@link URI#getAuthority()}. Some of that
+   * code looks a relic of the code anti-pattern of using "hdfs:file.txt"
+   * to define the path without declaring the hostname. It's retained
+   * for compatibility.
+   * @param conf FS configuration
+   * @param fsUri the FS URI
+   * @param path path to check
+   * @param defaultPort default port of FS
+   */
+  public static void checkPath(Configuration conf,
+      URI fsUri,
+      Path path,
+      int defaultPort) {
+    URI pathUri = path.toUri();
+    String thatScheme = pathUri.getScheme();
+    if (thatScheme == null) {
+      // fs is relative
+      return;
+    }
+    URI thisUri = canonicalizeUri(fsUri, defaultPort);
+    String thisScheme = thisUri.getScheme();
+    //hostname and scheme are not case sensitive in these checks
+    if (equalsIgnoreCase(thisScheme, thatScheme)) {// schemes match
+      String thisHost = thisUri.getHost();
+      String thatHost = pathUri.getHost();
+      if (thatHost == null &&                // path's host is null
+          thisHost != null) {                // fs has a host
+        URI defaultUri = FileSystem.getDefaultUri(conf);
+        if (equalsIgnoreCase(thisScheme, defaultUri.getScheme())) {
+          pathUri = defaultUri; // schemes match, so use this uri instead
+        } else {
+          pathUri = null; // can't determine auth of the path
+        }
+      }
+      if (pathUri != null) {
+        // canonicalize uri before comparing with this fs
+        pathUri = canonicalizeUri(pathUri, defaultPort);
+        thatHost = pathUri.getHost();
+        if (thisHost == thatHost ||       // hosts match
+            (thisHost != null &&
+                 equalsIgnoreCase(thisHost, thatHost))) {
+          return;
+        }
+      }
+    }
+    // make sure the exception strips out any auth details
+    throw new IllegalArgumentException(
+        "Wrong FS " + S3xLoginHelper.toString(pathUri)
+            + " -expected " + fsUri);
+  }
+
+  /**
+   * Simple tuple of login details.
+   */
+  public static class Login {
+    private final String user;
+    private final String password;
+
+    public static final Login EMPTY = new Login();
+
+    /**
+     * Create an instance with no login details.
+     * Calls to {@link #hasLogin()} return false.
+     */
+    public Login() {
+      this("", "");
+    }
+
+    public Login(String user, String password) {
+      this.user = user;
+      this.password = password;
+    }
+
+    /**
+     * Predicate to verify login details are defined.
+     * @return true if the username is defined (not null, not empty).
+     */
+    public boolean hasLogin() {
+      return StringUtils.isNotEmpty(user);
+    }
+
+    /**
+     * Equality test matches user and password.
+     * @param o other object
+     * @return true if the objects are considered equivalent.
+     */
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      Login that = (Login) o;
+      return Objects.equals(user, that.user) &&
+          Objects.equals(password, that.password);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(user, password);
+    }
+
+    public String getUser() {
+      return user;
+    }
+
+    public String getPassword() {
+      return password;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/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 606275c..79c9349 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
@@ -69,8 +69,11 @@ access to the data. Anyone with the credentials can not only read your datasets
 \u2014they can delete them.
 
 Do not inadvertently share these credentials through means such as
-1. Checking in Hadoop configuration files containing the credentials.
+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/`. They will end up in logs and error messages.
+1. Including the secrets in bug reports.
 
 If you do any of these: change your credentials immediately!
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java
index f21989c..4947f08 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3/TestS3FileSystem.java
@@ -27,24 +27,26 @@ import org.apache.hadoop.conf.Configuration;
 
 public class TestS3FileSystem extends TestCase {
 
+  public static final URI EXPECTED = URI.create("s3://c");
+
   public void testInitialization() throws IOException {
-    initializationTest("s3://a:b@c", "s3://a:b@c");
-    initializationTest("s3://a:b@c/", "s3://a:b@c");
-    initializationTest("s3://a:b@c/path", "s3://a:b@c");
-    initializationTest("s3://a@c", "s3://a@c");
-    initializationTest("s3://a@c/", "s3://a@c");
-    initializationTest("s3://a@c/path", "s3://a@c");
-    initializationTest("s3://c", "s3://c");
-    initializationTest("s3://c/", "s3://c");
-    initializationTest("s3://c/path", "s3://c");
+    initializationTest("s3://a:b@c");
+    initializationTest("s3://a:b@c/");
+    initializationTest("s3://a:b@c/path");
+    initializationTest("s3://a@c");
+    initializationTest("s3://a@c/");
+    initializationTest("s3://a@c/path");
+    initializationTest("s3://c");
+    initializationTest("s3://c/");
+    initializationTest("s3://c/path");
   }
   
-  private void initializationTest(String initializationUri, String expectedUri)
+  private void initializationTest(String initializationUri)
     throws IOException {
     
     S3FileSystem fs = new S3FileSystem(new InMemoryFileSystemStore());
     fs.initialize(URI.create(initializationUri), new Configuration());
-    assertEquals(URI.create(expectedUri), fs.getUri());
+    assertEquals(EXPECTED, fs.getUri());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
index 513cae2..29bfd59 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
@@ -27,6 +27,7 @@ import org.apache.commons.lang.reflect.FieldUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
+import org.apache.hadoop.fs.s3native.S3xLoginHelper;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
@@ -210,12 +211,11 @@ public class TestS3AConfiguration {
 
     provisionAccessKeys(conf);
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(new URI("s3a://foobar"), conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
+    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
   }
 
   void provisionAccessKeys(final Configuration conf) throws Exception {
@@ -241,13 +241,12 @@ public class TestS3AConfiguration {
 
     provisionAccessKeys(conf);
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
     URI uriWithUserInfo = new URI("s3a://123:456@foobar");
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(uriWithUserInfo, conf);
-    assertEquals("AccessKey incorrect.", "123", creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", "456", creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
+    assertEquals("AccessKey incorrect.", "123", creds.getUser());
+    assertEquals("SecretKey incorrect.", "456", creds.getPassword());
   }
 
   @Test
@@ -263,13 +262,12 @@ public class TestS3AConfiguration {
 
     provisionAccessKeys(conf);
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
     URI uriWithUserInfo = new URI("s3a://123@foobar");
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(uriWithUserInfo, conf);
-    assertEquals("AccessKey incorrect.", "123", creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
+    assertEquals("AccessKey incorrect.", "123", creds.getUser());
+    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
   }
 
   @Test
@@ -289,12 +287,11 @@ public class TestS3AConfiguration {
         EXAMPLE_KEY.toCharArray());
     provider.flush();
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID);
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(new URI("s3a://foobar"), conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
+    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
   }
 
   @Test
@@ -314,12 +311,11 @@ public class TestS3AConfiguration {
         EXAMPLE_ID.toCharArray());
     provider.flush();
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.SECRET_KEY, EXAMPLE_KEY);
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(new URI("s3a://foobar"), conf);
-    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(new URI("s3a://foobar"), conf);
+    assertEquals("AccessKey incorrect.", EXAMPLE_ID, creds.getUser());
+    assertEquals("SecretKey incorrect.", EXAMPLE_KEY, creds.getPassword());
   }
 
   @Test
@@ -345,13 +341,12 @@ public class TestS3AConfiguration {
     // using the original config with the s3a provider in the path.
     provisionAccessKeys(c);
 
-    S3AFileSystem s3afs = new S3AFileSystem();
     conf.set(Constants.ACCESS_KEY, EXAMPLE_ID + "LJM");
     URI uriWithUserInfo = new URI("s3a://123:456@foobar");
-    S3AFileSystem.AWSAccessKeys creds =
-        s3afs.getAWSAccessKeys(uriWithUserInfo, conf);
-    assertEquals("AccessKey incorrect.", "123", creds.getAccessKey());
-    assertEquals("SecretKey incorrect.", "456", creds.getAccessSecret());
+    S3xLoginHelper.Login creds =
+        S3AUtils.getAWSAccessKeys(uriWithUserInfo, conf);
+    assertEquals("AccessKey incorrect.", "123", creds.getUser());
+    assertEquals("SecretKey incorrect.", "456", creds.getPassword());
 
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java
new file mode 100644
index 0000000..f1da72b
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ACredentialsInURL.java
@@ -0,0 +1,153 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URLEncoder;
+import java.nio.file.AccessDeniedException;
+
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.TEST_FS_S3A_NAME;
+
+/**
+ * Tests that credentials can go into the URL. This includes a valid
+ * set, and a check that an invalid set do at least get stripped out
+ * of the final URI
+ */
+public class TestS3ACredentialsInURL extends Assert {
+  private S3AFileSystem fs;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestS3ACredentialsInURL.class);
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+  @After
+  public void teardown() {
+    IOUtils.closeStream(fs);
+  }
+
+  /**
+   * Test instantiation.
+   * @throws Throwable
+   */
+  @Test
+  public void testInstantiateFromURL() throws Throwable {
+
+    Configuration conf = new Configuration();
+    String accessKey = conf.get(Constants.ACCESS_KEY);
+    String secretKey = conf.get(Constants.SECRET_KEY);
+    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
+    Assume.assumeNotNull(fsname, accessKey, secretKey);
+    URI original = new URI(fsname);
+    URI secretsURI = createUriWithEmbeddedSecrets(original,
+        accessKey, secretKey);
+    if (secretKey.contains("/")) {
+      assertTrue("test URI encodes the / symbol", secretsURI.toString().
+          contains("%2F"));
+    }
+    assertFalse("Does not contain secrets", original.equals(secretsURI));
+
+    conf.set(TEST_FS_S3A_NAME, secretsURI.toString());
+    conf.unset(Constants.ACCESS_KEY);
+    conf.unset(Constants.SECRET_KEY);
+    fs = S3ATestUtils.createTestFileSystem(conf);
+    String fsURI = fs.getUri().toString();
+    assertFalse("FS URI contains a @ symbol", fsURI.contains("@"));
+    assertFalse("FS URI contains a % symbol", fsURI.contains("%"));
+    if (!original.toString().startsWith(fsURI)) {
+      fail("Filesystem URI does not match original");
+    }
+    validate("original path", new Path(original));
+    validate("bare path", new Path("/"));
+    validate("secrets path", new Path(secretsURI));
+  }
+
+  private void validate(String text, Path path) throws IOException {
+    try {
+      fs.canonicalizeUri(path.toUri());
+      fs.checkPath(path);
+      assertTrue(text + " Not a directory",
+          fs.getFileStatus(new Path("/")).isDirectory());
+      fs.globStatus(path);
+    } catch (AssertionError e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.debug("{} failure: {}", text, e, e);
+      fail(text + " Test failed");
+    }
+  }
+  
+  /**
+   * Set up some invalid credentials, verify login is rejected.
+   * @throws Throwable
+   */
+  @Test
+  public void testInvalidCredentialsFail() throws Throwable {
+    Configuration conf = new Configuration();
+    String fsname = conf.getTrimmed(TEST_FS_S3A_NAME, "");
+    Assume.assumeNotNull(fsname);
+    URI original = new URI(fsname);
+    URI testURI = createUriWithEmbeddedSecrets(original, "user", "//");
+
+    conf.set(TEST_FS_S3A_NAME, testURI.toString());
+    fs = S3ATestUtils.createTestFileSystem(conf);
+    try {
+      S3AFileStatus status = fs.getFileStatus(new Path("/"));
+      fail("Expected an AccessDeniedException, got " + status);
+    } catch (AccessDeniedException e) {
+      // expected
+    }
+
+  }
+
+  private URI createUriWithEmbeddedSecrets(URI original,
+      String accessKey,
+      String secretKey) throws URISyntaxException,
+      UnsupportedEncodingException {
+    String encodedSecretKey = URLEncoder.encode(secretKey, "UTF-8");
+    String formattedString = String.format("%s://%s:%s@%s/%s/",
+        original.getScheme(),
+        accessKey,
+        encodedSecretKey,
+        original.getHost(),
+        original.getPath());
+    URI testURI;
+    try {
+      testURI = new URI(formattedString);
+    } catch (URISyntaxException e) {
+      // inner cause is stripped to keep any secrets out of stack traces
+      throw new URISyntaxException("", "Could not encode URI");
+    }
+    return testURI;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4aefe119/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
new file mode 100644
index 0000000..bd2ac1e
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3native/TestS3xLoginHelper.java
@@ -0,0 +1,197 @@
+/*
+ * 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.s3native;
+
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * Test how URIs and login details are extracted from URIs.
+ */
+public class TestS3xLoginHelper extends Assert {
+  public static final String BUCKET = "s3a://bucket";
+  private static final URI ENDPOINT = uri(BUCKET);
+  public static final String S = "%2f";
+  public static final String USER = "user";
+  public static final String PASS = "pass";
+  public static final String PASLASHSLASH = "pa" + S + S;
+
+  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");
+  public static final URI USER_NO_PASS = uri("s3a://user@bucket");
+  public static final URI WITH_USER_AND_COLON = uri("s3a://user:@bucket");
+  public static final URI NO_USER = uri("s3a://:pass@bucket");
+  public static final URI NO_USER_NO_PASS = uri("s3a://:@bucket");
+  public static final URI NO_USER_NO_PASS_TWO_COLON = uri("s3a://::@bucket");
+
+  /**
+   * Construct a URI; raises an RTE if it won't parse.
+   * This allows it to be used in static constructors.
+   * @param s URI string
+   * @return the URI
+   * @throws RuntimeException on a URI syntax problem
+   */
+  private static URI uri(String s) {
+    try {
+      return new URI(s);
+    } catch (URISyntaxException e) {
+      throw new RuntimeException(e.toString(), e);
+    }
+  }
+
+  /**
+   * Assert that a built up FS URI matches the endpoint.
+   * @param uri URI to build the FS UIR from
+   */
+  private void assertMatchesEndpoint(URI uri) {
+    assertEquals("Source " + uri,
+        ENDPOINT, S3xLoginHelper.buildFSURI(uri));
+  }
+
+  /**
+   * Assert that the login/pass details from a URI match that expected.
+   * @param user username
+   * @param pass password
+   * @param uri URI to build login details from
+   * @return the login tuple
+   */
+  private S3xLoginHelper.Login assertMatchesLogin(String user,
+      String pass, URI 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)
+          + " actual=" + toString(actual));
+    }
+    return actual;
+  }
+
+  @Test
+  public void testSimpleFSURI() throws Throwable {
+    assertMatchesEndpoint(ENDPOINT);
+  }
+
+  @Test
+  public void testLoginSimple() throws Throwable {
+    S3xLoginHelper.Login login = assertMatchesLogin("", "", ENDPOINT);
+    assertFalse("Login of " + login, login.hasLogin());
+  }
+
+  @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 testLoginWithUser() throws Throwable {
+    assertMatchesLogin(USER, "", USER_NO_PASS);
+  }
+
+  @Test
+  public void testLoginWithUserAndColon() throws Throwable {
+    assertMatchesLogin(USER, "", WITH_USER_AND_COLON);
+  }
+
+  @Test
+  public void testLoginNoUser() throws Throwable {
+    assertMatchesLogin("", "", NO_USER);
+  }
+
+  @Test
+  public void testLoginNoUserNoPass() throws Throwable {
+    assertMatchesLogin("", "", NO_USER_NO_PASS);
+  }
+
+  @Test
+  public void testLoginNoUserNoPassTwoColon() throws Throwable {
+    assertMatchesLogin("", "", NO_USER_NO_PASS_TWO_COLON);
+  }
+
+  @Test
+  public void testFsUriWithUserAndPass() throws Throwable {
+    assertMatchesEndpoint(WITH_USER_AND_PASS);
+  }
+
+  @Test
+  public void testFsUriWithSlashInPass() throws Throwable {
+    assertMatchesEndpoint(WITH_SLASH_IN_PASS);
+  }
+
+  @Test
+  public void testFsUriWithUser() throws Throwable {
+    assertMatchesEndpoint(USER_NO_PASS);
+  }
+
+  @Test
+  public void testFsUriWithUserAndColon() throws Throwable {
+    assertMatchesEndpoint(WITH_USER_AND_COLON);
+  }
+
+  @Test
+  public void testFsiNoUser() throws Throwable {
+    assertMatchesEndpoint(NO_USER);
+  }
+
+  @Test
+  public void testFsUriNoUserNoPass() throws Throwable {
+    assertMatchesEndpoint(NO_USER_NO_PASS);
+  }
+
+  @Test
+  public void testFsUriNoUserNoPassTwoColon() throws Throwable {
+    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.
+   * @return login details for assertions.
+   */
+  public String toString(S3xLoginHelper.Login login) {
+    final StringBuilder sb = new StringBuilder("LoginTuple{");
+    sb.append("<'").append(login.getUser()).append('\'');
+    sb.append(", '").append(login.getPassword()).append('\'');
+    sb.append('>');
+    return sb.toString();
+  }
+}


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