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 2016/09/30 13:01:35 UTC

[1/5] brooklyn-server git commit: BROOKLYN—349: fix DSL resolution in location

Repository: brooklyn-server
Updated Branches:
  refs/heads/master 211694966 -> 438e74648


BROOKLYN\u2014349: fix DSL resolution in location

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

Branch: refs/heads/master
Commit: 5494ac49fb3c7f0bcfa3afcb8a28e2cb61a5f66e
Parents: 2116949
Author: Aled Sage <al...@gmail.com>
Authored: Wed Sep 21 17:58:21 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Sep 30 12:34:34 2016 +0100

----------------------------------------------------------------------
 .../brooklyn/spi/dsl/methods/DslComponent.java  |  79 +++++----
 .../brooklyn/JcloudsRebindWithYamlDslTest.java  | 171 +++++++++++++++++++
 .../brooklyn/core/mgmt/BrooklynTaskTags.java    |  31 +++-
 .../brooklyn/util/core/config/ConfigBag.java    |   9 +
 .../util/core/config/ResolvingConfigBag.java    |  28 ++-
 .../brooklyn/util/core/task/TaskBuilder.java    |   6 +
 .../brooklyn/util/core/task/ValueResolver.java  |   7 +-
 .../core/internal/ssh/RecordingSshTool.java     |   4 +
 8 files changed, 296 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5494ac49/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/DslComponent.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/DslComponent.java b/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/DslComponent.java
index a29ce25..0d3321b 100644
--- a/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/DslComponent.java
+++ b/camp/camp-brooklyn/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/dsl/methods/DslComponent.java
@@ -89,8 +89,12 @@ public class DslComponent extends BrooklynDslDeferredSupplier<Entity> {
     
     @Override
     public Task<Entity> newTask() {
-        return TaskBuilder.<Entity>builder().displayName(toString()).tag(BrooklynTaskTags.TRANSIENT_TASK_TAG)
-            .body(new EntityInScopeFinder(scopeComponent, scope, componentId)).build();
+        return TaskBuilder.<Entity>builder()
+                .displayName(toString())
+                .tag(BrooklynTaskTags.TRANSIENT_TASK_TAG)
+                .tagIfNotNull(BrooklynTaskTags.getTargetOrContextEntityTag(Tasks.current()))
+                .body(new EntityInScopeFinder(scopeComponent, scope, componentId))
+                .build();
     }
     
     protected static class EntityInScopeFinder implements Callable<Entity> {
@@ -300,13 +304,18 @@ public class DslComponent extends BrooklynDslDeferredSupplier<Entity> {
 
         @Override
         public Task<Object> newTask() {
-            return Tasks.builder().displayName("retrieving config for "+keyName).tag(BrooklynTaskTags.TRANSIENT_TASK_TAG).dynamic(false).body(new Callable<Object>() {
-                @Override
-                public Object call() throws Exception {
-                    Entity targetEntity = component.get();
-                    return targetEntity.getConfig(ConfigKeys.newConfigKey(Object.class, keyName));
-                }
-            }).build();
+            return Tasks.builder()
+                    .displayName("retrieving config for "+keyName)
+                    .tag(BrooklynTaskTags.TRANSIENT_TASK_TAG)
+                    .tagIfNotNull(BrooklynTaskTags.getTargetOrContextEntityTag(Tasks.current()))
+                    .dynamic(false)
+                    .body(new Callable<Object>() {
+                        @Override
+                        public Object call() throws Exception {
+                            Entity targetEntity = component.get();
+                            return targetEntity.getConfig(ConfigKeys.newConfigKey(Object.class, keyName));
+                        }})
+                    .build();
         }
 
         @Override
@@ -345,31 +354,35 @@ public class DslComponent extends BrooklynDslDeferredSupplier<Entity> {
 
         @Override
         public Task<Sensor<?>> newTask() {
-            return Tasks.<Sensor<?>>builder().displayName("looking up sensor for "+sensorName).dynamic(false).body(new Callable<Sensor<?>>() {
-                @Override
-                public Sensor<?> call() throws Exception {
-                    return resolve(sensorName, false);
-                }
-                
-                public Sensor<?> resolve(Object si, boolean resolved) throws ExecutionException, InterruptedException {
-                    if (si instanceof Sensor) return (Sensor<?>)si;
-                    if (si instanceof String) {
-                        Entity targetEntity = component.get();
-                        Sensor<?> result = null;
-                        if (targetEntity!=null) {
-                            result = targetEntity.getEntityType().getSensor((String)si);
+            return Tasks.<Sensor<?>>builder()
+                    .displayName("looking up sensor for "+sensorName)
+                    .dynamic(false)
+                    .tagIfNotNull(BrooklynTaskTags.getTargetOrContextEntityTag(Tasks.current()))
+                    .body(new Callable<Sensor<?>>() {
+                        @Override
+                        public Sensor<?> call() throws Exception {
+                            return resolve(sensorName, false);
                         }
-                        if (result!=null) return result;
-                        return Sensors.newSensor(Object.class, (String)si);
-                    }
-                    if (!resolved) {
-                        // attempt to resolve, and recurse
-                        final ExecutionContext executionContext = ((EntityInternal)entity()).getExecutionContext();
-                        return resolve(Tasks.resolveDeepValue(si, Object.class, executionContext), true);
-                    }
-                    throw new IllegalStateException("Cannot resolve '"+sensorName+"' as a sensor");
-                }
-            }).build();
+                        
+                        public Sensor<?> resolve(Object si, boolean resolved) throws ExecutionException, InterruptedException {
+                            if (si instanceof Sensor) return (Sensor<?>)si;
+                            if (si instanceof String) {
+                                Entity targetEntity = component.get();
+                                Sensor<?> result = null;
+                                if (targetEntity!=null) {
+                                    result = targetEntity.getEntityType().getSensor((String)si);
+                                }
+                                if (result!=null) return result;
+                                return Sensors.newSensor(Object.class, (String)si);
+                            }
+                            if (!resolved) {
+                                // attempt to resolve, and recurse
+                                final ExecutionContext executionContext = ((EntityInternal)entity()).getExecutionContext();
+                                return resolve(Tasks.resolveDeepValue(si, Object.class, executionContext), true);
+                            }
+                            throw new IllegalStateException("Cannot resolve '"+sensorName+"' as a sensor");
+                        }})
+                    .build();
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5494ac49/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithYamlDslTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithYamlDslTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithYamlDslTest.java
new file mode 100644
index 0000000..ce91156
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithYamlDslTest.java
@@ -0,0 +1,171 @@
+/*
+ * 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.camp.brooklyn;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.testng.Assert.assertEquals;
+
+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.entity.trait.Startable;
+import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
+import org.apache.brooklyn.core.typereg.RegisteredTypeLoadingContexts;
+import org.apache.brooklyn.entity.machine.MachineEntity;
+import org.apache.brooklyn.location.jclouds.ComputeServiceRegistry;
+import org.apache.brooklyn.location.jclouds.JcloudsLocation;
+import org.apache.brooklyn.location.jclouds.JcloudsRebindStubTest;
+import org.apache.brooklyn.location.ssh.SshMachineLocation;
+import org.apache.brooklyn.util.core.internal.ssh.RecordingSshTool;
+import org.apache.brooklyn.util.core.internal.ssh.RecordingSshTool.ExecCmd;
+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;
+
+/**
+ * This is primarily to test https://issues.apache.org/jira/browse/BROOKLYN-349
+ * 
+ * As per the other {@link JcloudsRebindStubTest} tests, it will connect to SoftLayer to retrieve
+ * image details (so needs real credentials), but it will then stub out the VM creation.
+ * 
+ * It could do with some cleanup at some point:
+ * <ul>
+ *   <li>There is an NPE at AddMachineMetrics$2.apply(AddMachineMetrics.java:111), despite having set
+ *       "metrics.usage.retrieve: false". Perhaps even with that set it will poll once before being disabled?!
+ *   <li>When "stopping" the dummy machine, it fails when executing commands against the real SoftLayer:
+ *       <pre>
+ *       2016-09-21 17:49:22,776 ERROR Cannot retry after server error, command has exceeded retry limit 5: [method=org.jclouds.softlayer.features.VirtualGuestApi.public abstract org.jclouds.softlayer.domain.VirtualGuest org.jclouds.softlayer.features.VirtualGuestApi.getVirtualGuest(long)[123], request=GET https://api.softlayer.com/rest/v3/SoftLayer_Virtual_Guest/123/getObject?objectMask=id%3Bhostname%3Bdomain%3BfullyQualifiedDomainName%3BpowerState%3BmaxCpu%3BmaxMemory%3BstatusId%3BoperatingSystem.passwords%3BprimaryBackendIpAddress%3BprimaryIpAddress%3BactiveTransactionCount%3BblockDevices.diskImage%3Bdatacenter%3BtagReferences%3BprivateNetworkOnlyFlag%3BsshKeys HTTP/1.1]
+ *       </pre>
+ *       Presumably we need to stub out that call as well somehow!
+ * </ul>
+ */
+@Test(groups={"Live", "Live-sanity"})
+public class JcloudsRebindWithYamlDslTest 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 LocalManagementContext createOrigManagementContext() {
+        origLauncher = new BrooklynCampPlatformLauncherNoServer() {
+            @Override
+            protected LocalManagementContext newMgmtContext() {
+                return JcloudsRebindWithYamlDslTest.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 JcloudsRebindWithYamlDslTest.super.createNewManagementContext(mementoDir);
+            }
+        };
+        newLauncher.launch();
+        return (LocalManagementContext) newLauncher.getBrooklynMgmt();
+    }
+    
+    @Override
+    protected JcloudsLocation newJcloudsLocation(ComputeServiceRegistry computeServiceRegistry) throws Exception {
+        ByonComputeServiceStaticRef.setInstance(computeServiceRegistry);
+        
+        String symbolicName = "my.catalog.app.id.load";
+        String catalogYaml = Joiner.on("\n").join(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  version: \"0.1.2\"",
+            "  itemType: entity",
+            "  item:",
+            "    brooklyn.parameters:",
+            "    - name: password",
+            "      default: myYamlPassword",
+            "    type: "+ MachineEntity.class.getName());
+        mgmt().getCatalog().addItems(catalogYaml, true);
+
+        String yaml = Joiner.on("\n").join(
+                "location:",
+                "  jclouds:softlayer:",
+                "    jclouds.computeServiceRegistry:",
+                "      $brooklyn:object:",
+                "        type: "+ByonComputeServiceStaticRef.class.getName(),
+                "    "+SshMachineLocation.SSH_TOOL_CLASS.getName() + ": " + RecordingSshTool.class.getName(),
+                "    waitForSshable: false",
+                "    useJcloudsSshInit: false",
+                "services:\n"+
+                "- type: "+symbolicName,
+                "  brooklyn.config:",
+                "    onbox.base.dir.skipResolution: true",
+                "    sshMonitoring.enabled: false",
+                "    metrics.usage.retrieve: false",
+                "    provisioning.properties:",
+                "      password: $brooklyn:config(\"password\")");
+        
+        EntitySpec<?> spec = 
+                mgmt().getTypeRegistry().createSpecFromPlan(CampTypePlanTransformer.FORMAT, yaml, RegisteredTypeLoadingContexts.spec(Application.class), EntitySpec.class);
+        final Entity app = mgmt().getEntityManager().createEntity(spec);
+        final MachineEntity entity = (MachineEntity) Iterables.getOnlyElement(app.getChildren());
+        app.invoke(Startable.START, ImmutableMap.<String, Object>of()).get();
+        
+        // Execute ssh (with RecordingSshTool), and confirm was given resolved password
+        entity.execCommand("mycmd");
+        Map<?, ?> constructorProps = RecordingSshTool.getLastConstructorProps();
+        ExecCmd execCmd = RecordingSshTool.getLastExecCmd();
+        assertEquals(constructorProps.get("password"), "myYamlPassword", "constructorProps: "+constructorProps+"; execProps: "+execCmd.props);
+        
+        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;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5494ac49/core/src/main/java/org/apache/brooklyn/core/mgmt/BrooklynTaskTags.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/BrooklynTaskTags.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/BrooklynTaskTags.java
index 58018b9..e49bee5 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/BrooklynTaskTags.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/BrooklynTaskTags.java
@@ -23,6 +23,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import java.io.ByteArrayOutputStream;
 import java.util.Collection;
 import java.util.LinkedHashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -53,6 +54,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
 import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
 
 /** Provides utilities for making Tasks easier to work with in Brooklyn.
  * Main thing at present is to supply (and find) wrapped entities for tasks to understand the
@@ -122,21 +124,42 @@ public class BrooklynTaskTags extends TaskTags {
         return new WrappedEntity(TARGET_ENTITY, entity);
     }
 
-    public static Entity getWrappedEntityOfType(Task<?> t, String wrappingType) {
+    public static WrappedEntity getWrappedEntityTagOfType(Task<?> t, String wrappingType) {
         if (t==null) return null;
-        return getWrappedEntityOfType(t.getTags(), wrappingType);
+        return getWrappedEntityTagOfType(t.getTags(), wrappingType);
     }
-    public static Entity getWrappedEntityOfType(Collection<?> tags, String wrappingType) {
+    public static WrappedEntity getWrappedEntityTagOfType(Collection<?> tags, String wrappingType) {
         for (Object x: tags)
             if ((x instanceof WrappedEntity) && ((WrappedEntity)x).wrappingType.equals(wrappingType))
-                return ((WrappedEntity)x).entity;
+                return (WrappedEntity)x;
         return null;
     }
 
+    public static Entity getWrappedEntityOfType(Task<?> t, String wrappingType) {
+        WrappedEntity wrapper = getWrappedEntityTagOfType(t, wrappingType);
+        return (wrapper == null) ? null : wrapper.entity;
+    }
+    public static Entity getWrappedEntityOfType(Collection<?> tags, String wrappingType) {
+        WrappedEntity wrapper = getWrappedEntityTagOfType(tags, wrappingType);
+        return (wrapper == null) ? null : wrapper.entity;
+    }
+
     public static Entity getContextEntity(Task<?> task) {
         return getWrappedEntityOfType(task, CONTEXT_ENTITY);
     }
 
+    public static Object getTargetOrContextEntityTag(Task<?> task) {
+        if (task == null) return null;
+        Object result = getWrappedEntityTagOfType(task, CONTEXT_ENTITY);
+        if (result!=null) return result;
+        result = getWrappedEntityTagOfType(task, TARGET_ENTITY);
+        if (result!=null) return result;
+        result = Tasks.tag(task, Entity.class, false);
+        if (result!=null) return result;
+        
+        return null;
+    }
+    
     public static Entity getTargetOrContextEntity(Task<?> t) {
         if (t==null) return null;
         Entity result = getWrappedEntityOfType(t, CONTEXT_ENTITY);

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5494ac49/core/src/main/java/org/apache/brooklyn/util/core/config/ConfigBag.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/util/core/config/ConfigBag.java b/core/src/main/java/org/apache/brooklyn/util/core/config/ConfigBag.java
index 96d2428..f34d881 100644
--- a/core/src/main/java/org/apache/brooklyn/util/core/config/ConfigBag.java
+++ b/core/src/main/java/org/apache/brooklyn/util/core/config/ConfigBag.java
@@ -620,6 +620,15 @@ public class ConfigBag {
         return this;
     }
 
+    /**
+     * Whether this config bag is "sealed" (i.e. whether no more modifications can be made to it).
+     * This method is for information only - it should not be overridden to try to change the 
+     * semantics, as internal methods access the (private) field directly.
+     */
+    protected final boolean isSealed() {
+        return sealed;
+    }
+    
     // TODO why have both this and mutable
     /** @see #getAllConfigMutable() */
     public Map<String, Object> getAllConfigRaw() {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5494ac49/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 337c7c4..7c6263f 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
@@ -31,6 +31,7 @@ import org.apache.brooklyn.util.guava.Maybe;
 import com.google.common.annotations.Beta;
 import com.google.common.base.Function;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 /**
  * As for {@link ConfigBag}, but resolves values that are of type {@link DeferredSupplier}.
@@ -82,7 +83,32 @@ public class ResolvingConfigBag extends ConfigBag {
         if (parentBag!=null)
             parentBag.markUsed(key);
     }
-    
+
+    // If copying from another {@link ResolvingConfigBag}, avoid resolving the config while doing 
+    // that copy.
+    @Override
+    protected ConfigBag copyWhileSynched(ConfigBag otherRaw) {
+        if (otherRaw instanceof ResolvingConfigBag) {
+            ResolvingConfigBag other = (ResolvingConfigBag) otherRaw;
+            if (isSealed()) 
+                throw new IllegalStateException("Cannot copy "+other+" to "+this+": this config bag has been sealed and is now immutable.");
+            putAll(other.getAllConfigUntransformed());
+            markAll(Sets.difference(other.getAllConfigUntransformed().keySet(), other.getUnusedConfigUntransformed().keySet()));
+            setDescription(other.getDescription());
+            return this;
+        } else {
+            return super.copyWhileSynched(otherRaw);
+        }
+    }
+
+    protected Map<String,Object> getAllConfigUntransformed() {
+        return super.getAllConfig();
+    }
+
+    protected Map<String,Object> getUnusedConfigUntransformed() {
+        return super.getUnusedConfig();
+    }
+
     @SuppressWarnings("unchecked")
     protected <T> T get(ConfigKey<T> key, boolean markUsed) {
         return (T) getTransformer().apply(super.get(key, markUsed));

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5494ac49/core/src/main/java/org/apache/brooklyn/util/core/task/TaskBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/util/core/task/TaskBuilder.java b/core/src/main/java/org/apache/brooklyn/util/core/task/TaskBuilder.java
index 1a60212..8750933 100644
--- a/core/src/main/java/org/apache/brooklyn/util/core/task/TaskBuilder.java
+++ b/core/src/main/java/org/apache/brooklyn/util/core/task/TaskBuilder.java
@@ -123,6 +123,12 @@ public class TaskBuilder<T> {
         return this;
     }
     
+    /** adds a tag to the given task */
+    public TaskBuilder<T> tagIfNotNull(Object tag) {
+        if (tag != null) tags.add(tag);
+        return this;
+    }
+    
     /** adds a flag to the given task */
     public TaskBuilder<T> flag(String flag, Object value) {
         flags.put(flag, value);

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5494ac49/core/src/main/java/org/apache/brooklyn/util/core/task/ValueResolver.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/util/core/task/ValueResolver.java b/core/src/main/java/org/apache/brooklyn/util/core/task/ValueResolver.java
index 0f85c3f..73309ef 100644
--- a/core/src/main/java/org/apache/brooklyn/util/core/task/ValueResolver.java
+++ b/core/src/main/java/org/apache/brooklyn/util/core/task/ValueResolver.java
@@ -337,8 +337,13 @@ public class ValueResolver<T> implements DeferredSupplier<T> {
                             }
                         } };
                     String description = getDescription();
-                    TaskBuilder<Object> tb = Tasks.<Object>builder().body(callable).displayName("Resolving dependent value").description(description);
+                    TaskBuilder<Object> tb = Tasks.<Object>builder()
+                            .body(callable)
+                            .displayName("Resolving dependent value")
+                            .description(description)
+                            .tagIfNotNull(BrooklynTaskTags.getTargetOrContextEntityTag(Tasks.current()));
                     if (isTransientTask) tb.tag(BrooklynTaskTags.TRANSIENT_TASK_TAG);
+                    
                     Task<Object> vt = exec.submit(tb.build());
                     // TODO to handle immediate resolution, it would be nice to be able to submit 
                     // so it executes in the current thread,

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5494ac49/core/src/test/java/org/apache/brooklyn/util/core/internal/ssh/RecordingSshTool.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/brooklyn/util/core/internal/ssh/RecordingSshTool.java b/core/src/test/java/org/apache/brooklyn/util/core/internal/ssh/RecordingSshTool.java
index 04e78cf..0981dfd 100644
--- a/core/src/test/java/org/apache/brooklyn/util/core/internal/ssh/RecordingSshTool.java
+++ b/core/src/test/java/org/apache/brooklyn/util/core/internal/ssh/RecordingSshTool.java
@@ -180,6 +180,10 @@ public class RecordingSshTool implements SshTool {
         return execScriptCmds.get(execScriptCmds.size()-1);
     }
     
+    public static Map<?,?> getLastConstructorProps() {
+        return constructorProps.get(constructorProps.size()-1);
+    }
+    
     public RecordingSshTool(Map<?,?> props) {
         constructorProps.add(props);
     }


[4/5] brooklyn-server git commit: Rename JcloudsLocationExternalConfigYamlTest.java

Posted by al...@apache.org.
Rename JcloudsLocationExternalConfigYamlTest.java

(to ...LiveTest)

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

Branch: refs/heads/master
Commit: 5533af13ed612186f83a2a2338dba4efb12a5f0f
Parents: eab2dfa
Author: Aled Sage <al...@gmail.com>
Authored: Fri Sep 30 12:44:30 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Sep 30 12:44:55 2016 +0100

----------------------------------------------------------------------
 ...loudsLocationExternalConfigYamlLiveTest.java | 121 ++++++++++++++++++
 .../JcloudsLocationExternalConfigYamlTest.java  | 124 -------------------
 2 files changed, 121 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5533af13/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsLocationExternalConfigYamlLiveTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsLocationExternalConfigYamlLiveTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsLocationExternalConfigYamlLiveTest.java
new file mode 100644
index 0000000..6a946d3
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsLocationExternalConfigYamlLiveTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.camp.brooklyn;
+
+import static org.testng.Assert.assertEquals;
+
+import java.io.StringReader;
+
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.location.Location;
+import org.apache.brooklyn.camp.brooklyn.ExternalConfigYamlTest.MyExternalConfigSupplier;
+import org.apache.brooklyn.camp.brooklyn.ExternalConfigYamlTest.MyExternalConfigSupplierWithoutMapArg;
+import org.apache.brooklyn.config.ConfigKey;
+import org.apache.brooklyn.core.config.ConfigKeys;
+import org.apache.brooklyn.core.entity.StartableApplication;
+import org.apache.brooklyn.core.internal.BrooklynProperties;
+import org.apache.brooklyn.core.objs.BrooklynObjectInternal;
+import org.apache.brooklyn.entity.software.base.EmptySoftwareProcess;
+import org.apache.brooklyn.location.jclouds.JcloudsLocation;
+import org.apache.brooklyn.util.core.internal.ssh.SshTool;
+import org.apache.brooklyn.util.core.task.DeferredSupplier;
+import org.apache.brooklyn.util.guava.Maybe;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+
+// also see ExternalConfigYamlTest
+public class JcloudsLocationExternalConfigYamlLiveTest extends AbstractYamlRebindTest {
+
+    private static final Logger log = LoggerFactory.getLogger(ExternalConfigYamlTest.class);
+
+    private static final ConfigKey<String> MY_CONFIG_KEY = ConfigKeys.newStringConfigKey("my.config.key");
+
+    @Override
+    protected BrooklynProperties createBrooklynProperties() {
+        BrooklynProperties props = BrooklynProperties.Factory.newDefault();
+        props.put("brooklyn.external.myprovider", MyExternalConfigSupplier.class.getName());
+        props.put("brooklyn.external.myprovider.mykey", "myval");
+        props.put("brooklyn.external.myproviderWithoutMapArg", MyExternalConfigSupplierWithoutMapArg.class.getName());
+        return props;
+    }
+
+    @Test(groups="Live")
+    public void testJcloudsInheritanceAndPasswordSecret() throws Exception {
+        String yaml = Joiner.on("\n").join(
+                "services:",
+                "- type: "+EmptySoftwareProcess.class.getName(),
+                "location:",
+                "  jclouds:aws-ec2:",
+                "    password: $brooklyn:external(\"myprovider\", \"mykey\")",
+                "    my.config.key: $brooklyn:external(\"myprovider\", \"mykey\")");
+
+        origApp = (StartableApplication) createAndStartApplication(new StringReader(yaml));
+
+        Entity entity = Iterables.getOnlyElement( origApp.getChildren() );
+        Location l = Iterables.getOnlyElement( entity.getLocations() );
+        log.info("Location: "+l);
+        assertEquals(l.config().get(MY_CONFIG_KEY), "myval");
+
+        Maybe<Object> rawConfig = ((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(MY_CONFIG_KEY);
+        log.info("Raw config: "+rawConfig);
+        Assert.assertTrue(rawConfig.isPresentAndNonNull());
+        Assert.assertTrue(rawConfig.get() instanceof DeferredSupplier, "Expected deferred raw value; got "+rawConfig.get());
+
+        rawConfig = ((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(SshTool.PROP_PASSWORD);
+        log.info("Raw config password: "+rawConfig);
+        Assert.assertTrue(rawConfig.isPresentAndNonNull());
+        Assert.assertTrue(rawConfig.get() instanceof DeferredSupplier, "Expected deferred raw value; got "+rawConfig.get());
+    }
+
+    @Test(groups="Live")
+    public void testProvisioningPropertyInheritance() throws Exception {
+        String yaml = Joiner.on("\n").join(
+                "services:",
+                "- type: "+EmptySoftwareProcess.class.getName(),
+                "  provisioning.properties:",
+                "      password: $brooklyn:external(\"myprovider\", \"mykey\")",
+                // note that these 2 do not get transferred -- see below
+                "      simple: 42",
+                "      my.config.key: $brooklyn:external(\"myprovider\", \"mykey\")",
+                "location: aws-ec2");
+
+        origApp = (StartableApplication) createAndStartApplication(new StringReader(yaml));
+
+        Entity entity = Iterables.getOnlyElement( origApp.getChildren() );
+        Location l = Iterables.getOnlyElement( entity.getLocations() );
+        log.info("Location: "+l);
+        assertEquals(l.config().get(JcloudsLocation.PASSWORD), "myval");
+
+        Maybe<Object> rawConfig = ((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(ConfigKeys.newStringConfigKey("password"));
+        log.info("Raw config password: "+rawConfig);
+        Assert.assertTrue(rawConfig.isPresentAndNonNull());
+        Assert.assertTrue(rawConfig.get() instanceof DeferredSupplier, "Expected deferred raw value; got "+rawConfig.get());
+
+        // these are null as only recognised provisioning properties are transmitted by jclouds
+        log.info("my config key: "+l.getConfig(MY_CONFIG_KEY));
+        log.info("my config key raw: "+((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(MY_CONFIG_KEY));
+        log.info("simple: "+l.getConfig(ConfigKeys.builder(Integer.class, "simple").build()));
+        log.info("simple raw: "+((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(ConfigKeys.builder(Integer.class, "simple").build()));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/5533af13/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsLocationExternalConfigYamlTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsLocationExternalConfigYamlTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsLocationExternalConfigYamlTest.java
deleted file mode 100644
index 7e99328..0000000
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsLocationExternalConfigYamlTest.java
+++ /dev/null
@@ -1,124 +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.camp.brooklyn;
-
-import static org.testng.Assert.assertEquals;
-
-import java.io.File;
-import java.io.StringReader;
-
-import org.apache.brooklyn.api.entity.Entity;
-import org.apache.brooklyn.api.location.Location;
-import org.apache.brooklyn.camp.brooklyn.ExternalConfigYamlTest.MyExternalConfigSupplier;
-import org.apache.brooklyn.camp.brooklyn.ExternalConfigYamlTest.MyExternalConfigSupplierWithoutMapArg;
-import org.apache.brooklyn.config.ConfigKey;
-import org.apache.brooklyn.core.config.ConfigKeys;
-import org.apache.brooklyn.core.entity.StartableApplication;
-import org.apache.brooklyn.core.internal.BrooklynProperties;
-import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
-import org.apache.brooklyn.core.mgmt.rebind.RebindTestUtils;
-import org.apache.brooklyn.core.objs.BrooklynObjectInternal;
-import org.apache.brooklyn.entity.software.base.EmptySoftwareProcess;
-import org.apache.brooklyn.location.jclouds.JcloudsLocation;
-import org.apache.brooklyn.util.core.internal.ssh.SshTool;
-import org.apache.brooklyn.util.core.task.DeferredSupplier;
-import org.apache.brooklyn.util.guava.Maybe;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.Iterables;
-
-// also see ExternalConfigYamlTest
-public class JcloudsLocationExternalConfigYamlTest extends AbstractYamlRebindTest {
-
-    private static final Logger log = LoggerFactory.getLogger(ExternalConfigYamlTest.class);
-
-    private static final ConfigKey<String> MY_CONFIG_KEY = ConfigKeys.newStringConfigKey("my.config.key");
-
-    @Override
-    protected BrooklynProperties createBrooklynProperties() {
-        BrooklynProperties props = BrooklynProperties.Factory.newDefault();
-        props.put("brooklyn.external.myprovider", MyExternalConfigSupplier.class.getName());
-        props.put("brooklyn.external.myprovider.mykey", "myval");
-        props.put("brooklyn.external.myproviderWithoutMapArg", MyExternalConfigSupplierWithoutMapArg.class.getName());
-        return props;
-    }
-
-    @Test(groups="Live")
-    public void testJcloudsInheritanceAndPasswordSecret() throws Exception {
-        String yaml = Joiner.on("\n").join(
-                "services:",
-                "- type: "+EmptySoftwareProcess.class.getName(),
-                "location:",
-                "  jclouds:aws-ec2:",
-                "    password: $brooklyn:external(\"myprovider\", \"mykey\")",
-                "    my.config.key: $brooklyn:external(\"myprovider\", \"mykey\")");
-
-        origApp = (StartableApplication) createAndStartApplication(new StringReader(yaml));
-
-        Entity entity = Iterables.getOnlyElement( origApp.getChildren() );
-        Location l = Iterables.getOnlyElement( entity.getLocations() );
-        log.info("Location: "+l);
-        assertEquals(l.config().get(MY_CONFIG_KEY), "myval");
-
-        Maybe<Object> rawConfig = ((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(MY_CONFIG_KEY);
-        log.info("Raw config: "+rawConfig);
-        Assert.assertTrue(rawConfig.isPresentAndNonNull());
-        Assert.assertTrue(rawConfig.get() instanceof DeferredSupplier, "Expected deferred raw value; got "+rawConfig.get());
-
-        rawConfig = ((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(SshTool.PROP_PASSWORD);
-        log.info("Raw config password: "+rawConfig);
-        Assert.assertTrue(rawConfig.isPresentAndNonNull());
-        Assert.assertTrue(rawConfig.get() instanceof DeferredSupplier, "Expected deferred raw value; got "+rawConfig.get());
-    }
-
-    @Test(groups="Live")
-    public void testProvisioningPropertyInheritance() throws Exception {
-        String yaml = Joiner.on("\n").join(
-                "services:",
-                "- type: "+EmptySoftwareProcess.class.getName(),
-                "  provisioning.properties:",
-                "      password: $brooklyn:external(\"myprovider\", \"mykey\")",
-                // note that these 2 do not get transferred -- see below
-                "      simple: 42",
-                "      my.config.key: $brooklyn:external(\"myprovider\", \"mykey\")",
-                "location: aws-ec2");
-
-        origApp = (StartableApplication) createAndStartApplication(new StringReader(yaml));
-
-        Entity entity = Iterables.getOnlyElement( origApp.getChildren() );
-        Location l = Iterables.getOnlyElement( entity.getLocations() );
-        log.info("Location: "+l);
-        assertEquals(l.config().get(JcloudsLocation.PASSWORD), "myval");
-
-        Maybe<Object> rawConfig = ((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(ConfigKeys.newStringConfigKey("password"));
-        log.info("Raw config password: "+rawConfig);
-        Assert.assertTrue(rawConfig.isPresentAndNonNull());
-        Assert.assertTrue(rawConfig.get() instanceof DeferredSupplier, "Expected deferred raw value; got "+rawConfig.get());
-
-        // these are null as only recognised provisioning properties are transmitted by jclouds
-        log.info("my config key: "+l.getConfig(MY_CONFIG_KEY));
-        log.info("my config key raw: "+((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(MY_CONFIG_KEY));
-        log.info("simple: "+l.getConfig(ConfigKeys.builder(Integer.class, "simple").build()));
-        log.info("simple raw: "+((BrooklynObjectInternal.ConfigurationSupportInternal)l.config()).getRaw(ConfigKeys.builder(Integer.class, "simple").build()));
-    }
-}
\ No newline at end of file


[3/5] brooklyn-server git commit: Refactor JcloudsRebindWithYamlDslTest

Posted by al...@apache.org.
Refactor JcloudsRebindWithYamlDslTest

Remove duplication from JcloudsRebindWithExternalConfigTest, creating
common abstract super-type.


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

Branch: refs/heads/master
Commit: 6604f7a9d7deefedce0283e20d883563dc7e366d
Parents: 5494ac4
Author: Aled Sage <al...@gmail.com>
Authored: Tue Sep 27 17:18:04 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Sep 30 12:44:55 2016 +0100

----------------------------------------------------------------------
 .../brooklyn/JcloudsRebindStubYamlTest.java     | 99 ++++++++++++++++++++
 .../JcloudsRebindWithExternalConfigTest.java    | 72 +-------------
 .../brooklyn/JcloudsRebindWithYamlDslTest.java  | 82 ++++------------
 .../core/mgmt/rebind/RebindTestFixture.java     |  2 +-
 .../location/jclouds/JcloudsRebindStubTest.java | 18 ++--
 5 files changed, 130 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/6604f7a9/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindStubYamlTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindStubYamlTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindStubYamlTest.java
new file mode 100644
index 0000000..c6fbcd1
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindStubYamlTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.camp.brooklyn;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.File;
+
+import org.apache.brooklyn.api.mgmt.ha.HighAvailabilityMode;
+import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
+import org.apache.brooklyn.location.jclouds.ComputeServiceRegistry;
+import org.apache.brooklyn.location.jclouds.JcloudsRebindStubTest;
+import org.testng.annotations.AfterMethod;
+
+/**
+ * Implementation notes. This relies on the test {@link JcloudsRebindStubTest#testRebind()}.
+ * It changes the setup for the test in the following ways:
+ * <ul>
+ *   <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 JcloudsRebindStubYamlTest#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.
+ */
+public abstract class JcloudsRebindStubYamlTest extends JcloudsRebindStubTest {
+
+    protected BrooklynCampPlatformLauncherNoServer origLauncher;
+    protected 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 LocalManagementContext createOrigManagementContext() {
+        origLauncher = new BrooklynCampPlatformLauncherNoServer() {
+            @Override
+            protected LocalManagementContext newMgmtContext() {
+                return JcloudsRebindStubYamlTest.super.createOrigManagementContext();
+            }
+        };
+        origLauncher.launch();
+        LocalManagementContext mgmt = (LocalManagementContext) origLauncher.getBrooklynMgmt();
+        return mgmt;
+    }
+
+    @Override
+    protected LocalManagementContext createNewManagementContext(final File mementoDir, final HighAvailabilityMode haMode) {
+        newLauncher = new BrooklynCampPlatformLauncherNoServer() {
+            @Override
+            protected LocalManagementContext newMgmtContext() {
+                return JcloudsRebindStubYamlTest.super.createNewManagementContext(mementoDir, haMode);
+            }
+        };
+        newLauncher.launch();
+        return (LocalManagementContext) newLauncher.getBrooklynMgmt();
+    }
+    
+    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;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/6604f7a9/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
index 160a9c2..fd3baa0 100644
--- 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
@@ -20,25 +20,20 @@ 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.api.mgmt.ha.HighAvailabilityMode;
 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;
@@ -46,37 +41,14 @@ 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 {
+public class JcloudsRebindWithExternalConfigTest extends JcloudsRebindStubYamlTest {
 
-    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();
@@ -93,39 +65,15 @@ public class JcloudsRebindWithExternalConfigTest extends JcloudsRebindStubTest {
     }
     
     @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, final HighAvailabilityMode haMode) {
-        newLauncher = new BrooklynCampPlatformLauncherNoServer() {
-            @Override
-            protected LocalManagementContext newMgmtContext() {
-                return JcloudsRebindWithExternalConfigTest.super.createNewManagementContext(mementoDir, haMode);
-            }
-        };
-        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:",
+                "  "+LOCATION_SPEC+":",
                 "    identity: $brooklyn:external(\"creds\", \"test-identity\")",
                 "    credential: $brooklyn:external(\"creds\", \"test-credential\")",
+                "    imageId: "+IMAGE_ID,
                 "    jclouds.computeServiceRegistry:",
                 "      $brooklyn:object:",
                 "        type: "+ByonComputeServiceStaticRef.class.getName(),
@@ -141,18 +89,4 @@ public class JcloudsRebindWithExternalConfigTest extends JcloudsRebindStubTest {
 
         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;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/6604f7a9/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithYamlDslTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithYamlDslTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithYamlDslTest.java
index ce91156..5864a9a 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithYamlDslTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsRebindWithYamlDslTest.java
@@ -18,10 +18,8 @@
  */
 package org.apache.brooklyn.camp.brooklyn;
 
-import static com.google.common.base.Preconditions.checkNotNull;
 import static org.testng.Assert.assertEquals;
 
-import java.io.File;
 import java.util.Map;
 
 import org.apache.brooklyn.api.entity.Application;
@@ -29,16 +27,16 @@ 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.entity.trait.Startable;
-import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
+import org.apache.brooklyn.core.location.Machines;
 import org.apache.brooklyn.core.typereg.RegisteredTypeLoadingContexts;
 import org.apache.brooklyn.entity.machine.MachineEntity;
 import org.apache.brooklyn.location.jclouds.ComputeServiceRegistry;
 import org.apache.brooklyn.location.jclouds.JcloudsLocation;
+import org.apache.brooklyn.location.jclouds.JcloudsMachineLocation;
 import org.apache.brooklyn.location.jclouds.JcloudsRebindStubTest;
 import org.apache.brooklyn.location.ssh.SshMachineLocation;
 import org.apache.brooklyn.util.core.internal.ssh.RecordingSshTool;
 import org.apache.brooklyn.util.core.internal.ssh.RecordingSshTool.ExecCmd;
-import org.testng.annotations.AfterMethod;
 import org.testng.annotations.Test;
 
 import com.google.common.base.Joiner;
@@ -46,7 +44,8 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 
 /**
- * This is primarily to test https://issues.apache.org/jira/browse/BROOKLYN-349
+ * This is primarily to test https://issues.apache.org/jira/browse/BROOKLYN-349.
+ * It confirms that entity "provisioning.properties" get passed through to the machine.
  * 
  * As per the other {@link JcloudsRebindStubTest} tests, it will connect to SoftLayer to retrieve
  * image details (so needs real credentials), but it will then stub out the VM creation.
@@ -63,47 +62,9 @@ import com.google.common.collect.Iterables;
  * </ul>
  */
 @Test(groups={"Live", "Live-sanity"})
-public class JcloudsRebindWithYamlDslTest extends JcloudsRebindStubTest {
+public class JcloudsRebindWithYamlDslTest extends JcloudsRebindStubYamlTest {
 
-    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 LocalManagementContext createOrigManagementContext() {
-        origLauncher = new BrooklynCampPlatformLauncherNoServer() {
-            @Override
-            protected LocalManagementContext newMgmtContext() {
-                return JcloudsRebindWithYamlDslTest.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 JcloudsRebindWithYamlDslTest.super.createNewManagementContext(mementoDir);
-            }
-        };
-        newLauncher.launch();
-        return (LocalManagementContext) newLauncher.getBrooklynMgmt();
-    }
+    protected Entity origApp;
     
     @Override
     protected JcloudsLocation newJcloudsLocation(ComputeServiceRegistry computeServiceRegistry) throws Exception {
@@ -124,7 +85,8 @@ public class JcloudsRebindWithYamlDslTest extends JcloudsRebindStubTest {
 
         String yaml = Joiner.on("\n").join(
                 "location:",
-                "  jclouds:softlayer:",
+                "  "+LOCATION_SPEC+":",
+                "    imageId: "+IMAGE_ID,
                 "    jclouds.computeServiceRegistry:",
                 "      $brooklyn:object:",
                 "        type: "+ByonComputeServiceStaticRef.class.getName(),
@@ -142,9 +104,15 @@ public class JcloudsRebindWithYamlDslTest extends JcloudsRebindStubTest {
         
         EntitySpec<?> spec = 
                 mgmt().getTypeRegistry().createSpecFromPlan(CampTypePlanTransformer.FORMAT, yaml, RegisteredTypeLoadingContexts.spec(Application.class), EntitySpec.class);
-        final Entity app = mgmt().getEntityManager().createEntity(spec);
-        final MachineEntity entity = (MachineEntity) Iterables.getOnlyElement(app.getChildren());
-        app.invoke(Startable.START, ImmutableMap.<String, Object>of()).get();
+        origApp = mgmt().getEntityManager().createEntity(spec);
+        
+        return (JcloudsLocation) Iterables.getOnlyElement(origApp.getLocations());
+    }
+
+    @Override
+    protected JcloudsMachineLocation obtainMachine(JcloudsLocation jcloudsLoc, Map<?,?> props) throws Exception {
+        final MachineEntity entity = (MachineEntity) Iterables.getOnlyElement(origApp.getChildren());
+        origApp.invoke(Startable.START, ImmutableMap.<String, Object>of()).get();
         
         // Execute ssh (with RecordingSshTool), and confirm was given resolved password
         entity.execCommand("mycmd");
@@ -152,20 +120,6 @@ public class JcloudsRebindWithYamlDslTest extends JcloudsRebindStubTest {
         ExecCmd execCmd = RecordingSshTool.getLastExecCmd();
         assertEquals(constructorProps.get("password"), "myYamlPassword", "constructorProps: "+constructorProps+"; execProps: "+execCmd.props);
         
-        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;
-        }
+        return Machines.findUniqueMachineLocation(entity.getLocations(), JcloudsMachineLocation.class).get();
     }
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/6604f7a9/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 5af282b..5ed62c1 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
@@ -43,7 +43,6 @@ 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;
-import org.apache.brooklyn.core.mgmt.persist.BrooklynPersistenceUtils;
 import org.apache.brooklyn.core.mgmt.persist.FileBasedObjectStore;
 import org.apache.brooklyn.core.mgmt.persist.PersistMode;
 import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests;
@@ -301,6 +300,7 @@ public abstract class RebindTestFixture<T extends StartableApplication> {
         return hotStandby(RebindOptions.create());
     }
 
+    @SuppressWarnings("unchecked")
     protected T hotStandby(RebindOptions options) throws Exception {
         if (newApp != null || newManagementContext != null) {
             throw new IllegalStateException("already rebound - use switchOriginalToNewManagementContext() if you are trying to rebind multiple times");

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/6604f7a9/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 bea39b6..6f09007 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
@@ -18,13 +18,12 @@
  */
 package org.apache.brooklyn.location.jclouds;
 
-import static com.google.common.base.Preconditions.checkNotNull;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertFalse;
 
 import java.net.URI;
 import java.util.List;
-import java.util.Set;
+import java.util.Map;
 
 import org.apache.brooklyn.api.mgmt.ManagementContext;
 import org.apache.brooklyn.core.entity.Entities;
@@ -32,8 +31,6 @@ import org.apache.brooklyn.core.internal.BrooklynProperties;
 import org.apache.brooklyn.core.mgmt.rebind.RebindTestFixtureWithApp;
 import org.apache.brooklyn.core.test.entity.TestApplication;
 import org.apache.brooklyn.util.exceptions.CompoundRuntimeException;
-import org.jclouds.compute.ComputeService;
-import org.jclouds.compute.RunNodesException;
 import org.jclouds.compute.domain.Image;
 import org.jclouds.compute.domain.NodeMetadata;
 import org.jclouds.compute.domain.NodeMetadata.Status;
@@ -44,7 +41,6 @@ import org.jclouds.compute.domain.Template;
 import org.jclouds.compute.domain.Volume;
 import org.jclouds.compute.domain.internal.HardwareImpl;
 import org.jclouds.compute.domain.internal.NodeMetadataImpl;
-import org.jclouds.compute.options.TemplateOptions;
 import org.jclouds.domain.LocationScope;
 import org.jclouds.domain.LoginCredentials;
 import org.jclouds.domain.internal.LocationImpl;
@@ -81,8 +77,8 @@ public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
     private static final Logger LOG = LoggerFactory.getLogger(JcloudsRebindStubTest.class);
 
     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";
+    public static final String LOCATION_SPEC = "jclouds:" + PROVIDER;
+    public static final String IMAGE_ID = "UBUNTU_14_64";
     
     protected List<ManagementContext> mgmts;
     protected Multimap<ManagementContext, JcloudsSshMachineLocation> machines;
@@ -172,7 +168,7 @@ public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
                         Predicates.<Image>alwaysTrue(), // supportsImage, 
                         (String)null, // hypervisor
                         false),
-                SOFTLAYER_IMAGE_ID,
+                IMAGE_ID,
                 new OperatingSystem(
                         OsFamily.CENTOS, 
                         "myOsName", 
@@ -192,7 +188,7 @@ public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
 
         JcloudsLocation origJcloudsLoc = newJcloudsLocation(computeServiceRegistry);
     
-        JcloudsSshMachineLocation origMachine = (JcloudsSshMachineLocation) origJcloudsLoc.obtain(ImmutableMap.of("imageId", SOFTLAYER_IMAGE_ID));
+        JcloudsSshMachineLocation origMachine = (JcloudsSshMachineLocation) obtainMachine(origJcloudsLoc, ImmutableMap.of("imageId", IMAGE_ID));
         
         String origHostname = origMachine.getHostname();
         NodeMetadata origNode = origMachine.getNode();
@@ -217,6 +213,10 @@ public class JcloudsRebindStubTest extends RebindTestFixtureWithApp {
         assertEquals(newJcloudsLoc.getProvider(), origJcloudsLoc.getProvider());
     }
     
+    protected JcloudsMachineLocation obtainMachine(JcloudsLocation jcloudsLoc, Map<?,?> props) throws Exception {
+        return (JcloudsMachineLocation) jcloudsLoc.obtain(ImmutableMap.of("imageId", IMAGE_ID));
+    }
+    
     protected JcloudsLocation newJcloudsLocation(ComputeServiceRegistry computeServiceRegistry) throws Exception {
         return (JcloudsLocation) mgmt().getLocationRegistry().getLocationManaged("jclouds:softlayer", ImmutableMap.of(
                 JcloudsLocation.COMPUTE_SERVICE_REGISTRY, computeServiceRegistry, 


[2/5] brooklyn-server git commit: Fix loc.getHostname when DSL for credentials

Posted by al...@apache.org.
Fix loc.getHostname when DSL for credentials

If using DSL $brooklyn:config(\u201cpassword\u201d) for machine\u2019s creds, then
if call machine.getHostname() outside of the entity\u2019s context it will
fail to resolve the password so will throw an exception.

Instead, we now defer calling getLoginCredentials(), to only do it for
AWS (where we ask it for the hostname), and then if there is an 
exception we fall back to the defaults.

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

Branch: refs/heads/master
Commit: eab2dfa291b18ecfb490b89c7dd96bb33710311e
Parents: 6604f7a
Author: Aled Sage <al...@gmail.com>
Authored: Tue Sep 27 17:24:39 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Sep 30 12:44:55 2016 +0100

----------------------------------------------------------------------
 .../location/jclouds/JcloudsLocation.java       | 181 +++++++++----------
 .../jclouds/JcloudsSshMachineLocation.java      |  13 +-
 .../jclouds/JcloudsAddressesLiveTest.java       |   3 +-
 3 files changed, 94 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/eab2dfa2/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 4681ba7..64e43b0 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
@@ -180,6 +180,7 @@ import com.google.common.base.Predicates;
 import com.google.common.base.Splitter;
 import com.google.common.base.Stopwatch;
 import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -3266,47 +3267,51 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
      * For some clouds (e.g. aws-ec2), it will attempt to find the public hostname.
      */
     protected String getPublicHostname(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, LoginCredentials userCredentials, ConfigBag setup) {
+        return getPublicHostname(node, sshHostAndPort, Suppliers.ofInstance(userCredentials), setup);
+    }
+    
+    protected String getPublicHostname(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, Supplier<? extends LoginCredentials> userCredentials, ConfigBag setup) {
         String provider = (setup != null) ? setup.get(CLOUD_PROVIDER) : null;
         Boolean lookupAwsHostname = (setup != null) ? setup.get(LOOKUP_AWS_HOSTNAME) : null;
         if (provider == null) provider= getProvider();
 
         if ("aws-ec2".equals(provider) && Boolean.TRUE.equals(lookupAwsHostname)) {
-            HostAndPort inferredHostAndPort = null;
-            if (!sshHostAndPort.isPresent()) {
-                try {
-                    String vmIp = getFirstReachableAddress(node, setup);
-                    int port = node.getLoginPort();
-                    inferredHostAndPort = HostAndPort.fromParts(vmIp, port);
-                } catch (Exception e) {
-                    LOG.warn("Error reaching aws-ec2 instance "+node.getId()+"@"+node.getLocation()+" on port "+node.getLoginPort()+"; falling back to jclouds metadata for address", e);
-                }
-            }
-            if (sshHostAndPort.isPresent() || inferredHostAndPort != null) {
-                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, 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);
-                            // 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);
-                        }
-                    }
-                }
-            }
+            Maybe<String> result = getHostnameAws(node, sshHostAndPort, userCredentials, setup);
+            if (result.isPresent()) return result.get();
         }
 
         return getPublicHostnameGeneric(node, setup);
     }
 
+    /**
+     * Attempts to obtain the private hostname or IP of the node, as advertised by the cloud provider.
+     * 
+     * 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) {
+        return getPrivateHostname(node, sshHostAndPort, Suppliers.ofInstance(userCredentials), setup);
+    }
+    
+    protected String getPrivateHostname(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, Supplier<? extends LoginCredentials> userCredentials, ConfigBag setup) {
+        String provider = (setup != null) ? setup.get(CLOUD_PROVIDER) : null;
+        Boolean lookupAwsHostname = (setup != null) ? setup.get(LOOKUP_AWS_HOSTNAME) : null;
+        if (provider == null) provider = getProvider();
+
+        // TODO Discouraged to do cloud-specific things; think of this code for aws as an
+        // exceptional situation rather than a pattern to follow. We need a better way to
+        // do cloud-specific things.
+        if ("aws-ec2".equals(provider) && Boolean.TRUE.equals(lookupAwsHostname)) {
+            Maybe<String> result = getHostnameAws(node, sshHostAndPort, userCredentials, setup);
+            if (result.isPresent()) return result.get();
+        }
+
+        return getPrivateHostnameGeneric(node, setup);
+    }
+
     private String getPublicHostnameGeneric(NodeMetadata node, @Nullable ConfigBag setup) {
         // JcloudsUtil.getFirstReachableAddress() (probably) already succeeded so at least one of the provided
         // public and private IPs is reachable. Prefer the public IP. Don't use hostname as a fallback
@@ -3328,60 +3333,26 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         }
     }
 
-    private String getPublicHostnameAws(HostAndPort hostAndPort, LoginCredentials userCredentials, ConfigBag setup) {
-        SshMachineLocation sshLocByIp = null;
-        try {
-            // TODO messy way to get an SSH session
-            sshLocByIp = createTemporarySshMachineLocation(hostAndPort, userCredentials, setup);
-
-            ByteArrayOutputStream outStream = new ByteArrayOutputStream();
-            ByteArrayOutputStream errStream = new ByteArrayOutputStream();
-            int exitcode = sshLocByIp.execCommands(
-                    MutableMap.of("out", outStream, "err", errStream),
-                    "get public AWS hostname",
-                    ImmutableList.of(
-                            BashCommands.INSTALL_CURL,
-                            "echo `curl --silent --retry 20 http://169.254.169.254/latest/meta-data/public-hostname`; exit"));
-            String outString = new String(outStream.toByteArray());
-            String[] outLines = outString.split("\n");
-            for (String line : outLines) {
-                if (line.startsWith("ec2-")) return line.trim();
+    private String getPrivateHostnameGeneric(NodeMetadata node, @Nullable ConfigBag setup) {
+        //prefer the private address to the hostname because hostname is sometimes wrong/abbreviated
+        //(see that javadoc; also e.g. on rackspace/cloudstack, the hostname is not registered with any DNS).
+        //Don't return local-only address (e.g. never 127.0.0.1)
+        if (groovyTruth(node.getPrivateAddresses())) {
+            for (String p : node.getPrivateAddresses()) {
+                if (Networking.isLocalOnly(p)) continue;
+                return p;
             }
-            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);
         }
-    }
-
-    /**
-     * Attempts to obtain the private hostname or IP of the node, as advertised by the cloud provider.
-     * 
-     * 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;
-        Boolean lookupAwsHostname = (setup != null) ? setup.get(LOOKUP_AWS_HOSTNAME) : null;
-        
-        if (provider == null) provider = getProvider();
-
-        // TODO Discouraged to do cloud-specific things; think of this code for aws as an
-        // exceptional situation rather than a pattern to follow. We need a better way to
-        // do cloud-specific things.
-        if ("aws-ec2".equals(provider) && Boolean.TRUE.equals(lookupAwsHostname)) {
-            Maybe<String> result = getPrivateHostnameAws(node, sshHostAndPort, userCredentials, setup);
-            if (result.isPresent()) return result.get();
+        if (groovyTruth(node.getPublicAddresses())) {
+            return node.getPublicAddresses().iterator().next();
+        } else if (groovyTruth(node.getHostname())) {
+            return node.getHostname();
+        } else {
+            return null;
         }
-
-        return getPrivateHostnameGeneric(node, setup);
     }
 
-    private Maybe<String> getPrivateHostnameAws(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, LoginCredentials userCredentials, ConfigBag setup) {
-        // TODO Remove duplication from getPublicHostname.
-        // TODO Don't like 
+    private Maybe<String> getHostnameAws(NodeMetadata node, Optional<HostAndPort> sshHostAndPort, Supplier<? extends LoginCredentials> userCredentials, ConfigBag setup) {
         HostAndPort inferredHostAndPort = null;
         if (!sshHostAndPort.isPresent()) {
             try {
@@ -3393,32 +3364,42 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
             }
         }
         if (sshHostAndPort.isPresent() || inferredHostAndPort != null) {
-            HostAndPort hostAndPortToUse = sshHostAndPort.isPresent() ? sshHostAndPort.get() : inferredHostAndPort;
-            try {
-                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);
+            if (isWindows(node, setup)) {
+                LOG.warn("Cannpy query aws-ec2 Windows instance "+node.getId()+"@"+node.getLocation()+" over ssh for its hostname; falling back to jclouds metadata for address");
+            } else {
+                HostAndPort hostAndPortToUse = sshHostAndPort.isPresent() ? sshHostAndPort.get() : inferredHostAndPort;
+                try {
+                    return Maybe.of(getHostnameAws(hostAndPortToUse, userCredentials.get(), setup));
+                } catch (Exception e) {
+                    LOG.warn("Error querying aws-ec2 instance "+node.getId()+"@"+node.getLocation()+" over ssh for its hostname; falling back to jclouds metadata for address", e);
+                }
             }
         }
         return Maybe.absent();
     }
 
-    private String getPrivateHostnameGeneric(NodeMetadata node, @Nullable ConfigBag setup) {
-        //prefer the private address to the hostname because hostname is sometimes wrong/abbreviated
-        //(see that javadoc; also e.g. on rackspace/cloudstack, the hostname is not registered with any DNS).
-        //Don't return local-only address (e.g. never 127.0.0.1)
-        if (groovyTruth(node.getPrivateAddresses())) {
-            for (String p : node.getPrivateAddresses()) {
-                if (Networking.isLocalOnly(p)) continue;
-                return p;
+    private String getHostnameAws(HostAndPort hostAndPort, LoginCredentials userCredentials, ConfigBag setup) {
+        SshMachineLocation sshLocByIp = null;
+        try {
+            // TODO messy way to get an SSH session
+            sshLocByIp = createTemporarySshMachineLocation(hostAndPort, userCredentials, setup);
+
+            ByteArrayOutputStream outStream = new ByteArrayOutputStream();
+            ByteArrayOutputStream errStream = new ByteArrayOutputStream();
+            int exitcode = sshLocByIp.execCommands(
+                    MutableMap.of("out", outStream, "err", errStream),
+                    "get public AWS hostname",
+                    ImmutableList.of(
+                            BashCommands.INSTALL_CURL,
+                            "echo `curl --silent --retry 20 http://169.254.169.254/latest/meta-data/public-hostname`; exit"));
+            String outString = new String(outStream.toByteArray());
+            String[] outLines = outString.split("\n");
+            for (String line : outLines) {
+                if (line.startsWith("ec2-")) return line.trim();
             }
-        }
-        if (groovyTruth(node.getPublicAddresses())) {
-            return node.getPublicAddresses().iterator().next();
-        } else if (groovyTruth(node.getHostname())) {
-            return node.getHostname();
-        } else {
-            return null;
+            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);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/eab2dfa2/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsSshMachineLocation.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsSshMachineLocation.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsSshMachineLocation.java
index 714fda4..3ef596a 100644
--- a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsSshMachineLocation.java
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsSshMachineLocation.java
@@ -63,6 +63,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
 import com.google.common.base.Optional;
+import com.google.common.base.Supplier;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -312,7 +313,7 @@ public class JcloudsSshMachineLocation extends SshMachineLocation implements Jcl
             Optional<NodeMetadata> node = getOptionalNode();
             if (node.isPresent()) {
                 HostAndPort sshHostAndPort = getSshHostAndPort();
-                LoginCredentials creds = getLoginCredentials();
+                Supplier<LoginCredentials> creds = getLoginCredentialsSupplier();
                 hostname = jcloudsParent.getPublicHostname(node.get(), Optional.of(sshHostAndPort), creds, config().getBag());
                 requestPersist();
 
@@ -355,7 +356,7 @@ public class JcloudsSshMachineLocation extends SshMachineLocation implements Jcl
                 // If we can't get the node (i.e. the cloud provider doesn't know that id, because it has
                 // been terminated), then we don't care as much about getting the right id!
                 HostAndPort sshHostAndPort = getSshHostAndPort();
-                LoginCredentials creds = getLoginCredentials();
+                Supplier<LoginCredentials> creds = getLoginCredentialsSupplier();
                 privateHostname = jcloudsParent.getPrivateHostname(node.get(), Optional.of(sshHostAndPort), creds, config().getBag());
 
             } else {
@@ -496,6 +497,14 @@ public class JcloudsSshMachineLocation extends SshMachineLocation implements Jcl
         }
     }
 
+    private Supplier<LoginCredentials> getLoginCredentialsSupplier() {
+        return new Supplier<LoginCredentials>() {
+            @Override public LoginCredentials get() {
+                return getLoginCredentials();
+            }
+        };
+    }
+    
     private LoginCredentials getLoginCredentials() {
         OsCredential creds = LocationConfigUtils.getOsCredential(new ResolvingConfigBag(getManagementContext(), config().getBag()));
         

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/eab2dfa2/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 12e064d..35a5e6a 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
@@ -107,11 +107,12 @@ public class JcloudsAddressesLiveTest extends AbstractJcloudsLiveTest {
         assertNotNull(subnetIp, msg);
         assertReachableFromMachine(machine, subnetIp, msg);
 
-        // hostname is reachable from inside; not necessarily reachable from outside
+        // hostname is reachable from inside; for AWS machines, it is also reachable from outside
         assertNotNull(hostname, msg);
         assertReachableFromMachine(machine, hostname, msg);
         
         assertNotNull(subnetHostname, msg);
+        assertReachable(machine, subnetHostname, msg);
         assertReachableFromMachine(machine, subnetHostname, msg);
     }
 


[5/5] brooklyn-server git commit: This closes #341

Posted by al...@apache.org.
This closes #341


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

Branch: refs/heads/master
Commit: 438e7464897a06b0c6276ad796bdc2fe69c12680
Parents: 2116949 5533af1
Author: Aled Sage <al...@gmail.com>
Authored: Fri Sep 30 14:01:13 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Sep 30 14:01:13 2016 +0100

----------------------------------------------------------------------
 .../brooklyn/spi/dsl/methods/DslComponent.java  |  79 ++++----
 ...loudsLocationExternalConfigYamlLiveTest.java | 121 +++++++++++++
 .../JcloudsLocationExternalConfigYamlTest.java  | 124 -------------
 .../brooklyn/JcloudsRebindStubYamlTest.java     |  99 ++++++++++
 .../JcloudsRebindWithExternalConfigTest.java    |  72 +-------
 .../brooklyn/JcloudsRebindWithYamlDslTest.java  | 125 +++++++++++++
 .../brooklyn/core/mgmt/BrooklynTaskTags.java    |  31 +++-
 .../brooklyn/util/core/config/ConfigBag.java    |   9 +
 .../util/core/config/ResolvingConfigBag.java    |  28 ++-
 .../brooklyn/util/core/task/TaskBuilder.java    |   6 +
 .../brooklyn/util/core/task/ValueResolver.java  |   7 +-
 .../core/mgmt/rebind/RebindTestFixture.java     |   2 +-
 .../core/internal/ssh/RecordingSshTool.java     |   4 +
 .../location/jclouds/JcloudsLocation.java       | 181 +++++++++----------
 .../jclouds/JcloudsSshMachineLocation.java      |  13 +-
 .../jclouds/JcloudsAddressesLiveTest.java       |   3 +-
 .../location/jclouds/JcloudsRebindStubTest.java |  18 +-
 17 files changed, 577 insertions(+), 345 deletions(-)
----------------------------------------------------------------------