You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by al...@apache.org on 2015/11/17 19:28:50 UTC

[6/7] incubator-brooklyn git commit: Fix JcloudsLocation login

Fix JcloudsLocation login

- registerJcloudsSshMachineLocation: use the userCredentials that
  are passed into the method, rather than looking up what we expect
  from the config.
  (Hopefully fixes testSpecifyingPasswordAndSshKeysPrefersKeys).
- Fix waitForSshable to when password supplied on JcloudsLocation
  config, and using useJcloudsSshInit=false.
- Fix use of DISABLE_ROOT_AND_PASSWORD_SSH: if for root user with
  password-based login, then explicitly set PasswordAuthentication=yes.
  If root user, then explicitly set PermitRootLogin=true
- Fix getPublicHostnameAws so pass in the credentials (some code-paths
  will not have configured the credentials in the generic config).
  Same for getPrivateHostnameAws.
- Tidy up the createTemporarySshMachineLocaton
- Logging of timing: include how long to get semaphore
- Logging of failure: include how long for each stage

Previously hit a problem when the jclouds location was configured with
a password + a blank ssh key, and using useJcloudsSshInit=false.
It failed to ssh to the newly provisioned AWS VM in waitForSshable. It
got the node.getCredentials, which gives us the ssh-key from the AWS
key-pair. But then we instantiate an SshMachineLocation to check if we
can connection (before continuing with the user-setup). That
machineLocation inherits the password (which has not been set on the
machine), so it fails to ssh - the password takes precedence over the
ssh-key.


Project: http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/commit/f34e7e69
Tree: http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/tree/f34e7e69
Diff: http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/diff/f34e7e69

Branch: refs/heads/master
Commit: f34e7e69839d4b1898fb55348d9aaf6743fed6c4
Parents: bf67238
Author: Aled Sage <al...@gmail.com>
Authored: Thu Nov 5 22:28:35 2015 +0000
Committer: Aled Sage <al...@gmail.com>
Committed: Tue Nov 17 17:49:27 2015 +0000

----------------------------------------------------------------------
 .../location/jclouds/JcloudsLocation.java       | 226 +++++++++++--------
 .../jclouds/AbstractJcloudsLiveTest.java        |   1 +
 .../location/jclouds/JcloudsLoginLiveTest.java  |  51 ++++-
 3 files changed, 168 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/f34e7e69/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
index 4e322bd..642aafb 100644
--- a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
@@ -645,6 +645,12 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         String groupId = elvis(setup.get(GROUP_ID), cloudMachineNamer.generateNewGroupId(setup));
         NodeMetadata node = null;
         JcloudsMachineLocation machineLocation = null;
+        Duration semaphoreTimestamp = null;
+        Duration templateTimestamp = null;
+        Duration provisionTimestamp = null;
+        Duration usableTimestamp = null;
+        Duration customizedTimestamp = null;
+        Stopwatch provisioningStopwatch = Stopwatch.createStarted();
         
         try {
             LOG.info("Creating VM "+setup.getDescription()+" in "+this);
@@ -659,9 +665,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
             } else {
                 LOG.debug("Acquired in {} machine-creation permit immediately", this);
             }
-
-            Stopwatch provisioningStopwatch = Stopwatch.createStarted();
-            Duration templateTimestamp, provisionTimestamp, usableTimestamp, customizedTimestamp;
+            semaphoreTimestamp = Duration.of(provisioningStopwatch);
 
             LoginCredentials userCredentials = null;
             Set<? extends NodeMetadata> nodes;
@@ -770,6 +774,11 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                 if (customCredentials.getOptionalPassword().isPresent()) setup.put(PASSWORD, customCredentials.getOptionalPassword().get());
                 if (customCredentials.getOptionalPrivateKey().isPresent()) setup.put(PRIVATE_KEY_DATA, customCredentials.getOptionalPrivateKey().get());
             }
+            if (userCredentials == null || (!userCredentials.getOptionalPassword().isPresent() && !userCredentials.getOptionalPrivateKey().isPresent())) {
+                // We either don't have any userCredentials, or it is missing both a password/key.
+                // TODO See waitForSshable, which now handles if the node.getLoginCredentials has both a password+key
+                userCredentials = extractVmCredentials(setup, node, initialCredentials);
+            }
             if (userCredentials == null) {
                 // TODO See waitForSshable, which now handles if the node.getLoginCredentials has both a password+key
                 userCredentials = extractVmCredentials(setup, node, initialCredentials);
@@ -984,7 +993,9 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                                 + " && key=" + userCredentials.getOptionalPrivateKey().or("<absent>")
                                 : "")
                         + " ready after "+Duration.of(provisioningStopwatch).toStringRounded()
-                        + " ("+template+" template built in "+Duration.of(templateTimestamp).toStringRounded()+";"
+                        + " ("
+                        + "semaphore obtained in "+Duration.of(semaphoreTimestamp).toStringRounded()+";"
+                        + template+" template built in "+Duration.of(templateTimestamp).subtract(semaphoreTimestamp).toStringRounded()+";"
                         + " "+node+" provisioned in "+Duration.of(provisionTimestamp).subtract(templateTimestamp).toStringRounded()+";"
                         + " "+machineLocation+" connection usable in "+Duration.of(usableTimestamp).subtract(provisionTimestamp).toStringRounded()+";"
                         + " and os customized in "+Duration.of(customizedTimestamp).subtract(usableTimestamp).toStringRounded()+" - "+Joiner.on(", ").join(customisationForLogging)+")";
@@ -1014,16 +1025,29 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                     + "For more information on VPC vs classic see http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/using-vpc.html.");
             }
             
-            LOG.error("Failed to start VM for {}{}: {}",
-                    new Object[] {setup.getDescription(), (destroyNode ? " (destroying "+node+")" : ""), e.getMessage()});
+            LOG.error("Failed to start VM for "+setup.getDescription() + (destroyNode ? " (destroying)" : "")
+                    + (node != null ? "; node "+node : "")
+                    + " after "+Duration.of(provisioningStopwatch).toStringRounded()
+                    + (semaphoreTimestamp != null ? " ("
+                            + "semaphore obtained in "+Duration.of(semaphoreTimestamp).toStringRounded()+";"
+                            + (templateTimestamp != null && semaphoreTimestamp != null ? " template built in "+Duration.of(templateTimestamp).subtract(semaphoreTimestamp).toStringRounded()+";" : "")
+                            + (provisionTimestamp != null && templateTimestamp != null ? " node provisioned in "+Duration.of(provisionTimestamp).subtract(templateTimestamp).toStringRounded()+";" : "")
+                            + (usableTimestamp != null && provisioningStopwatch != null ? " connection usable in "+Duration.of(usableTimestamp).subtract(provisionTimestamp).toStringRounded()+";" : "")
+                            + (customizedTimestamp != null && usableTimestamp != null ? " and OS customized in "+Duration.of(customizedTimestamp).subtract(usableTimestamp).toStringRounded() : "")
+                            + ")"
+                            : "")
+                    + ": "+e.getMessage());
             LOG.debug(Throwables.getStackTraceAsString(e));
-            
+
             if (destroyNode) {
+                Stopwatch destroyingStopwatch = Stopwatch.createStarted();
                 if (machineLocation != null) {
                     releaseSafely(machineLocation);
                 } else {
                     releaseNodeSafely(node);
                 }
+                LOG.info("Destroyed " + (machineLocation != null ? "machine " + machineLocation : "node " + node)
+                        + " in " + Duration.of(destroyingStopwatch).toStringRounded());
             }
 
             throw Exceptions.propagate(e);
@@ -1552,19 +1576,27 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         }
     }
 
+    /**
+     * Creates a temporary ssh machine location (i.e. will not be persisted), which uses the given credentials.
+     * It ignores any credentials (e.g. password, key-phrase, etc) that are supplied in the config.
+     */
     protected SshMachineLocation createTemporarySshMachineLocation(HostAndPort hostAndPort, LoginCredentials creds, ConfigBag config) {
+        String initialUser = creds.getUser();
         Optional<String> initialPassword = creds.getOptionalPassword();
         Optional<String> initialPrivateKey = creds.getOptionalPrivateKey();
-        String initialUser = creds.getUser();
 
         Map<String,Object> sshProps = Maps.newLinkedHashMap(config.getAllConfig());
         sshProps.put("user", initialUser);
         sshProps.put("address", hostAndPort.getHostText());
         sshProps.put("port", hostAndPort.getPort());
         sshProps.put(AbstractLocation.TEMPORARY_LOCATION.getName(), true);
+        sshProps.remove("password");
+        sshProps.remove("privateKeyData");
+        sshProps.remove("privateKeyFile");
+        sshProps.remove("privateKeyPassphrase");
+
         if (initialPassword.isPresent()) sshProps.put("password", initialPassword.get());
         if (initialPrivateKey.isPresent()) sshProps.put("privateKeyData", initialPrivateKey.get());
-        if (initialPrivateKey.isPresent()) sshProps.put("privateKeyData", initialPrivateKey.get());
 
         if (isManaged()) {
             return getManagementContext().getLocationManager().createLocation(sshProps, SshMachineLocation.class);
@@ -1602,48 +1634,25 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                     commands.add(statement.render(scriptOsFamily));
                 }
 
-                Optional<String> initialPassword = initialCredentials.getOptionalPassword();
-                Optional<String> initialPrivateKey = initialCredentials.getOptionalPrivateKey();
                 String initialUser = initialCredentials.getUser();
                 String address = hostAndPortOverride.isPresent() ? hostAndPortOverride.get().getHostText() : JcloudsUtil.getFirstReachableAddress(computeService.getContext(), node);
                 int port = hostAndPortOverride.isPresent() ? hostAndPortOverride.get().getPort() : node.getLoginPort();
                 
-                Map<String,Object> sshProps = Maps.newLinkedHashMap(config.getAllConfig());
-                sshProps.put("user", initialUser);
-                sshProps.put("address", address);
-                sshProps.put("port", port);
-                if (initialPassword.isPresent()) {
-                    sshProps.put("password", initialPassword.get());
-                } else {
-                    sshProps.remove("password");
-                }
-                if (initialPrivateKey.isPresent()) {
-                    sshProps.put("privateKeyData", initialPrivateKey.get());
-                } else {
-                    sshProps.remove("privateKeyData");
-                }
-    
                 // TODO Retrying lots of times as workaround for vcloud-director. There the guest customizations
                 // can cause the VM to reboot shortly after it was ssh'able.
                 Map<String,Object> execProps = Maps.newLinkedHashMap();
                 execProps.put(ShellTool.PROP_RUN_AS_ROOT.getName(), true);
                 execProps.put(SshTool.PROP_SSH_TRIES.getName(), 50);
                 execProps.put(SshTool.PROP_SSH_TRIES_TIMEOUT.getName(), 10*60*1000);
-    
+
                 if (LOG.isDebugEnabled()) {
                     LOG.debug("VM {}: executing user creation/setup via {}@{}:{}; commands: {}", new Object[] {
                             config.getDescription(), initialUser, address, port, commands});
                 }
-    
-                SshMachineLocation sshLoc = null;
+
+                HostAndPort hostAndPort = hostAndPortOverride.isPresent() ? hostAndPortOverride.get() : HostAndPort.fromParts(address, port);
+                SshMachineLocation sshLoc = createTemporarySshMachineLocation(hostAndPort, initialCredentials, config);
                 try {
-                    if (isManaged()) {
-                        sshProps.put(AbstractLocation.TEMPORARY_LOCATION.getName(), true);
-                        sshLoc = getManagementContext().getLocationManager().createLocation(sshProps, SshMachineLocation.class);
-                    } else {
-                        sshLoc = new SshMachineLocation(sshProps);
-                    }
-    
                     // BROOKLYN-188: for SUSE, need to specify the path (for groupadd, useradd, etc)
                     Map<String, ?> env = ImmutableMap.of("PATH", sbinPath());
                     
@@ -1766,6 +1775,9 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
 
                 if (credential.isUsingPassword()) {
                     createdUserCreds = LoginCredentials.builder().user(user).password(credential.getPassword()).build();
+                    if (Boolean.FALSE.equals(config.get(DISABLE_ROOT_AND_PASSWORD_SSH))) {
+                        statements.add(org.jclouds.scriptbuilder.statements.ssh.SshStatements.sshdConfig(ImmutableMap.of("PasswordAuthentication", "yes")));
+                    }
                 } else if (credential.hasKey()) {
                     createdUserCreds = LoginCredentials.builder().user(user).privateKey(credential.getPrivateKeyData()).build();
                 }
@@ -1788,6 +1800,8 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
             if (config.get(PUBLIC_KEY_FILE) != null) config.put(PUBLIC_KEY_FILE, "");
 
         } else if (Strings.isBlank(user) || user.equals(loginUser) || user.equals(ROOT_USERNAME)) {
+            boolean useKey = Strings.isNonBlank(credential.getPublicKeyData());
+            
             // For subsequent ssh'ing, we'll be using the loginUser
             if (Strings.isBlank(user)) {
                 user = loginUser;
@@ -1800,12 +1814,20 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
             statements.add(new ReplaceShadowPasswordEntry(Sha512Crypt.function(), user, passwordToSet));
             createdUserCreds = LoginCredentials.builder().user(user).password(passwordToSet).build();
 
-            if (Strings.isNonBlank(credential.getPublicKeyData())) {
+            if (useKey) {
                 statements.add(new AuthorizeRSAPublicKeys("~"+user+"/.ssh", ImmutableList.of(credential.getPublicKeyData())));
-                if (!credential.isUsingPassword() && Strings.isNonBlank(credential.getPrivateKeyData())) {
+                if (Strings.isNonBlank(credential.getPrivateKeyData())) {
                     createdUserCreds = LoginCredentials.builder().user(user).privateKey(credential.getPrivateKeyData()).build();
                 }
             }
+            
+            if (!useKey || Boolean.FALSE.equals(config.get(DISABLE_ROOT_AND_PASSWORD_SSH))) {
+                // ensure password is permitted for ssh
+                statements.add(org.jclouds.scriptbuilder.statements.ssh.SshStatements.sshdConfig(ImmutableMap.of("PasswordAuthentication", "yes")));
+                if (user.equals(ROOT_USERNAME)) {
+                    statements.add(org.jclouds.scriptbuilder.statements.ssh.SshStatements.sshdConfig(ImmutableMap.of("PermitRootLogin", "yes")));
+                }
+            }
 
         } else {
             String pubKey = credential.getPublicKeyData();
@@ -1883,8 +1905,10 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                 createdUserCreds = LoginCredentials.builder().user(user).privateKey(privKey).build();
             } else if (passwordToSet!=null) {
                 createdUserCreds = LoginCredentials.builder().user(user).password(passwordToSet).build();
-                
-                // if setting a password also ensure password is permitted for ssh
+            }
+            
+            if (!useKey || Boolean.FALSE.equals(config.get(DISABLE_ROOT_AND_PASSWORD_SSH))) {
+                // ensure password is permitted for ssh
                 statements.add(org.jclouds.scriptbuilder.statements.ssh.SshStatements.sshdConfig(ImmutableMap.of("PasswordAuthentication", "yes")));
             }
         }
@@ -2096,13 +2120,13 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         return registerJcloudsSshMachineLocation(null, node, null, sshHostAndPort, setup);
     }
 
-    protected JcloudsSshMachineLocation registerJcloudsSshMachineLocation(ComputeService computeService, NodeMetadata node, LoginCredentials initialCredentials, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) throws IOException {
-        if (initialCredentials==null)
-            initialCredentials = node.getCredentials();
+    protected JcloudsSshMachineLocation registerJcloudsSshMachineLocation(ComputeService computeService, NodeMetadata node, LoginCredentials userCredentials, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) throws IOException {
+        if (userCredentials == null)
+            userCredentials = node.getCredentials();
 
         String vmHostname = getPublicHostname(node, sshHostAndPort, setup);
 
-        JcloudsSshMachineLocation machine = createJcloudsSshMachineLocation(computeService, node, vmHostname, sshHostAndPort, setup);
+        JcloudsSshMachineLocation machine = createJcloudsSshMachineLocation(computeService, node, vmHostname, sshHostAndPort, userCredentials, setup);
         registerJcloudsMachineLocation(node.getId(), machine);
         return machine;
     }
@@ -2113,13 +2137,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         vmInstanceIds.put(machine, nodeId);
     }
     
-    /** @deprecated since 0.7.0 use variant which takes compute service; no longer called internally,
-     * so marked final to force any overrides to switch to new syntax */
-    @Deprecated
-    protected final JcloudsSshMachineLocation createJcloudsSshMachineLocation(NodeMetadata node, String vmHostname, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) throws IOException {
-        return createJcloudsSshMachineLocation(null, node, vmHostname, sshHostAndPort, setup);
-    }
-    protected JcloudsSshMachineLocation createJcloudsSshMachineLocation(ComputeService computeService, NodeMetadata node, String vmHostname, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) throws IOException {
+    protected JcloudsSshMachineLocation createJcloudsSshMachineLocation(ComputeService computeService, NodeMetadata node, String vmHostname, Optional<HostAndPort> sshHostAndPort, LoginCredentials userCredentials, ConfigBag setup) throws IOException {
         Map<?,?> sshConfig = extractSshConfig(setup, node);
         String nodeAvailabilityZone = extractAvailabilityZone(setup, node);
         String nodeRegion = extractRegion(setup, node);
@@ -2159,11 +2177,13 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                     .configure("displayName", vmHostname)
                     .configure("address", address)
                     .configure(JcloudsSshMachineLocation.SSH_PORT, sshHostAndPort.isPresent() ? sshHostAndPort.get().getPort() : node.getLoginPort())
-                    .configure("user", getUser(setup))
                     // don't think "config" does anything
                     .configure(sshConfig)
                     // FIXME remove "config" -- inserted directly, above
                     .configure("config", sshConfig)
+                    .configure("user", userCredentials.getUser())
+                    .configure(SshMachineLocation.PASSWORD, userCredentials.getOptionalPassword().orNull())
+                    .configure(SshMachineLocation.PRIVATE_KEY_DATA, userCredentials.getOptionalPrivateKey().orNull())
                     .configure("jcloudsParent", this)
                     .configure("node", node)
                     .configureIfNotNull(CLOUD_AVAILABILITY_ZONE_ID, nodeAvailabilityZone)
@@ -2180,11 +2200,13 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                     .put("displayName", vmHostname)
                     .put("address", address)
                     .put("port", sshHostAndPort.isPresent() ? sshHostAndPort.get().getPort() : node.getLoginPort())
-                    .put("user", getUser(setup))
                     // don't think "config" does anything
                     .putAll(sshConfig)
                     // FIXME remove "config" -- inserted directly, above
                     .put("config", sshConfig)
+                    .put("user", userCredentials.getUser())
+                    .putIfNotNull(SshMachineLocation.PASSWORD.getName(), userCredentials.getOptionalPassword().orNull())
+                    .putIfNotNull(SshMachineLocation.PRIVATE_KEY_DATA.getName(), userCredentials.getOptionalPrivateKey().orNull())
                     .put("callerContext", setup.get(CALLER_CONTEXT))
                     .putIfNotNull(CLOUD_AVAILABILITY_ZONE_ID.getName(), nodeAvailabilityZone)
                     .putIfNotNull(CLOUD_REGION_ID.getName(), nodeRegion)
@@ -2575,31 +2597,47 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         String connectionDetails = user + "@" + vmIp + ":" + vmPort;
         final HostAndPort hostAndPort = hostAndPortOverride.isPresent() ? hostAndPortOverride.get() : HostAndPort.fromParts(vmIp, vmPort);
         final AtomicReference<LoginCredentials> credsSuccessful = new AtomicReference<LoginCredentials>();
-        
+
+        // Don't use config that relates to the final user credentials (those have nothing to do 
+        // with the initial credentials of the VM returned by the cloud provider).
+        ConfigBag sshProps = ConfigBag.newInstanceCopying(setup);
+        sshProps.remove("password");
+        sshProps.remove("privateKeyData");
+        sshProps.remove("privateKeyFile");
+        sshProps.remove("privateKeyPassphrase");
+
         final Map<SshMachineLocation, LoginCredentials> machinesToTry = Maps.newLinkedHashMap();
         for (LoginCredentials creds : credentialsToTry) {
-            machinesToTry.put(createTemporarySshMachineLocation(hostAndPort, creds, setup), creds);
+            machinesToTry.put(createTemporarySshMachineLocation(hostAndPort, creds, sshProps), creds);
         }
-        Callable<Boolean> checker = new Callable<Boolean>() {
-            public Boolean call() {
-                for (Map.Entry<SshMachineLocation, LoginCredentials> entry : machinesToTry.entrySet()) {
-                    SshMachineLocation machine = entry.getKey();
-                    int exitstatus = machine.execScript(
-                            ImmutableMap.of(
-                                    SshTool.PROP_SSH_TRIES_TIMEOUT.getName(), Duration.THIRTY_SECONDS.toMilliseconds(),
-                                    SshTool.PROP_SSH_TRIES.getName(), 1), 
-                            "check-connectivity", 
-                            ImmutableList.of("true"));
-                    boolean success = (exitstatus == 0);
-                    if (success) {
-                        credsSuccessful.set(entry.getValue());
-                        return true;
+        try {
+            Callable<Boolean> checker = new Callable<Boolean>() {
+                public Boolean call() {
+                    for (Map.Entry<SshMachineLocation, LoginCredentials> entry : machinesToTry.entrySet()) {
+                        SshMachineLocation machine = entry.getKey();
+                        int exitstatus = machine.execScript(
+                                ImmutableMap.of(
+                                        SshTool.PROP_SSH_TRIES_TIMEOUT.getName(), Duration.THIRTY_SECONDS.toMilliseconds(),
+                                        SshTool.PROP_SSH_TRIES.getName(), 1), 
+                                "check-connectivity", 
+                                ImmutableList.of("true"));
+                        boolean success = (exitstatus == 0);
+                        if (success) {
+                            credsSuccessful.set(entry.getValue());
+                            return true;
+                        }
                     }
-                }
-                return false;
-            }};
-
-        waitForReachable(checker, connectionDetails, credentialsToTry, setup, timeout);
+                    return false;
+                }};
+    
+            waitForReachable(checker, connectionDetails, credentialsToTry, setup, timeout);
+        } finally {
+            for (SshMachineLocation machine : machinesToTry.keySet()) {
+                getManagementContext().getLocationManager().unmanage(machine);
+                Streams.closeQuietly(machine);
+            }
+        }
+        
         return credsSuccessful.get();
     }
 
@@ -2702,12 +2740,16 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         }
     }
 
+    protected String getPublicHostname(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) {
+        return getPublicHostname(node, sshHostAndPort, node.getCredentials(), setup);
+    }
+    
     /**
      * Attempts to obtain the hostname or IP of the node, as advertised by the cloud provider.
      * Prefers public, reachable IPs.
      * For some clouds (e.g. aws-ec2), it will attempt to find the public hostname.
      */
-    protected String getPublicHostname(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) {
+    protected String getPublicHostname(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, LoginCredentials userCredentials, ConfigBag setup) {
         String provider = (setup != null) ? setup.get(CLOUD_PROVIDER) : null;
         if (provider == null) provider= getProvider();
 
@@ -2731,7 +2773,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                 } else {
                     HostAndPort hostAndPortToUse = sshHostAndPort.isPresent() ? sshHostAndPort.get() : inferredHostAndPort;
                     try {
-                        return getPublicHostnameAws(hostAndPortToUse, setup);
+                        return getPublicHostnameAws(hostAndPortToUse, userCredentials, setup);
                     } catch (Exception e) {
                         if (inferredHostAndPort != null) { 
                             LOG.warn("Error querying aws-ec2 instance "+node.getId()+"@"+node.getLocation()+" over ssh for its hostname; falling back to first reachable IP", e);
@@ -2768,27 +2810,11 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         }
     }
 
-    private String getPublicHostnameAws(HostAndPort sshHostAndPort, ConfigBag setup) {
+    private String getPublicHostnameAws(HostAndPort hostAndPort, LoginCredentials userCredentials, ConfigBag setup) {
         SshMachineLocation sshLocByIp = null;
         try {
-            ConfigBag sshConfig = extractSshConfig(setup, new ConfigBag());
-
             // TODO messy way to get an SSH session
-            if (isManaged()) {
-                sshLocByIp = getManagementContext().getLocationManager().createLocation(LocationSpec.create(SshMachineLocation.class)
-                        .configure("address", sshHostAndPort.getHostText())
-                        .configure("port", sshHostAndPort.getPort())
-                        .configure("user", getUser(setup))
-                        .configure(sshConfig.getAllConfig()));
-            } else {
-                MutableMap<Object, Object> locationProps = MutableMap.builder()
-                        .put("address", sshHostAndPort.getHostText())
-                        .put("port", sshHostAndPort.getPort())
-                        .put("user", getUser(setup))
-                        .putAll(sshConfig.getAllConfig())
-                        .build();
-                sshLocByIp = new SshMachineLocation(locationProps);
-            }
+            sshLocByIp = createTemporarySshMachineLocation(hostAndPort, userCredentials, setup);
 
             ByteArrayOutputStream outStream = new ByteArrayOutputStream();
             ByteArrayOutputStream errStream = new ByteArrayOutputStream();
@@ -2803,7 +2829,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
             for (String line : outLines) {
                 if (line.startsWith("ec2-")) return line.trim();
             }
-            throw new IllegalStateException("Could not obtain aws-ec2 hostname for vm "+sshHostAndPort+"; exitcode="+exitcode+"; stdout="+outString+"; stderr="+new String(errStream.toByteArray()));
+            throw new IllegalStateException("Could not obtain aws-ec2 hostname for vm "+hostAndPort+"; exitcode="+exitcode+"; stdout="+outString+"; stderr="+new String(errStream.toByteArray()));
         } finally {
             Streams.closeQuietly(sshLocByIp);
         }
@@ -2815,6 +2841,10 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
      * For some clouds (e.g. aws-ec2), it will attempt to find the fully qualified hostname (as that works in public+private).
      */
     protected String getPrivateHostname(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) {
+        return getPrivateHostname(node, sshHostAndPort, node.getCredentials(), setup);
+    }
+    
+    protected String getPrivateHostname(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, LoginCredentials userCredentials, ConfigBag setup) {
         String provider = (setup != null) ? setup.get(CLOUD_PROVIDER) : null;
         if (provider == null) provider= getProvider();
 
@@ -2822,14 +2852,14 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         // exceptional situation rather than a pattern to follow. We need a better way to
         // do cloud-specific things.
         if ("aws-ec2".equals(provider)) {
-            Maybe<String> result = getPrivateHostnameAws(node, sshHostAndPort, setup);
+            Maybe<String> result = getPrivateHostnameAws(node, sshHostAndPort, userCredentials, setup);
             if (result.isPresent()) return result.get();
         }
 
         return getPrivateHostnameGeneric(node, setup);
     }
 
-    private Maybe<String> getPrivateHostnameAws(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) {
+    private Maybe<String> getPrivateHostnameAws(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, LoginCredentials userCredentials, ConfigBag setup) {
         // TODO Remove duplication from getPublicHostname.
         // TODO Don't like 
         HostAndPort inferredHostAndPort = null;
@@ -2845,7 +2875,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         if (sshHostAndPort.isPresent() || inferredHostAndPort != null) {
             HostAndPort hostAndPortToUse = sshHostAndPort.isPresent() ? sshHostAndPort.get() : inferredHostAndPort;
             try {
-                return Maybe.of(getPublicHostnameAws(hostAndPortToUse, setup));
+                return Maybe.of(getPublicHostnameAws(hostAndPortToUse, userCredentials, setup));
             } catch (Exception e) {
                 LOG.warn("Error querying aws-ec2 instance instance "+node.getId()+"@"+node.getLocation()+" over ssh for its hostname; falling back to jclouds metadata for address", e);
             }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/f34e7e69/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/AbstractJcloudsLiveTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/AbstractJcloudsLiveTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/AbstractJcloudsLiveTest.java
index 311819d..7723a01 100644
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/AbstractJcloudsLiveTest.java
+++ b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/AbstractJcloudsLiveTest.java
@@ -52,6 +52,7 @@ public class AbstractJcloudsLiveTest {
     public static final String AWS_EC2_PROVIDER = "aws-ec2";
     public static final String AWS_EC2_MICRO_HARDWARE_ID = "t1.micro";
     public static final String AWS_EC2_SMALL_HARDWARE_ID = "m1.small";
+    public static final String AWS_EC2_MEDIUM_HARDWARE_ID = "m3.medium";
     public static final String AWS_EC2_EUWEST_REGION_NAME = "eu-west-1";
     public static final String AWS_EC2_USEAST_REGION_NAME = "us-east-1";
 

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/f34e7e69/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsLoginLiveTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsLoginLiveTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsLoginLiveTest.java
index 4e3f567..d8c38fa 100644
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsLoginLiveTest.java
+++ b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsLoginLiveTest.java
@@ -19,19 +19,21 @@
 package org.apache.brooklyn.location.jclouds;
 
 import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
 
 import java.io.File;
 import java.util.Map;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testng.Assert;
-import org.testng.annotations.Test;
 import org.apache.brooklyn.api.location.LocationSpec;
 import org.apache.brooklyn.location.ssh.SshMachineLocation;
 import org.apache.brooklyn.util.collections.MutableMap;
 import org.apache.brooklyn.util.os.Os;
 import org.apache.brooklyn.util.stream.Streams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -171,27 +173,50 @@ public class JcloudsLoginLiveTest extends AbstractJcloudsLiveTest {
     }
 
     @Test(groups = {"Live"})
-    protected void testSpecifyingPasswordAndSshKeysPrefersKeys() throws Exception {
+    public void testSpecifyingPasswordAndSshKeysPrefersKeysAndDisablesPassword() throws Exception {
+        runSpecifyingPasswordAndSshKeysPrefersKeys(false);
+    }
+
+    @Test(groups = {"Live"})
+    public void testSpecifyingPasswordAndSshKeysPrefersKeysAndAllowsPassword() throws Exception {
+        runSpecifyingPasswordAndSshKeysPrefersKeys(true);
+    }
+
+    protected void runSpecifyingPasswordAndSshKeysPrefersKeys(boolean leavePasswordSsh) throws Exception {
         brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.USER.getName(), "myname");
         brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.PRIVATE_KEY_FILE.getName(), "~/.ssh/id_rsa");
         brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.PUBLIC_KEY_FILE.getName(), "~/.ssh/id_rsa.pub");
         brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.PASSWORD.getName(), "mypassword");
+        if (leavePasswordSsh) {
+            brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.DISABLE_ROOT_AND_PASSWORD_SSH.getName(), false);
+        }
         jcloudsLocation = (JcloudsLocation) managementContext.getLocationRegistry().resolve(AWS_EC2_LOCATION_SPEC);
         
         machine = createEc2Machine();
         assertSshable(machine);
         
-        assertSshable(ImmutableMap.builder()
-                .put("address", machine.getAddress())
-                .put("user", "myname")
-                .put(SshMachineLocation.PRIVATE_KEY_FILE, Os.tidyPath("~/.ssh/id_rsa"))
-                .build());
+        assertNull(machine.config().get(SshMachineLocation.PASSWORD));
+        assertNotNull(machine.config().get(SshMachineLocation.PRIVATE_KEY_DATA));
         
         assertSshable(ImmutableMap.builder()
                 .put("address", machine.getAddress())
                 .put("user", "myname")
-                .put(SshMachineLocation.PASSWORD, "mypassword")
+                .put(SshMachineLocation.PRIVATE_KEY_FILE, Os.tidyPath("~/.ssh/id_rsa"))
                 .build());
+
+        if (leavePasswordSsh) {
+            assertSshable(ImmutableMap.builder()
+                    .put("address", machine.getAddress())
+                    .put("user", "myname")
+                    .put(SshMachineLocation.PASSWORD, "mypassword")
+                    .build());
+        } else {
+            assertNotSshable(ImmutableMap.builder()
+                    .put("address", machine.getAddress())
+                    .put("user", "myname")
+                    .put(SshMachineLocation.PASSWORD, "mypassword")
+                    .build());
+        }
     }
 
     @Test(groups = {"Live"})
@@ -244,6 +269,7 @@ public class JcloudsLoginLiveTest extends AbstractJcloudsLiveTest {
         brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.USER.getName(), "myname");
         brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.PASSWORD.getName(), "mypassword");
         brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.PUBLIC_KEY_FILE.getName(), "~/.ssh/id_rsa.pub");
+        brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.DISABLE_ROOT_AND_PASSWORD_SSH.getName(), false);
         jcloudsLocation = (JcloudsLocation) managementContext.getLocationRegistry().resolve(AWS_EC2_LOCATION_SPEC);
         
         machine = createEc2Machine();
@@ -270,6 +296,7 @@ public class JcloudsLoginLiveTest extends AbstractJcloudsLiveTest {
             
             brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.USER.getName(), "root");
             brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.PASSWORD.getName(), "mypassword");
+            brooklynProperties.put(BROOKLYN_PROPERTIES_PREFIX+JcloudsLocationConfig.DISABLE_ROOT_AND_PASSWORD_SSH.getName(), false);
             jcloudsLocation = (JcloudsLocation) managementContext.getLocationRegistry().resolve(AWS_EC2_LOCATION_SPEC);
             
             machine = createEc2Machine();
@@ -351,7 +378,7 @@ public class JcloudsLoginLiveTest extends AbstractJcloudsLiveTest {
         return obtainMachine(MutableMap.<String,Object>builder()
                 .putAll(conf)
                 .putIfAbsent("imageId", AWS_EC2_CENTOS_IMAGE_ID)
-                .putIfAbsent("hardwareId", AWS_EC2_SMALL_HARDWARE_ID)
+                .putIfAbsent("hardwareId", AWS_EC2_MEDIUM_HARDWARE_ID)
                 .putIfAbsent("inboundPorts", ImmutableList.of(22))
                 .build());
     }