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/04/06 11:42:38 UTC

[1/6] brooklyn-server git commit: BROOKLYN-249: JcloudsLocation resolves external config

Repository: brooklyn-server
Updated Branches:
  refs/heads/master 17c21d05c -> cce75a105


BROOKLYN-249: JcloudsLocation resolves external config

Previously we were just resolving it for identity/credential in 
getComputeService(). Now we resolve it for everything that obtain()
does.

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

Branch: refs/heads/master
Commit: 99dd3ba4fc507988bdecd4e57c2b36cbb465b2a5
Parents: 3896826
Author: Aled Sage <al...@gmail.com>
Authored: Tue Apr 5 17:58:31 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Tue Apr 5 17:58:31 2016 +0100

----------------------------------------------------------------------
 .../apache/brooklyn/util/core/config/ResolvingConfigBag.java   | 6 ++++++
 .../org/apache/brooklyn/location/jclouds/JcloudsLocation.java  | 4 +++-
 2 files changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/99dd3ba4/core/src/main/java/org/apache/brooklyn/util/core/config/ResolvingConfigBag.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/util/core/config/ResolvingConfigBag.java b/core/src/main/java/org/apache/brooklyn/util/core/config/ResolvingConfigBag.java
index 5c6c3ad..337c7c4 100644
--- a/core/src/main/java/org/apache/brooklyn/util/core/config/ResolvingConfigBag.java
+++ b/core/src/main/java/org/apache/brooklyn/util/core/config/ResolvingConfigBag.java
@@ -75,6 +75,7 @@ public class ResolvingConfigBag extends ConfigBag {
         }
         return transformer;
     }
+    
     @Override
     public void markUsed(String key) {
         super.markUsed(key);
@@ -82,6 +83,11 @@ public class ResolvingConfigBag extends ConfigBag {
             parentBag.markUsed(key);
     }
     
+    @SuppressWarnings("unchecked")
+    protected <T> T get(ConfigKey<T> key, boolean markUsed) {
+        return (T) getTransformer().apply(super.get(key, markUsed));
+    }
+
     @Override
     public Map<String,Object> getAllConfig() {
         // Lazily transform copy of map

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/99dd3ba4/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 5544692..14c5cfb 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
@@ -603,7 +603,9 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
      * (for initial login, and a user potentially to create for subsequent ie normal access) */
     @Override
     public MachineLocation obtain(Map<?,?> flags) throws NoMachinesAvailableException {
-        ConfigBag setup = ConfigBag.newInstanceExtending(config().getBag(), flags);
+        ConfigBag setupRaw = ConfigBag.newInstanceExtending(config().getBag(), flags);
+        ConfigBag setup = ResolvingConfigBag.newInstanceExtending(getManagementContext(), setupRaw);
+
         Integer attempts = setup.get(MACHINE_CREATE_ATTEMPTS);
         List<Exception> exceptions = Lists.newArrayList();
         if (attempts == null || attempts < 1) attempts = 1;


[2/6] brooklyn-server git commit: Trim publicKeyData

Posted by sv...@apache.org.
Trim publicKeyData

Otherwise get incorrect warning about the publicKey not matching
the privateKey.

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

Branch: refs/heads/master
Commit: ee3159bf4e94a1ef9a32c8c5c46a790be2c56146
Parents: 99dd3ba
Author: Aled Sage <al...@gmail.com>
Authored: Tue Apr 5 17:59:03 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Tue Apr 5 17:59:03 2016 +0100

----------------------------------------------------------------------
 .../org/apache/brooklyn/core/location/LocationConfigUtils.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/ee3159bf/core/src/main/java/org/apache/brooklyn/core/location/LocationConfigUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/location/LocationConfigUtils.java b/core/src/main/java/org/apache/brooklyn/core/location/LocationConfigUtils.java
index 40ecd72..bc2eb49 100644
--- a/core/src/main/java/org/apache/brooklyn/core/location/LocationConfigUtils.java
+++ b/core/src/main/java/org/apache/brooklyn/core/location/LocationConfigUtils.java
@@ -405,8 +405,8 @@ public class LocationConfigUtils {
     private static String getKeyDataFromDataKeyOrFileKey(ConfigBag config, ConfigKey<String> dataKey, ConfigKey<String> fileKey) {
         boolean unused = config.isUnused(dataKey);
         String data = config.get(dataKey);
-        if (groovyTruth(data) && !unused) {
-            return data;
+        if (Strings.isNonBlank(data) && !unused) {
+            return data.trim();
         }
         
         String file = config.get(fileKey);


[3/6] brooklyn-server git commit: BROOKLYN-249: fix location rebind for external config

Posted by sv...@apache.org.
BROOKLYN-249: fix location rebind for external config

Fixes it only for ConfigKeys, rather than for fields that use
`@SetFromFlag`. The latter can be addressed in a separate pull request.

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

Branch: refs/heads/master
Commit: b35ec4aabbcc2fe2dc07f1b1cc23a0f92d3d9155
Parents: ee3159b
Author: Aled Sage <al...@gmail.com>
Authored: Tue Apr 5 18:00:21 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Tue Apr 5 18:00:21 2016 +0100

----------------------------------------------------------------------
 .../mgmt/rebind/BasicLocationRebindSupport.java     | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/b35ec4aa/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/BasicLocationRebindSupport.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/BasicLocationRebindSupport.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/BasicLocationRebindSupport.java
index b789501..0f10672 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/BasicLocationRebindSupport.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/BasicLocationRebindSupport.java
@@ -29,11 +29,11 @@ import org.apache.brooklyn.api.mgmt.rebind.mementos.LocationMemento;
 import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.core.location.AbstractLocation;
 import org.apache.brooklyn.core.mgmt.rebind.dto.MementosGenerators;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.brooklyn.util.collections.MutableMap;
 import org.apache.brooklyn.util.core.flags.FlagUtils;
 import org.apache.brooklyn.util.core.flags.TypeCoercions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Sets;
 
@@ -64,6 +64,7 @@ public class BasicLocationRebindSupport extends AbstractBrooklynObjectRebindSupp
         return memento;
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     protected void addConfig(RebindContext rebindContext, LocationMemento memento) {
         // FIXME Treat config like we do for entities; this code will disappear when locations become entities.
@@ -89,14 +90,15 @@ public class BasicLocationRebindSupport extends AbstractBrooklynObjectRebindSupp
                 // And use magic of setFieldFromFlag's magic to either set config or field as appropriate.
                 if (ConfigKey.class.isAssignableFrom(fieldType)) {
                     ConfigKey<?> configKey = (ConfigKey<?>) FlagUtils.getField(location, field);
-                    value = TypeCoercions.coerce(entry.getValue(), configKey.getTypeToken());
+                    location.config().set((ConfigKey<Object>)configKey, entry.getValue());
                 } else {
                     value = TypeCoercions.coerce(entry.getValue(), fieldType);
+                    if (value != null) {
+                        location.config().addToLocalBag(MutableMap.of(flagName, value));
+                        FlagUtils.setFieldFromFlag(location, flagName, value);
+                    }
                 }
-                if (value != null) {
-                    location.config().addToLocalBag(MutableMap.of(flagName, value));
-                    FlagUtils.setFieldFromFlag(location, flagName, value);
-                }
+                
             } catch (NoSuchElementException e) {
                 // FIXME How to do findFieldForFlag without throwing exception if it's not there?
             }


[4/6] brooklyn-server git commit: BROOKLYN-249: test JcloudsLocation external config

Posted by sv...@apache.org.
BROOKLYN-249: test JcloudsLocation external config


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

Branch: refs/heads/master
Commit: bd2e812468b9c04fbc0acaf2ee2246dbd26c90e5
Parents: b35ec4a
Author: Aled Sage <al...@gmail.com>
Authored: Tue Apr 5 18:05:43 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Tue Apr 5 18:35:45 2016 +0100

----------------------------------------------------------------------
 camp/camp-brooklyn/pom.xml                      | 13 ++++
 .../core/mgmt/rebind/RebindTestFixture.java     | 11 ++++
 .../byon/ByonLocationResolverRebindTest.java    | 69 ++++++++++++++++++++
 .../jclouds/AbstractJcloudsLiveTest.java        |  2 +-
 .../location/jclouds/JcloudsRebindStubTest.java | 34 +++++++---
 5 files changed, 118 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bd2e8124/camp/camp-brooklyn/pom.xml
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/pom.xml b/camp/camp-brooklyn/pom.xml
index 5e2f3bf..ac12285 100644
--- a/camp/camp-brooklyn/pom.xml
+++ b/camp/camp-brooklyn/pom.xml
@@ -140,6 +140,19 @@
         </dependency>
         <dependency>
             <groupId>org.apache.brooklyn</groupId>
+            <artifactId>brooklyn-locations-jclouds</artifactId>
+            <version>${project.version}</version>
+            <classifier>tests</classifier>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <!-- jsr311 excluded from jclouds; see jclouds module's pom for further comments. -->
+            <groupId>javax.ws.rs</groupId>
+            <artifactId>javax.ws.rs-api</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.brooklyn</groupId>
             <artifactId>brooklyn-test-support</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bd2e8124/core/src/test/java/org/apache/brooklyn/core/mgmt/rebind/RebindTestFixture.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/brooklyn/core/mgmt/rebind/RebindTestFixture.java b/core/src/test/java/org/apache/brooklyn/core/mgmt/rebind/RebindTestFixture.java
index f43042f..8bbd04d 100644
--- a/core/src/test/java/org/apache/brooklyn/core/mgmt/rebind/RebindTestFixture.java
+++ b/core/src/test/java/org/apache/brooklyn/core/mgmt/rebind/RebindTestFixture.java
@@ -39,6 +39,7 @@ import org.apache.brooklyn.core.entity.Entities;
 import org.apache.brooklyn.core.entity.EntityFunctions;
 import org.apache.brooklyn.core.entity.StartableApplication;
 import org.apache.brooklyn.core.entity.trait.Startable;
+import org.apache.brooklyn.core.internal.BrooklynProperties;
 import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
 import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
 import org.apache.brooklyn.core.mgmt.persist.BrooklynMementoPersisterToObjectStore;
@@ -86,12 +87,21 @@ public abstract class RebindTestFixture<T extends StartableApplication> {
         LOG.info("Test "+getClass()+" persisting to "+mementoDir);
     }
 
+    protected BrooklynProperties createBrooklynProperties() {
+        if (useLiveManagementContext()) {
+            return BrooklynProperties.Factory.newDefault();
+        } else {
+            return BrooklynProperties.Factory.newEmpty();
+        }
+    }
+
     /** @return A started management context */
     protected LocalManagementContext createOrigManagementContext() {
         return RebindTestUtils.managementContextBuilder(mementoDir, classLoader)
                 .persistPeriodMillis(getPersistPeriodMillis())
                 .forLive(useLiveManagementContext())
                 .emptyCatalog(useEmptyCatalog())
+                .properties(createBrooklynProperties())
                 .buildStarted();
     }
 
@@ -106,6 +116,7 @@ public abstract class RebindTestFixture<T extends StartableApplication> {
         return RebindTestUtils.managementContextBuilder(mementoDir, classLoader)
                 .forLive(useLiveManagementContext())
                 .emptyCatalog(useEmptyCatalog())
+                .properties(createBrooklynProperties())
                 .buildUnstarted();
     }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bd2e8124/core/src/test/java/org/apache/brooklyn/location/byon/ByonLocationResolverRebindTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/brooklyn/location/byon/ByonLocationResolverRebindTest.java b/core/src/test/java/org/apache/brooklyn/location/byon/ByonLocationResolverRebindTest.java
new file mode 100644
index 0000000..9f3b3a3
--- /dev/null
+++ b/core/src/test/java/org/apache/brooklyn/location/byon/ByonLocationResolverRebindTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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.byon;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+import org.apache.brooklyn.api.location.LocationSpec;
+import org.apache.brooklyn.api.location.MachineLocation;
+import org.apache.brooklyn.api.location.MachineProvisioningLocation;
+import org.apache.brooklyn.core.mgmt.rebind.RebindTestFixtureWithApp;
+import org.apache.brooklyn.location.ssh.SshMachineLocation;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+public class ByonLocationResolverRebindTest extends RebindTestFixtureWithApp {
+
+    @Test
+    public void testRebindByon() throws Exception {
+        String spec = "byon(hosts=\"1.1.1.1\")";
+        MachineProvisioningLocation<MachineLocation> provisioner = resolve(spec);
+        
+        rebind();
+        
+        @SuppressWarnings("unchecked")
+        MachineProvisioningLocation<MachineLocation> newProvisioner = (MachineProvisioningLocation<MachineLocation>) mgmt().getLocationManager().getLocation(provisioner.getId());
+        MachineLocation newLocation = newProvisioner.obtain(ImmutableMap.of());
+        assertTrue(newLocation instanceof SshMachineLocation, "Expected location to be SshMachineLocation, found " + newLocation);
+    }
+
+    @Test
+    public void testRebindWhenOnlyByonLocationSpec() throws Exception {
+        int before = mgmt().getLocationManager().getLocations().size();
+        String spec = "byon(hosts=\"1.1.1.1\")";
+        getLocationSpec(spec);
+        
+        rebind();
+
+        int after = mgmt().getLocationManager().getLocations().size();
+        assertEquals(after, before);
+    }
+
+    @SuppressWarnings("unchecked")
+    private LocationSpec<FixedListMachineProvisioningLocation<MachineLocation>> getLocationSpec(String val) {
+        return (LocationSpec<FixedListMachineProvisioningLocation<MachineLocation>>) mgmt().getLocationRegistry().getLocationSpec(val).get();
+    }
+
+    @SuppressWarnings("unchecked")
+    private FixedListMachineProvisioningLocation<MachineLocation> resolve(String val) {
+        return (FixedListMachineProvisioningLocation<MachineLocation>) mgmt().getLocationRegistry().getLocationManaged(val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bd2e8124/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 7723a01..4313c43 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
@@ -112,7 +112,7 @@ public class AbstractJcloudsLiveTest {
         return LocalManagementContextForTests.builder(true).useDefaultProperties().build();
     }
     
-    protected static void stripBrooklynProperties(BrooklynProperties props) {
+    public static void stripBrooklynProperties(BrooklynProperties props) {
         // remove all location properties except for identity and credential
         // (so key, scripts, etc settings don't interfere with tests) 
         for (String key : ImmutableSet.copyOf(props.asMapWithStringKeys().keySet())) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bd2e8124/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsRebindStubTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsRebindStubTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsRebindStubTest.java
index 1d33bc5..b89e1b2 100644
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsRebindStubTest.java
+++ b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/JcloudsRebindStubTest.java
@@ -74,10 +74,15 @@ import com.google.common.collect.Multimaps;
 public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
 
     // TODO Duplication of AbstractJcloudsLiveTest, because we're subclassing RebindTestFixture instead.
+    
+    // TODO The ByonComputeServiceRegistry extends ComputeServiceRegistryImpl, which means when it  
+    // is serialized it will try to serialize the cachedComputeServices. That will try to serialize 
+    // threads and all sorts!
 
     private static final Logger LOG = LoggerFactory.getLogger(JcloudsRebindStubTest.class);
 
-    public static final String SOFTLAYER_LOCATION_SPEC = "jclouds:" + AbstractJcloudsLiveTest.SOFTLAYER_PROVIDER;
+    public static final String PROVIDER = AbstractJcloudsLiveTest.SOFTLAYER_PROVIDER;
+    public static final String SOFTLAYER_LOCATION_SPEC = "jclouds:" + PROVIDER;
     public static final String SOFTLAYER_IMAGE_ID = "UBUNTU_14_64";
     
     protected List<ManagementContext> mgmts;
@@ -89,10 +94,6 @@ public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
         super.setUp();
         mgmts = Lists.newCopyOnWriteArrayList(ImmutableList.<ManagementContext>of(origManagementContext));
         machines = Multimaps.synchronizedMultimap(ArrayListMultimap.<ManagementContext, JcloudsSshMachineLocation>create());
-        
-        // Don't let any defaults from brooklyn.properties (except credentials) interfere with test
-        brooklynProperties = origManagementContext.getBrooklynProperties();
-        AbstractJcloudsLiveTest.stripBrooklynProperties(brooklynProperties);
     }
 
     @AfterMethod(alwaysRun=true)
@@ -120,6 +121,14 @@ public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
     }
 
     @Override
+    protected BrooklynProperties createBrooklynProperties() {
+        // Don't let any defaults from brooklyn.properties (except credentials) interfere with test
+        BrooklynProperties result = super.createBrooklynProperties();
+        AbstractJcloudsLiveTest.stripBrooklynProperties(result);
+        return result;
+    }
+    
+    @Override
     protected boolean useLiveManagementContext() {
         return true;
     }
@@ -130,7 +139,7 @@ public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
         mgmts.add(newManagementContext);
         return result;
     }
-    
+
     @Test(groups={"Live", "Live-sanity"})
     public void testRebind() throws Exception {
         LocationImpl locImpl = new LocationImpl(
@@ -181,10 +190,8 @@ public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
                 "myHostname");
         
         ByonComputeServiceRegistry computeServiceRegistry = new ByonComputeServiceRegistry(node);
-        JcloudsLocation origJcloudsLoc = (JcloudsLocation) mgmt().getLocationRegistry().getLocationManaged("jclouds:softlayer", ImmutableMap.of(
-                JcloudsLocation.COMPUTE_SERVICE_REGISTRY, computeServiceRegistry,
-                JcloudsLocation.WAIT_FOR_SSHABLE, false,
-                JcloudsLocation.USE_JCLOUDS_SSH_INIT, false));
+
+        JcloudsLocation origJcloudsLoc = newJcloudsLocation(computeServiceRegistry);
     
         JcloudsSshMachineLocation origMachine = (JcloudsSshMachineLocation) origJcloudsLoc.obtain(ImmutableMap.of("imageId", SOFTLAYER_IMAGE_ID));
         
@@ -211,6 +218,13 @@ public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
         assertEquals(newJcloudsLoc.getProvider(), origJcloudsLoc.getProvider());
     }
     
+    protected JcloudsLocation newJcloudsLocation(ComputeServiceRegistry computeServiceRegistry) throws Exception {
+        return (JcloudsLocation) mgmt().getLocationRegistry().getLocationManaged("jclouds:softlayer", ImmutableMap.of(
+                JcloudsLocation.COMPUTE_SERVICE_REGISTRY, computeServiceRegistry, 
+                JcloudsLocation.WAIT_FOR_SSHABLE, false,
+                JcloudsLocation.USE_JCLOUDS_SSH_INIT, false));
+    }
+    
     protected static class ByonComputeServiceRegistry extends ComputeServiceRegistryImpl implements ComputeServiceRegistry {
         private final NodeMetadata node;
 


[6/6] brooklyn-server git commit: Closes #103

Posted by sv...@apache.org.
Closes #103

Fix BROOKLYN-249

See more detailed descriptions in each commit message.


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

Branch: refs/heads/master
Commit: cce75a105aa850044f448a4ba79772a84d1d2d7f
Parents: 17c21d0 48cfa90
Author: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Authored: Wed Apr 6 12:42:20 2016 +0300
Committer: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Committed: Wed Apr 6 12:42:20 2016 +0300

----------------------------------------------------------------------
 camp/camp-brooklyn/pom.xml                      |  13 ++
 .../JcloudsRebindWithExternalConfigTest.java    | 139 +++++++++++++++++++
 .../core/location/LocationConfigUtils.java      |   4 +-
 .../mgmt/rebind/BasicLocationRebindSupport.java |  16 ++-
 .../util/core/config/ResolvingConfigBag.java    |   6 +
 .../core/mgmt/rebind/RebindTestFixture.java     |  11 ++
 .../byon/ByonLocationResolverRebindTest.java    |  69 +++++++++
 .../location/jclouds/JcloudsLocation.java       |   4 +-
 .../jclouds/AbstractJcloudsLiveTest.java        |   2 +-
 .../location/jclouds/JcloudsRebindStubTest.java |  34 +++--
 10 files changed, 277 insertions(+), 21 deletions(-)
----------------------------------------------------------------------



[5/6] brooklyn-server git commit: BROOKLYN-249: Adds JcloudsRebindWithExternalConfigTest

Posted by sv...@apache.org.
BROOKLYN-249: Adds JcloudsRebindWithExternalConfigTest


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

Branch: refs/heads/master
Commit: 48cfa908a0001a3ab66a21febc7404ee34eb652d
Parents: bd2e812
Author: Aled Sage <al...@gmail.com>
Authored: Wed Apr 6 10:33:21 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Wed Apr 6 10:40:53 2016 +0100

----------------------------------------------------------------------
 .../JcloudsRebindWithExternalConfigTest.java    | 139 +++++++++++++++++++
 1 file changed, 139 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/48cfa908/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithExternalConfigTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithExternalConfigTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithExternalConfigTest.java
new file mode 100644
index 0000000..6d0d4ec
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithExternalConfigTest.java
@@ -0,0 +1,139 @@
+package org.apache.brooklyn.camp.brooklyn;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.File;
+import java.util.Map;
+
+import org.apache.brooklyn.api.entity.Application;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.entity.EntitySpec;
+import org.apache.brooklyn.camp.brooklyn.spi.creation.CampTypePlanTransformer;
+import org.apache.brooklyn.core.config.external.InPlaceExternalConfigSupplier;
+import org.apache.brooklyn.core.entity.trait.Startable;
+import org.apache.brooklyn.core.internal.BrooklynProperties;
+import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
+import org.apache.brooklyn.core.typereg.RegisteredTypeLoadingContexts;
+import org.apache.brooklyn.location.jclouds.ComputeServiceRegistry;
+import org.apache.brooklyn.location.jclouds.JcloudsLocation;
+import org.apache.brooklyn.location.jclouds.JcloudsPropertiesFromBrooklynProperties;
+import org.apache.brooklyn.location.jclouds.JcloudsRebindStubTest;
+import org.apache.brooklyn.util.collections.MutableMap;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+
+/**
+ * Implementation notes. This relies on the test {@link JcloudsRebindStubTest#testRebind()}.
+ * It changes the setup for the test in the following ways:
+ * <ul>
+ *   <li>Brooklyn properties defines external config
+ *   <li>Location is defined in YAML, and refers to the external config for the identity/credential.
+ *   <li>When creating management context, it also creates {@link BrooklynCampPlatformLauncherNoServer}.
+ *   <li>It uses {@link JcloudsRebindWithExternalConfigTest#ByonComputeServiceStaticRef} to allow
+ *       the test's {@link ComputeServiceRegistry} to be wired up via YAML.
+ * </ul>
+ * 
+ * See {@link JcloudsRebindStubTest} for explanation why this is "Live" - it will not create VMs,
+ * but does retrieve list of images etc.
+ */
+@Test(groups={"Live", "Live-sanity"})
+public class JcloudsRebindWithExternalConfigTest extends JcloudsRebindStubTest {
+
+    private BrooklynCampPlatformLauncherNoServer origLauncher;
+    private BrooklynCampPlatformLauncherNoServer newLauncher;
+
+    @Override
+    @AfterMethod(alwaysRun=true)
+    public void tearDown() throws Exception {
+        try {
+            super.tearDown();
+        } finally {
+            ByonComputeServiceStaticRef.clearInstance();
+            if (origLauncher != null) origLauncher.stopServers();
+            if (newLauncher != null) newLauncher.stopServers();
+        }
+    }
+    
+    @Override
+    protected BrooklynProperties createBrooklynProperties() {
+        BrooklynProperties result = super.createBrooklynProperties();
+        
+        Map<Object,Object> jcloudsProps = MutableMap.<Object,Object>copyOf(new JcloudsPropertiesFromBrooklynProperties().getJcloudsProperties(PROVIDER, null, "testname", result.asMapWithStringKeys()));
+        String identity = checkNotNull((String)jcloudsProps.get("identity"), "identity");
+        String credential = checkNotNull((String)jcloudsProps.get("credential"), "credential");
+        
+        result.put("brooklyn.external.creds", InPlaceExternalConfigSupplier.class.getName());
+        result.put("brooklyn.external.creds.test-identity", identity);
+        result.put("brooklyn.external.creds.test-credential", credential);
+        
+        return result;
+    }
+    
+    @Override
+    protected LocalManagementContext createOrigManagementContext() {
+        origLauncher = new BrooklynCampPlatformLauncherNoServer() {
+            @Override
+            protected LocalManagementContext newMgmtContext() {
+                return JcloudsRebindWithExternalConfigTest.super.createOrigManagementContext();
+            }
+        };
+        origLauncher.launch();
+        LocalManagementContext mgmt = (LocalManagementContext) origLauncher.getBrooklynMgmt();
+        return mgmt;
+    }
+
+    @Override
+    protected LocalManagementContext createNewManagementContext(final File mementoDir) {
+        newLauncher = new BrooklynCampPlatformLauncherNoServer() {
+            @Override
+            protected LocalManagementContext newMgmtContext() {
+                return JcloudsRebindWithExternalConfigTest.super.createNewManagementContext(mementoDir);
+            }
+        };
+        newLauncher.launch();
+        return (LocalManagementContext) newLauncher.getBrooklynMgmt();
+    }
+    
+    @Override
+    protected JcloudsLocation newJcloudsLocation(ComputeServiceRegistry computeServiceRegistry) throws Exception {
+        ByonComputeServiceStaticRef.setInstance(computeServiceRegistry);
+        
+        String yaml = Joiner.on("\n").join(
+                "location:",
+                "  jclouds:softlayer:",
+                "    identity: $brooklyn:external(\"creds\", \"test-identity\")",
+                "    credential: $brooklyn:external(\"creds\", \"test-credential\")",
+                "    jclouds.computeServiceRegistry:",
+                "      $brooklyn:object:",
+                "        type: "+ByonComputeServiceStaticRef.class.getName(),
+                "    waitForSshable: false",
+                "    useJcloudsSshInit: false",
+                "services:\n"+
+                "- type: org.apache.brooklyn.entity.stock.BasicApplication");
+        
+        EntitySpec<?> spec = 
+                mgmt().getTypeRegistry().createSpecFromPlan(CampTypePlanTransformer.FORMAT, yaml, RegisteredTypeLoadingContexts.spec(Application.class), EntitySpec.class);
+        final Entity app = mgmt().getEntityManager().createEntity(spec);
+        app.invoke(Startable.START, ImmutableMap.<String, Object>of()).get();
+
+        return (JcloudsLocation) Iterables.getOnlyElement(app.getLocations());
+    }
+
+    public static class ByonComputeServiceStaticRef {
+        private static volatile ComputeServiceRegistry instance;
+
+        public ComputeServiceRegistry asComputeServiceRegistry() {
+            return checkNotNull(instance, "instance");
+        }
+        static void setInstance(ComputeServiceRegistry val) {
+            instance = val;
+        }
+        static void clearInstance() {
+            instance = null;
+        }
+    }
+}