You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by sv...@apache.org on 2016/09/26 16:37:20 UTC

[3/5] brooklyn-server git commit: Add JcloudsWinrmingLiveTest

Add JcloudsWinrmingLiveTest

A test for JcloudsLocation provisioning (and WinRM\u2019ing to) a Windows VM.

Deletes the old AwsEc2LocationWindowsLiveTest, which was massively out
of date and wrong.

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

Branch: refs/heads/master
Commit: 9c9dd47810627da6b05450e89cbce40787faac17
Parents: d0d8cf0
Author: Aled Sage <al...@gmail.com>
Authored: Fri Sep 23 21:25:18 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Sep 23 21:25:18 2016 +0100

----------------------------------------------------------------------
 .../location/jclouds/JcloudsLocation.java       |  4 +-
 .../jclouds/AbstractJcloudsLiveTest.java        | 27 ++++--
 .../jclouds/JcloudsAddressesLiveTest.java       |  2 +-
 .../location/jclouds/JcloudsLoginLiveTest.java  |  2 +-
 .../jclouds/JcloudsWinrmingLiveTest.java        | 47 ++++++++++
 ...hineProvisioningLocationJcloudsLiveTest.java |  4 +-
 .../provider/AwsEc2LocationWindowsLiveTest.java | 95 --------------------
 7 files changed, 74 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c9dd478/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 dc4b93c..bc57874 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
@@ -1218,7 +1218,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
      * @see #registerMachine(ConfigBag)
      */
     @Override
-    public MachineLocation resumeMachine(Map<?, ?> flags) {
+    public JcloudsMachineLocation resumeMachine(Map<?, ?> flags) {
         ConfigBag setup = ConfigBag.newInstanceExtending(config().getBag(), flags);
         LOG.info("{} using resuming node matching properties: {}", this, Sanitizer.sanitize(setup));
         ComputeService computeService = getComputeService(setup);
@@ -1229,7 +1229,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         // hostname and addresses populated.
         node = findNodeOrThrow(setup);
         LOG.debug("{} resumed {}", this, node);
-        MachineLocation registered = registerMachineLocation(setup, node);
+        JcloudsMachineLocation registered = registerMachineLocation(setup, node);
         LOG.info("{} resumed and registered {}", this, registered);
         return registered;
     }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c9dd478/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 4313c43..4e2e7cc 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
@@ -25,6 +25,7 @@ import static org.testng.Assert.assertNotNull;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.brooklyn.util.core.internal.winrm.WinRmToolResponse;
 import org.apache.brooklyn.util.exceptions.CompoundRuntimeException;
 import org.apache.brooklyn.api.location.MachineLocation;
 import org.slf4j.Logger;
@@ -36,6 +37,7 @@ import org.apache.brooklyn.core.internal.BrooklynProperties;
 import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
 import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests;
 import org.apache.brooklyn.location.ssh.SshMachineLocation;
+import org.apache.brooklyn.location.winrm.WinRmMachineLocation;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -67,7 +69,7 @@ public class AbstractJcloudsLiveTest {
     protected BrooklynProperties brooklynProperties;
     protected LocalManagementContext managementContext;
     
-    protected List<JcloudsSshMachineLocation> machines;
+    protected List<JcloudsMachineLocation> machines;
     protected JcloudsLocation jcloudsLocation;
     
     @BeforeMethod(alwaysRun=true)
@@ -135,6 +137,11 @@ public class AbstractJcloudsLiveTest {
         assertEquals(result, 0);
     }
 
+    protected void assertWinrmable(WinRmMachineLocation machine) {
+        WinRmToolResponse result = machine.executeCommand(ImmutableList.of("echo mySimpleWinrmCmd"));
+        assertEquals(result.getStatusCode(), 0, "stdout="+result.getStdOut()+"; stderr="+result.getStdErr());
+    }
+
     // Use this utility method to ensure machines are released on tearDown
     protected JcloudsSshMachineLocation obtainMachine(Map<?, ?> conf) throws Exception {
         assertNotNull(jcloudsLocation);
@@ -147,16 +154,24 @@ public class AbstractJcloudsLiveTest {
         return obtainMachine(ImmutableMap.of());
     }
     
-    protected void releaseMachine(JcloudsSshMachineLocation machine) {
+    protected JcloudsWinRmMachineLocation obtainWinrmMachine(Map<?, ?> conf) throws Exception {
+        assertNotNull(jcloudsLocation);
+        JcloudsWinRmMachineLocation result = (JcloudsWinRmMachineLocation)jcloudsLocation.obtain(conf);
+        machines.add(checkNotNull(result, "result"));
+        return result;
+    }
+
+    // Use this utility method to ensure machines are released on tearDown
+    protected void releaseMachine(JcloudsMachineLocation machine) {
         assertNotNull(jcloudsLocation);
         machines.remove(machine);
         jcloudsLocation.release(machine);
     }
     
-    protected List<Exception> releaseMachineSafely(Iterable<? extends JcloudsSshMachineLocation> machines) {
+    protected List<Exception> releaseMachineSafely(Iterable<? extends JcloudsMachineLocation> machines) {
         List<Exception> exceptions = Lists.newArrayList();
         
-        for (JcloudsSshMachineLocation machine : machines) {
+        for (JcloudsMachineLocation machine : machines) {
             try {
                 releaseMachine(machine);
             } catch (Exception e) {
@@ -175,8 +190,8 @@ public class AbstractJcloudsLiveTest {
 
     protected MachineLocation resumeMachine(Map<?, ?> flags) {
         assertNotNull(jcloudsLocation);
-        MachineLocation location = jcloudsLocation.resumeMachine(flags);
-        machines.add((JcloudsSshMachineLocation) location);
+        JcloudsMachineLocation location = jcloudsLocation.resumeMachine(flags);
+        machines.add(location);
         return location;
     }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c9dd478/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsAddressesLiveTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsAddressesLiveTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsAddressesLiveTest.java
index 4ffb57e..12e064d 100644
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsAddressesLiveTest.java
+++ b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsAddressesLiveTest.java
@@ -195,7 +195,7 @@ public class JcloudsAddressesLiveTest extends AbstractJcloudsLiveTest {
     }
 
     @Override
-    protected void releaseMachine(JcloudsSshMachineLocation machine) {
+    protected void releaseMachine(JcloudsMachineLocation machine) {
         jcloudsLocation.release(machine);
     }
     

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c9dd478/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 0b737b9..3744ee9 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
@@ -368,7 +368,7 @@ public class JcloudsLoginLiveTest extends AbstractJcloudsLiveTest {
     }
     
     @Override
-    protected void releaseMachine(JcloudsSshMachineLocation machine) {
+    protected void releaseMachine(JcloudsMachineLocation machine) {
         jcloudsLocation.release(machine);
     }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c9dd478/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsWinrmingLiveTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsWinrmingLiveTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsWinrmingLiveTest.java
new file mode 100644
index 0000000..1c3e63e
--- /dev/null
+++ b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsWinrmingLiveTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.brooklyn.location.jclouds;
+
+import org.apache.brooklyn.util.collections.MutableMap;
+import org.jclouds.compute.domain.OsFamily;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Tests the initial WinRM command execution, for a VM provisioned through jclouds.
+ */
+public class JcloudsWinrmingLiveTest extends AbstractJcloudsLiveTest {
+
+    public static final String AWS_EC2_LOCATION_SPEC = "jclouds:" + AWS_EC2_PROVIDER + ":" + AWS_EC2_EUWEST_REGION_NAME;
+    public static final String AWS_EC2_IMAGE_NAME_REGEX = "Windows_Server-2012-R2_RTM-English-64Bit-Base-.*";
+
+    @Test(groups = {"Live"})
+    public void testCreatesWindowsVm() throws Exception {
+        jcloudsLocation = (JcloudsLocation) managementContext.getLocationRegistry().getLocationManaged(AWS_EC2_LOCATION_SPEC);
+        
+        JcloudsWinRmMachineLocation machine = obtainWinrmMachine(MutableMap.<String,Object>builder()
+                .putIfAbsent("inboundPorts", ImmutableList.of(5986, 5985, 3389))
+                .put(JcloudsLocation.IMAGE_NAME_REGEX.getName(), AWS_EC2_IMAGE_NAME_REGEX)
+                .put(JcloudsLocation.USE_JCLOUDS_SSH_INIT.getName(), false)
+                .put(JcloudsLocation.OS_FAMILY_OVERRIDE.getName(), OsFamily.WINDOWS)
+                .build());
+        assertWinrmable(machine);
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c9dd478/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/SingleMachineProvisioningLocationJcloudsLiveTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/SingleMachineProvisioningLocationJcloudsLiveTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/SingleMachineProvisioningLocationJcloudsLiveTest.java
index 35a0a89..23cb580 100644
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/SingleMachineProvisioningLocationJcloudsLiveTest.java
+++ b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/SingleMachineProvisioningLocationJcloudsLiveTest.java
@@ -105,10 +105,10 @@ private static final Logger log = LoggerFactory.getLogger(SingleMachineProvision
     }
     
     @Override
-    protected void releaseMachine(JcloudsSshMachineLocation machine) {
+    protected void releaseMachine(JcloudsMachineLocation machine) {
         if (location.getChildren().contains(machine)) {
             machines.remove(machine);
-            location.release(machine);
+            location.release((JcloudsSshMachineLocation) machine);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/9c9dd478/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/provider/AwsEc2LocationWindowsLiveTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/provider/AwsEc2LocationWindowsLiveTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/provider/AwsEc2LocationWindowsLiveTest.java
deleted file mode 100644
index b5fdf82..0000000
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/provider/AwsEc2LocationWindowsLiveTest.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.brooklyn.location.jclouds.provider;
-
-import static org.testng.Assert.assertNotNull;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.brooklyn.api.location.NoMachinesAvailableException;
-import org.apache.brooklyn.api.mgmt.ManagementContext;
-import org.apache.brooklyn.core.entity.Entities;
-import org.apache.brooklyn.location.jclouds.JcloudsLocation;
-import org.apache.brooklyn.location.jclouds.JcloudsSshMachineLocation;
-import org.apache.brooklyn.location.ssh.SshMachineLocation;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.BeforeMethod;
-import org.testng.annotations.Test;
-
-import com.google.common.collect.ImmutableMap;
-
-public class AwsEc2LocationWindowsLiveTest {
-    private static final Logger LOG = LoggerFactory.getLogger(AwsEc2LocationWindowsLiveTest.class);
-    
-    private static final String PROVIDER = "aws-ec2";
-    private static final String EUWEST_REGION_NAME = "eu-west-1";
-    private static final String EUWEST_IMAGE_ID = EUWEST_REGION_NAME+"/"+"ami-7f0c260b";//"ami-41d3d635"
-    private static final String LOCATION_ID = "jclouds:"+PROVIDER+":"+EUWEST_REGION_NAME;
-    
-    protected JcloudsLocation loc;
-    protected Collection<SshMachineLocation> machines = new ArrayList<SshMachineLocation>();
-    protected ManagementContext ctx;
-    
-    @BeforeMethod(groups = "Live")
-    public void setUp() {
-        ctx = Entities.newManagementContext(ImmutableMap.of("provider", PROVIDER));
-
-        loc = (JcloudsLocation) ctx.getLocationRegistry().getLocationManaged(LOCATION_ID);
-    }
-
-    @AfterMethod(groups = "Live")
-    public void tearDown() throws Exception {
-        List<Exception> exceptions = new ArrayList<Exception>();
-        for (SshMachineLocation machine : machines) {
-            try {
-                loc.release(machine);
-            } catch (Exception e) {
-                LOG.warn("Error releasing machine $it; continuing...", e);
-                exceptions.add(e);
-            }
-        }
-        if (!exceptions.isEmpty()) {
-            throw exceptions.get(0);
-        }
-        machines.clear();
-    }
-    
-    // TODO Note careful choice of image due to jclouds 1.4 issue 886
-    // TODO Blocks for long time, waiting for IP:22 to be reachable, before falling back to using public IP
-    //      10*2 minutes per attempt in jclouds 1.4 because done sequentially, and done twice by us so test takes 40 minutes!
-    @Test(enabled=true, groups = "Live")
-    public void testProvisionWindowsVm() throws NoMachinesAvailableException {
-        JcloudsSshMachineLocation machine = (JcloudsSshMachineLocation) obtainMachine(ImmutableMap.of("imageId", EUWEST_IMAGE_ID));
-
-        LOG.info("Provisioned Windows VM {}; checking if has password", machine);
-        assertNotNull(machine.waitForPassword());
-    }
-    
-    // Use this utility method to ensure machines are released on tearDown
-    protected SshMachineLocation obtainMachine(Map<?, ?> flags) throws NoMachinesAvailableException {
-        JcloudsSshMachineLocation result = (JcloudsSshMachineLocation) loc.obtain(flags);
-        machines.add(result);
-        return result;
-    }
-}