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/06/14 22:47:53 UTC

[03/10] incubator-brooklyn git commit: Windows support fixes (for vCloud Director)

Windows support fixes (for vCloud Director)

- WinRmMachineLocation respects host:port
- Adds JcloudsLocation config option osFamilyOverride. If set, then
  assumes it is a Windows VM even if the metadata from the cloud says
  “unrecognised”.
- Fixes WinRM login over NAT

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

Branch: refs/heads/master
Commit: 22638627997fc410409872ab9dee39a581d86258
Parents: 51e7e0b
Author: Aled Sage <al...@gmail.com>
Authored: Fri Jun 12 16:45:50 2015 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Jun 12 16:45:50 2015 +0100

----------------------------------------------------------------------
 .../location/basic/WinRmMachineLocation.java    |  15 +--
 .../location/jclouds/JcloudsLocation.java       | 108 +++++++++++++------
 .../location/jclouds/JcloudsLocationConfig.java |   3 +
 .../location/jclouds/JcloudsLocationTest.java   |   2 +-
 4 files changed, 86 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/22638627/core/src/main/java/brooklyn/location/basic/WinRmMachineLocation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/location/basic/WinRmMachineLocation.java b/core/src/main/java/brooklyn/location/basic/WinRmMachineLocation.java
index cb59102..36c0360 100644
--- a/core/src/main/java/brooklyn/location/basic/WinRmMachineLocation.java
+++ b/core/src/main/java/brooklyn/location/basic/WinRmMachineLocation.java
@@ -53,6 +53,7 @@ import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
+import com.google.common.net.HostAndPort;
 
 public class WinRmMachineLocation extends AbstractLocation implements MachineLocation {
 
@@ -113,6 +114,12 @@ public class WinRmMachineLocation extends AbstractLocation implements MachineLoc
         return (address != null) ? address.getHostAddress() : null;
     }
 
+    @Nullable
+    protected String getHostAndPort() {
+        String host = getHostname();
+        return (host == null) ? null : host + ":" + config().get(WINRM_PORT);
+    }
+
     @Override
     public Set<String> getPublicAddresses() {
         InetAddress address = getAddress();
@@ -150,7 +157,7 @@ public class WinRmMachineLocation extends AbstractLocation implements MachineLoc
     }
 
     protected WinRmToolResponse executeScriptNoRetry(List<String> script) {
-        WinRmTool winRmTool = WinRmTool.connect(getHostname(), getUser(), getPassword());
+        WinRmTool winRmTool = WinRmTool.connect(getHostAndPort(), getUser(), getPassword());
         WinRmToolResponse response = winRmTool.executeScript(script);
         return response;
     }
@@ -183,7 +190,7 @@ public class WinRmMachineLocation extends AbstractLocation implements MachineLoc
     }
 
     public WinRmToolResponse executePsScriptNoRetry(List<String> psScript) {
-        WinRmTool winRmTool = WinRmTool.connect(getHostname(), getUser(), getPassword());
+        WinRmTool winRmTool = WinRmTool.connect(getHostAndPort(), getUser(), getPassword());
         WinRmToolResponse response = winRmTool.executePs(psScript);
         return response;
     }
@@ -231,10 +238,6 @@ public class WinRmMachineLocation extends AbstractLocation implements MachineLoc
     @Override
     public void init() {
         super.init();
-        
-        getRequiredConfig(ADDRESS);
-        getRequiredConfig(USER);
-        getRequiredConfig(PASSWORD);
     }
 
     public String getUser() {

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/22638627/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocation.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocation.java b/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocation.java
index 7d3d50e..15ea553 100644
--- a/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocation.java
+++ b/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocation.java
@@ -328,17 +328,25 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
     
     /**
      * Whether VMs provisioned from this image will be Windows. Assume windows if the image
-     * explicitly says so, or if the config explicitly says {@link JcloudsLocationConfig#OS_FAMILY} 
-     * is Windows.
+     * explicitly says so, or if image does not tell us then fall back to whether the config 
+     * explicitly says windows in {@link JcloudsLocationConfig#OS_FAMILY}.
+     * 
+     * Will first look at {@link JcloudsLocationConfig#OS_FAMILY_OVERRIDE}, to check if that 
+     * is set. If so, no further checks are done: the value is compared against {@link OsFamily#WINDOWS}.
      * 
      * We believe the config (e.g. from brooklyn.properties) because for some clouds there is 
      * insufficient meta-data so the Image might not tell us. Thus a user can work around it
      * by explicitly supplying configuration. 
      */
     public boolean isWindows(Image image, ConfigBag config) {
+        OsFamily override = config.get(OS_FAMILY_OVERRIDE);
+        if (override != null) return override == OsFamily.WINDOWS;
+        
         OsFamily confFamily = config.get(OS_FAMILY);
         OperatingSystem os = (image != null) ? image.getOperatingSystem() : null;
-        return (os != null) ? (OsFamily.WINDOWS == os.getFamily()) : (OsFamily.WINDOWS == confFamily);
+        return (os != null && os.getFamily() != OsFamily.UNRECOGNIZED) 
+                ? (OsFamily.WINDOWS == os.getFamily()) 
+                : (OsFamily.WINDOWS == confFamily);
     }
 
     /**
@@ -347,9 +355,14 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
      * @see {@link #isWindows(Image, ConfigBag)}
      */
     public boolean isWindows(NodeMetadata node, ConfigBag config) {
+        OsFamily override = config.get(OS_FAMILY_OVERRIDE);
+        if (override != null) return override == OsFamily.WINDOWS;
+        
         OsFamily confFamily = config.get(OS_FAMILY);
         OperatingSystem os = (node != null) ? node.getOperatingSystem() : null;
-        return (os != null) ? (OsFamily.WINDOWS == os.getFamily()) : (OsFamily.WINDOWS == confFamily);
+        return (os != null && os.getFamily() != OsFamily.UNRECOGNIZED) 
+                ? (OsFamily.WINDOWS == os.getFamily()) 
+                : (OsFamily.WINDOWS == confFamily);
     }
 
     protected Semaphore getMachineCreationSemaphore() {
@@ -682,14 +695,24 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                 throw new IllegalStateException("No nodes returned by jclouds create-nodes in " + setup.getDescription());
 
             boolean windows = isWindows(node, setup);
-
+            if (windows) {
+                int newLoginPort = node.getLoginPort() == 22 ? 5985 : node.getLoginPort();
+                String newLoginUser = "root".equals(node.getCredentials().getUser()) ? "Administrator" : node.getCredentials().getUser();
+                LOG.debug("jclouds created Windows VM {}; transforming connection details: loginPort from {} to {}; loginUser from {} to {}", 
+                        new Object[] {node, node.getLoginPort(), newLoginPort, node.getCredentials().getUser(), newLoginUser});
+                
+                node = NodeMetadataBuilder.fromNodeMetadata(node)
+                        .loginPort(newLoginPort)
+                        .credentials(LoginCredentials.builder(node.getCredentials()).user(newLoginUser).build())
+                        .build();
+            }
             // FIXME How do we influence the node.getLoginPort, so it is set correctly for Windows?
             // Setup port-forwarding, if required
             Optional<HostAndPort> sshHostAndPortOverride;
             if (usePortForwarding) {
                 sshHostAndPortOverride = Optional.of(portForwarder.openPortForwarding(
                         node,
-                        (windows ? 5985 : node.getLoginPort()),
+                        node.getLoginPort(),
                         Optional.<Integer>absent(),
                         Protocol.TCP,
                         Cidr.UNIVERSAL));
@@ -745,7 +768,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
 //            WinRmMachineLocation winRmMachineLocation = null;
             // Create a JcloudsSshMachineLocation, and register it
             if (windows) {
-                machineLocation = registerWinRmMachineLocation(computeService, node, setup);
+                machineLocation = registerWinRmMachineLocation(computeService, node, userCredentials, sshHostAndPortOverride, setup);
             } else {
                 machineLocation = registerJcloudsSshMachineLocation(computeService, node, userCredentials, sshHostAndPortOverride, setup);
                 if (template!=null && machineLocation.getTemplate()==null) {
@@ -1075,11 +1098,13 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                             }
                             if (userDataMethod != null) {
                                 try {
-                                    userDataMethod.invoke(Strings.toString(v));
+                                    userDataMethod.invoke(t, Strings.toString(v));
                                 } catch (InvocationTargetException e) {
-                                    LOG.info("Problem invoking "+userDataMethod.getName()+" of "+t.getClass()+", for setting userData", e);
+                                    LOG.info("Problem invoking "+userDataMethod.getName()+" of "+t.getClass()+", for setting userData (rethrowing)", e);
+                                    throw Exceptions.propagate(e);
                                 } catch (IllegalAccessException e) {
-                                    LOG.info("Unable to reflectively invoke "+userDataMethod.getName()+" of "+t.getClass()+", for setting userData", e);
+                                    LOG.debug("Unable to reflectively invoke "+userDataMethod.getName()+" of "+t.getClass()+", for setting userData (rethrowing)", e);
+                                    throw Exceptions.propagate(e);
                                 }
                             } else {
                                 LOG.info("ignoring userDataString({}) in VM creation because not supported for cloud/type ({})", v, t.getClass());
@@ -1446,12 +1471,10 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         if (!userCreation.statements.isEmpty()) {
             // If unsure of OS family, default to unix for rendering statements.
             org.jclouds.scriptbuilder.domain.OsFamily scriptOsFamily;
-            if (node.getOperatingSystem() == null) {
-                scriptOsFamily = org.jclouds.scriptbuilder.domain.OsFamily.UNIX;
+            if (isWindows(node, config)) {
+                scriptOsFamily = org.jclouds.scriptbuilder.domain.OsFamily.WINDOWS;
             } else {
-                scriptOsFamily = (node.getOperatingSystem().getFamily() == org.jclouds.compute.domain.OsFamily.WINDOWS)
-                        ? org.jclouds.scriptbuilder.domain.OsFamily.WINDOWS
-                        : org.jclouds.scriptbuilder.domain.OsFamily.UNIX;
+                scriptOsFamily = org.jclouds.scriptbuilder.domain.OsFamily.UNIX;
             }
 
             boolean windows = isWindows(node, config);
@@ -1903,16 +1926,16 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         String vmHostname = getPublicHostname(node, sshHostAndPort, setup);
 
         JcloudsSshMachineLocation machine = createJcloudsSshMachineLocation(computeService, node, vmHostname, sshHostAndPort, setup);
-        registerJcloudsSshMachineLocation(node.getId(), machine);
+        registerJcloudsMachineLocation(node.getId(), machine);
         return machine;
     }
 
     @VisibleForTesting
-    protected void registerJcloudsSshMachineLocation(String nodeId, JcloudsSshMachineLocation machine) {
+    protected void registerJcloudsMachineLocation(String nodeId, JcloudsMachineLocation machine) {
         machine.setParent(this);
         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
@@ -1997,17 +2020,18 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         }
     }
 
-    protected JcloudsWinRmMachineLocation registerWinRmMachineLocation(ComputeService computeService, NodeMetadata node, ConfigBag setup) {
-        // FIMXE: Need to write WinRM equivalent of getPublicHostname
-        String vmHostname = node.getPublicAddresses().iterator().next();
-        
-        JcloudsWinRmMachineLocation winRmMachineLocation = createWinRmMachineLocation(computeService, node, vmHostname, setup);
-        winRmMachineLocation.setParent(this);
-        vmInstanceIds.put(winRmMachineLocation, node.getId());
-        return winRmMachineLocation;
+    protected JcloudsWinRmMachineLocation registerWinRmMachineLocation(ComputeService computeService, NodeMetadata node, LoginCredentials initialCredentials, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) {
+        if (initialCredentials==null)
+            initialCredentials = node.getCredentials();
+
+        String vmHostname = getPublicHostname(node, sshHostAndPort, setup);
+
+        JcloudsWinRmMachineLocation machine = createWinRmMachineLocation(computeService, node, vmHostname, sshHostAndPort, setup);
+        registerJcloudsMachineLocation(node.getId(), machine);
+        return machine;
     }
 
-    protected JcloudsWinRmMachineLocation createWinRmMachineLocation(ComputeService computeService, NodeMetadata node, String vmHostname, ConfigBag setup) {
+    protected JcloudsWinRmMachineLocation createWinRmMachineLocation(ComputeService computeService, NodeMetadata node, String vmHostname, Optional<HostAndPort> sshHostAndPort, ConfigBag setup) {
         String nodeAvailabilityZone = extractAvailabilityZone(setup, node);
         String nodeRegion = extractRegion(setup, node);
         if (nodeRegion == null) {
@@ -2015,11 +2039,14 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
             nodeRegion = extractProvider(setup, node);
         }
         
+        String address = sshHostAndPort.isPresent() ? sshHostAndPort.get().getHostText() : vmHostname;
+
         if (isManaged()) {
             return getManagementContext().getLocationManager().createLocation(LocationSpec.create(JcloudsWinRmMachineLocation.class)
                     .configure("jcloudsParent", this)
                     .configure("displayName", vmHostname)
-                    .configure("address", vmHostname)
+                    .configure("address", address)
+                    .configure(WinRmMachineLocation.WINRM_PORT, sshHostAndPort.isPresent() ? sshHostAndPort.get().getPort() : node.getLoginPort())
                     .configure("user", getUser(setup))
                     .configure(WinRmMachineLocation.USER, setup.get(USER))
                     .configure(WinRmMachineLocation.PASSWORD, setup.get(PASSWORD))
@@ -2463,13 +2490,24 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                 }
             }
             if (sshHostAndPort.isPresent() || inferredHostAndPort != null) {
-                HostAndPort hostAndPortToUse = sshHostAndPort.isPresent() ? sshHostAndPort.get() : inferredHostAndPort;
-                try {
-                    return getPublicHostnameAws(hostAndPortToUse, setup);
-                } catch (Exception e) {
-                    LOG.warn("Error querying aws-ec2 instance instance "+node.getId()+"@"+node.getLocation()+" over ssh for its hostname; falling back to first reachable IP", e);
-                    // We've already found a reachable address so settle for that, rather than doing it again
-                    if (inferredHostAndPort != null) return inferredHostAndPort.getHostText();
+                if (isWindows(node, setup)) {
+                    if (inferredHostAndPort != null) {
+                        LOG.warn("Cannot querying aws-ec2 Windows instance "+node.getId()+"@"+node.getLocation()+" over ssh for its hostname; falling back to first reachable IP");
+                        return inferredHostAndPort.getHostText();
+                    }
+                } else {
+                    HostAndPort hostAndPortToUse = sshHostAndPort.isPresent() ? sshHostAndPort.get() : inferredHostAndPort;
+                    try {
+                        return getPublicHostnameAws(hostAndPortToUse, 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);
+                            // We've already found a reachable address so settle for that, rather than doing it again
+                            return inferredHostAndPort.getHostText();
+                        } else {
+                            LOG.warn("Error querying aws-ec2 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/22638627/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocationConfig.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocationConfig.java b/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocationConfig.java
index 5e8780e..ab0007a 100644
--- a/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocationConfig.java
+++ b/locations/jclouds/src/main/java/brooklyn/location/jclouds/JcloudsLocationConfig.java
@@ -251,6 +251,9 @@ public interface JcloudsLocationConfig extends CloudLocationConfig {
     public static final ConfigKey<String> OS_VERSION_REGEX = ConfigKeys.newStringConfigKey("osVersionRegex", 
         "Regular expression for the OS version to load");
 
+    public static final ConfigKey<OsFamily> OS_FAMILY_OVERRIDE = ConfigKeys.newConfigKey(OsFamily.class, "osFamilyOverride", 
+            "OS family of VMs (ignores VM metadata from jclouds, and assumes this value)");
+
     public static final ConfigKey<ComputeServiceRegistry> COMPUTE_SERVICE_REGISTRY = ConfigKeys.newConfigKey(
             ComputeServiceRegistry.class,
             "jclouds.computeServiceRegistry",

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/22638627/locations/jclouds/src/test/java/brooklyn/location/jclouds/JcloudsLocationTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/brooklyn/location/jclouds/JcloudsLocationTest.java b/locations/jclouds/src/test/java/brooklyn/location/jclouds/JcloudsLocationTest.java
index 62817c2..3411da3 100644
--- a/locations/jclouds/src/test/java/brooklyn/location/jclouds/JcloudsLocationTest.java
+++ b/locations/jclouds/src/test/java/brooklyn/location/jclouds/JcloudsLocationTest.java
@@ -520,7 +520,7 @@ public class JcloudsLocationTest implements JcloudsLocationConfig {
                 .configure("port", 22) 
                 .configure("user", "bob")
                 .configure("jcloudsParent", this));
-            registerJcloudsSshMachineLocation("bogus", result);
+            registerJcloudsMachineLocation("bogus", result);
             
             // explicitly invoke this customizer, to comply with tests
             for (JcloudsLocationCustomizer customizer : getCustomizers(config().getBag())) {