You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by he...@apache.org on 2016/01/19 16:51:15 UTC

[1/3] incubator-brooklyn git commit: clean up how start(Location) works, for Dynamic{Cluster, Fabric}

Repository: incubator-brooklyn
Updated Branches:
  refs/heads/master 0fa772a6f -> 7d2ae06ec


clean up how start(Location) works, for Dynamic{Cluster,Fabric}


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

Branch: refs/heads/master
Commit: e3da35dcf9720c6627756f2d8c85431f7f7743ee
Parents: 1c7ab43
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Tue Jan 19 13:33:05 2016 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Tue Jan 19 13:33:05 2016 +0000

----------------------------------------------------------------------
 .../brooklyn/core/entity/AbstractEntity.java    | 13 +++++---
 .../brooklyn/core/entity/EntityInternal.java    |  4 ++-
 .../entity/group/DynamicClusterImpl.java        | 35 +++++++++++---------
 .../entity/group/DynamicFabricImpl.java         | 25 +++++---------
 4 files changed, 39 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e3da35dc/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java b/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java
index 0599373..5422fb6 100644
--- a/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java
+++ b/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java
@@ -874,15 +874,18 @@ public abstract class AbstractEntity extends AbstractBrooklynObject implements E
 
     @Override
     public void addLocations(Collection<? extends Location> newLocations) {
+        if (newLocations==null || newLocations.isEmpty()) {
+            return;
+        }
         synchronized (locations) {
             List<Location> oldLocations = locations.get();
-            Set<Location> truelyNewLocations = Sets.newLinkedHashSet(newLocations);
-            truelyNewLocations.removeAll(oldLocations);
-            if (truelyNewLocations.size() > 0) {
-                locations.set(ImmutableList.<Location>builder().addAll(oldLocations).addAll(truelyNewLocations).build());
+            Set<Location> trulyNewLocations = Sets.newLinkedHashSet(newLocations);
+            trulyNewLocations.removeAll(oldLocations);
+            if (trulyNewLocations.size() > 0) {
+                locations.set(ImmutableList.<Location>builder().addAll(oldLocations).addAll(trulyNewLocations).build());
             }
             
-            for (Location loc : truelyNewLocations) {
+            for (Location loc : trulyNewLocations) {
                 sensors().emit(AbstractEntity.LOCATION_ADDED, loc);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e3da35dc/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/EntityInternal.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/EntityInternal.java b/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/EntityInternal.java
index 4056eca..3602bee 100644
--- a/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/EntityInternal.java
+++ b/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/entity/EntityInternal.java
@@ -21,6 +21,8 @@ package org.apache.brooklyn.core.entity;
 import java.util.Collection;
 import java.util.Map;
 
+import javax.annotation.Nullable;
+
 import org.apache.brooklyn.api.effector.Effector;
 import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.entity.EntityLocal;
@@ -48,7 +50,7 @@ import com.google.common.annotations.Beta;
 @Beta
 public interface EntityInternal extends BrooklynObjectInternal, EntityLocal, Rebindable {
     
-    void addLocations(Collection<? extends Location> locations);
+    void addLocations(@Nullable Collection<? extends Location> locations);
 
     void removeLocations(Collection<? extends Location> locations);
 

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e3da35dc/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java
index 16a82d4..e1dffa8 100644
--- a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java
+++ b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java
@@ -21,6 +21,7 @@ package org.apache.brooklyn.entity.group;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
@@ -311,9 +312,10 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
         setConfigEvenIfOwned(FACTORY, factory);
     }
 
-    private Location getLocation() {
+    private Location getLocation(boolean required) {
         Collection<? extends Location> ll = Locations.getLocationsCheckingAncestors(getLocations(), this);
         try {
+            if (!required && ll.isEmpty()) return null;
             return Iterables.getOnlyElement(ll);
         } catch (Exception e) {
             Exceptions.propagateIfFatal(e);
@@ -360,18 +362,17 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
 
     @Override
     public void start(Collection<? extends Location> locsO) {
-        if (locsO!=null) {
-            checkArgument(locsO.size() <= 1, "Wrong number of locations supplied to start %s: %s", this, locsO);
-            addLocations(locsO);
-        }
-        Location loc = getLocation();
+        addLocations(locsO);
+        Location loc = getLocation(false);
 
         EntitySpec<?> spec = getConfig(MEMBER_SPEC);
         if (spec!=null) {
-            setDefaultDisplayName("Cluster of "+JavaClassNames.simpleClassName(spec.getType()) +" ("+loc+")");
+            setDefaultDisplayName("Cluster of "+JavaClassNames.simpleClassName(spec.getType()) +
+                (loc!=null ? " ("+loc+")" : ""));
         }
 
         if (isAvailabilityZoneEnabled()) {
+            if (loc==null) throw new IllegalStateException("When using availability zones, a location must be specified on the cluster");
             sensors().set(SUB_LOCATIONS, findSubLocations(loc));
         }
 
@@ -574,7 +575,7 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
             Location memberLoc = null;
             if (isAvailabilityZoneEnabled()) {
                 // this member's location could be a machine provisioned by a sub-location, or the actual sub-location
-                List<Location> subLocations = findSubLocations(getLocation());
+                List<Location> subLocations = findSubLocations(getLocation(true));
                 Collection<Location> actualMemberLocs = member.getLocations();
                 boolean foundMatch = false;
                 for (Iterator<Location> iter = actualMemberLocs.iterator(); !foundMatch && iter.hasNext();) {
@@ -605,7 +606,7 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
                 // Replacing member, so new member should be in the same location as that being replaced.
                 // Expect this to agree with `getMemberSpec().getLocations()` (if set). If not, then 
                 // presumably there was a reason this specific member was started somewhere else!
-                memberLoc = getLocation();
+                memberLoc = getLocation(false);
             }
 
             Entity replacement = replaceMember(member, memberLoc, ImmutableMap.of());
@@ -616,7 +617,7 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
     /**
      * @throws StopFailedRuntimeException If stop failed, after successfully starting replacement
      */
-    protected Entity replaceMember(Entity member, Location memberLoc, Map<?, ?> extraFlags) {
+    protected Entity replaceMember(Entity member, @Nullable Location memberLoc, Map<?, ?> extraFlags) {
         synchronized (mutex) {
             ReferenceWithError<Optional<Entity>> added = addInSingleLocation(memberLoc, extraFlags);
 
@@ -653,7 +654,7 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
     protected List<Location> getNonFailedSubLocations() {
         List<Location> result = Lists.newArrayList();
         Set<Location> failed = Sets.newLinkedHashSet();
-        List<Location> subLocations = findSubLocations(getLocation());
+        List<Location> subLocations = findSubLocations(getLocation(true));
         Set<Location> oldFailedSubLocations = getAttribute(FAILED_SUB_LOCATIONS);
         if (oldFailedSubLocations == null)
             oldFailedSubLocations = ImmutableSet.<Location> of();
@@ -722,7 +723,7 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
                         + ", when expected delta " + delta + " in " + this);
             }
         } else {
-            chosenLocations = Collections.nCopies(delta, getLocation());
+            chosenLocations = Collections.nCopies(delta, getLocation(false));
         }
 
         // create and start the entities.
@@ -759,8 +760,8 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
         }
     }
 
-    protected ReferenceWithError<Optional<Entity>> addInSingleLocation(Location location, Map<?,?> flags) {
-        ReferenceWithError<Collection<Entity>> added = addInEachLocation(ImmutableList.of(location), flags);
+    protected ReferenceWithError<Optional<Entity>> addInSingleLocation(@Nullable Location location, Map<?,?> flags) {
+        ReferenceWithError<Collection<Entity>> added = addInEachLocation(Arrays.asList(location), flags);
         
         Optional<Entity> result = Iterables.isEmpty(added.getWithoutError()) ? Optional.<Entity>absent() : Optional.of(Iterables.getOnlyElement(added.get()));
         if (!added.hasError()) {
@@ -885,7 +886,7 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
     }
 
     @Override
-    public Entity addNode(Location loc, Map<?, ?> extraFlags) {
+    public Entity addNode(@Nullable Location loc, Map<?, ?> extraFlags) {
         // In case subclasses are foolish and do not call super.init() when overriding.
         initialiseMemberId();
         Map<?, ?> createFlags = MutableMap.builder()
@@ -917,7 +918,9 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
         if (memberSpec == null) memberSpec = getMemberSpec();
         
         if (memberSpec != null) {
-            return addChild(EntitySpec.create(memberSpec).configure(flags).location(loc));
+            EntitySpec<?> specConfigured = EntitySpec.create(memberSpec).configure(flags);
+            if (loc!=null) specConfigured.location(loc);
+            return addChild(specConfigured);
         }
 
         EntityFactory<?> factory = getFactory();

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e3da35dc/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicFabricImpl.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicFabricImpl.java b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicFabricImpl.java
index 92bf5d2..34051c5 100644
--- a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicFabricImpl.java
+++ b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicFabricImpl.java
@@ -18,13 +18,11 @@
  */
 package org.apache.brooklyn.entity.group;
 
-import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.brooklyn.util.groovy.GroovyJavaMethods.elvis;
 import static org.apache.brooklyn.util.groovy.GroovyJavaMethods.truth;
 
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
@@ -112,16 +110,11 @@ public class DynamicFabricImpl extends AbstractGroupImpl implements DynamicFabri
     
     @Override
     public void start(Collection<? extends Location> locsO) {
-        if (locsO!=null) {
-            addLocations(locsO);
-        }
-        Collection<Location> locs = Collections.unmodifiableCollection(Locations.getLocationsCheckingAncestors(getLocations(), this));
-
-        List<Location> newLocations = MutableList.copyOf(locsO);
-        if (newLocations.isEmpty()) newLocations.addAll(locs);
+        addLocations(locsO);
+        List<Location> locationsToStart = MutableList.copyOf(Locations.getLocationsCheckingAncestors(locsO, this));
         
-        Preconditions.checkNotNull(newLocations, "locations must be supplied");
-        Preconditions.checkArgument(newLocations.size() >= 1, "One or more locations must be supplied");
+        Preconditions.checkNotNull(locationsToStart, "locations must be supplied");
+        Preconditions.checkArgument(locationsToStart.size() >= 1, "One or more locations must be supplied");
         
         int locIndex = 0;
         
@@ -137,8 +130,8 @@ public class DynamicFabricImpl extends AbstractGroupImpl implements DynamicFabri
                     Location it = null;
                     if (child.getLocations().isEmpty())
                         // give him any of these locations if he has none, allowing round robin here
-                        if (!newLocations.isEmpty()) {
-                            it = newLocations.get(locIndex++ % newLocations.size());
+                        if (!locationsToStart.isEmpty()) {
+                            it = locationsToStart.get(locIndex++ % locationsToStart.size());
                             ((EntityInternal)child).addLocations(Arrays.asList(it));
                         }
                     
@@ -148,12 +141,12 @@ public class DynamicFabricImpl extends AbstractGroupImpl implements DynamicFabri
                 }
             }
             // remove all the locations we applied to existing nodes
-            while (locIndex-->0 && !newLocations.isEmpty())
-                newLocations.remove(0);
+            while (locIndex-->0 && !locationsToStart.isEmpty())
+                locationsToStart.remove(0);
 
             // finally (and usually) we create new entities for locations passed in
             // (unless they were consumed by pre-existing children which didn't have locations)
-            for (Location it : newLocations) {
+            for (Location it : locationsToStart) {
                 Entity e = addCluster(it);
                 
                 ((EntityInternal)e).addLocations(Arrays.asList(it));


[3/3] incubator-brooklyn git commit: This closes #1161

Posted by he...@apache.org.
This closes #1161


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

Branch: refs/heads/master
Commit: 7d2ae06ecf1a3907b73625436180c23fa3eab99a
Parents: 0fa772a ba8bb7f
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Tue Jan 19 15:50:59 2016 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Tue Jan 19 15:50:59 2016 +0000

----------------------------------------------------------------------
 .../messaging/kafka/KafkaClusterImpl.java       |  6 ++-
 .../ControlledDynamicWebAppClusterImpl.java     |  7 ++-
 .../brooklyn/core/entity/AbstractEntity.java    | 13 +++---
 .../brooklyn/core/entity/EntityInternal.java    |  4 +-
 .../core/location/BasicLocationRegistry.java    |  4 +-
 .../entity/group/DynamicClusterImpl.java        | 47 +++++++++++---------
 .../entity/group/DynamicFabricImpl.java         | 25 ++++-------
 .../entity/stock/BasicStartableImpl.java        |  3 +-
 .../brooklyn/entity/stock/DataEntityImpl.java   |  1 +
 .../entity/group/DynamicClusterTest.java        | 16 +++----
 .../software/base/SameServerEntityImpl.java     |  7 ++-
 .../test/framework/ParallelTestCaseImpl.java    |  2 +-
 12 files changed, 76 insertions(+), 59 deletions(-)
----------------------------------------------------------------------



[2/3] incubator-brooklyn git commit: tidy several other places start(Collection) is assuming locations are always passed in, not inherited

Posted by he...@apache.org.
tidy several other places start(Collection<Location>) is assuming locations are always passed in, not inherited


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

Branch: refs/heads/master
Commit: ba8bb7f6dba6553888a669bd9cc1d4e3f27cb3dd
Parents: e3da35d
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Tue Jan 19 13:52:09 2016 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Tue Jan 19 14:40:46 2016 +0000

----------------------------------------------------------------------
 .../entity/messaging/kafka/KafkaClusterImpl.java    |  6 +++++-
 .../webapp/ControlledDynamicWebAppClusterImpl.java  |  7 +++++--
 .../core/location/BasicLocationRegistry.java        |  4 +++-
 .../brooklyn/entity/group/DynamicClusterImpl.java   | 14 +++++++-------
 .../brooklyn/entity/stock/BasicStartableImpl.java   |  3 ++-
 .../brooklyn/entity/stock/DataEntityImpl.java       |  1 +
 .../brooklyn/entity/group/DynamicClusterTest.java   | 16 ++++++++--------
 .../entity/software/base/SameServerEntityImpl.java  |  7 ++++++-
 .../test/framework/ParallelTestCaseImpl.java        |  2 +-
 9 files changed, 38 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ba8bb7f6/brooklyn-library/software/messaging/src/main/java/org/apache/brooklyn/entity/messaging/kafka/KafkaClusterImpl.java
----------------------------------------------------------------------
diff --git a/brooklyn-library/software/messaging/src/main/java/org/apache/brooklyn/entity/messaging/kafka/KafkaClusterImpl.java b/brooklyn-library/software/messaging/src/main/java/org/apache/brooklyn/entity/messaging/kafka/KafkaClusterImpl.java
index 8f4e7fb..a95adf9 100644
--- a/brooklyn-library/software/messaging/src/main/java/org/apache/brooklyn/entity/messaging/kafka/KafkaClusterImpl.java
+++ b/brooklyn-library/software/messaging/src/main/java/org/apache/brooklyn/entity/messaging/kafka/KafkaClusterImpl.java
@@ -28,6 +28,7 @@ import org.apache.brooklyn.core.entity.AbstractEntity;
 import org.apache.brooklyn.core.entity.Entities;
 import org.apache.brooklyn.core.entity.trait.Startable;
 import org.apache.brooklyn.core.feed.ConfigToAttributes;
+import org.apache.brooklyn.core.location.Locations;
 import org.apache.brooklyn.enricher.stock.Enrichers;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -104,11 +105,14 @@ public class KafkaClusterImpl extends AbstractEntity implements KafkaCluster {
     @Override
     public void start(Collection<? extends Location> locations) {
         if (isLegacyConstruction()) {
+            // TODO should no longer be needed?
             init();
         }
 
-        if (locations.isEmpty()) locations = getLocations();
+        locations = MutableList.copyOf(Locations.getLocationsCheckingAncestors(locations, this));
+
         Iterables.getOnlyElement(locations); // Assert just one
+        // set it; here we don't allow changing locations
         addLocations(locations);
 
         List<Entity> childrenToStart = MutableList.<Entity>of(getCluster());

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ba8bb7f6/brooklyn-library/software/webapp/src/main/java/org/apache/brooklyn/entity/webapp/ControlledDynamicWebAppClusterImpl.java
----------------------------------------------------------------------
diff --git a/brooklyn-library/software/webapp/src/main/java/org/apache/brooklyn/entity/webapp/ControlledDynamicWebAppClusterImpl.java b/brooklyn-library/software/webapp/src/main/java/org/apache/brooklyn/entity/webapp/ControlledDynamicWebAppClusterImpl.java
index 78898bd..8c52746 100644
--- a/brooklyn-library/software/webapp/src/main/java/org/apache/brooklyn/entity/webapp/ControlledDynamicWebAppClusterImpl.java
+++ b/brooklyn-library/software/webapp/src/main/java/org/apache/brooklyn/entity/webapp/ControlledDynamicWebAppClusterImpl.java
@@ -38,6 +38,7 @@ import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
 import org.apache.brooklyn.core.entity.trait.Startable;
 import org.apache.brooklyn.core.entity.trait.StartableMethods;
 import org.apache.brooklyn.core.feed.ConfigToAttributes;
+import org.apache.brooklyn.core.location.Locations;
 import org.apache.brooklyn.enricher.stock.Enrichers;
 import org.apache.brooklyn.entity.group.DynamicGroupImpl;
 import org.apache.brooklyn.entity.proxy.LoadBalancer;
@@ -207,7 +208,8 @@ public class ControlledDynamicWebAppClusterImpl extends DynamicGroupImpl impleme
                 init();
             }
 
-            if (locations.isEmpty()) locations = getLocations();
+            locations = Locations.getLocationsCheckingAncestors(locations, this);
+            // store inherited locations
             addLocations(locations);
 
             LoadBalancer loadBalancer = getController();
@@ -229,7 +231,8 @@ public class ControlledDynamicWebAppClusterImpl extends DynamicGroupImpl impleme
                 }
             }
 
-            Entities.invokeEffectorList(this, childrenToStart, Startable.START, ImmutableMap.of("locations", locations)).get();
+            // don't propagate start locations
+            Entities.invokeEffectorList(this, childrenToStart, Startable.START, ImmutableMap.of("locations", MutableList.of())).get();
             if (startControllerTask != null) {
                 startControllerTask.get();
             }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ba8bb7f6/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/location/BasicLocationRegistry.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/location/BasicLocationRegistry.java b/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/location/BasicLocationRegistry.java
index 4954393..60fb061 100644
--- a/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/location/BasicLocationRegistry.java
+++ b/brooklyn-server/core/src/main/java/org/apache/brooklyn/core/location/BasicLocationRegistry.java
@@ -430,7 +430,9 @@ public class BasicLocationRegistry implements LocationRegistry {
     public List<Location> resolve(Iterable<?> spec) {
         List<Location> result = new ArrayList<Location>();
         for (Object id : spec) {
-            if (id instanceof String) {
+            if (id==null) {
+                // drop a null entry
+            } if (id instanceof String) {
                 result.add(resolve((String) id));
             } else if (id instanceof Location) {
                 result.add((Location) id);

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ba8bb7f6/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java
index e1dffa8..f434fcf 100644
--- a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java
+++ b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/group/DynamicClusterImpl.java
@@ -314,14 +314,14 @@ public class DynamicClusterImpl extends AbstractGroupImpl implements DynamicClus
 
     private Location getLocation(boolean required) {
         Collection<? extends Location> ll = Locations.getLocationsCheckingAncestors(getLocations(), this);
-        try {
-            if (!required && ll.isEmpty()) return null;
-            return Iterables.getOnlyElement(ll);
-        } catch (Exception e) {
-            Exceptions.propagateIfFatal(e);
-            if (ll.isEmpty()) throw new IllegalStateException("No location available for "+this);
-            else throw new IllegalStateException("Ambiguous location for "+this+"; expected one but had "+ll);
+        if (ll.isEmpty()) {
+            if (!required) return null;
+            throw new IllegalStateException("No location available for "+this);
+        }
+        if (ll.size()>1) {
+            throw new IllegalStateException("Ambiguous location for "+this+"; expected one but had "+ll);
         }
+        return Iterables.getOnlyElement(ll);
     }
 
     protected boolean isAvailabilityZoneEnabled() {

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ba8bb7f6/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/BasicStartableImpl.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/BasicStartableImpl.java b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/BasicStartableImpl.java
index b5a8616..54be703 100644
--- a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/BasicStartableImpl.java
+++ b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/BasicStartableImpl.java
@@ -48,11 +48,12 @@ public class BasicStartableImpl extends AbstractEntity implements BasicStartable
 
     @Override
     public void start(Collection<? extends Location> locations) {
-        log.info("Starting entity "+this+" at "+locations);
         try {
             ServiceStateLogic.setExpectedState(this, Lifecycle.STARTING);
 
             addLocations(locations);
+            locations = Locations.getLocationsCheckingAncestors(locations, this);
+            log.info("Starting entity "+this+" at "+locations);
 
             // essentially does StartableMethods.start(this, locations),
             // but optionally filters locations for each child

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ba8bb7f6/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/DataEntityImpl.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/DataEntityImpl.java b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/DataEntityImpl.java
index 9ee3b28..cc6d8d7 100644
--- a/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/DataEntityImpl.java
+++ b/brooklyn-server/core/src/main/java/org/apache/brooklyn/entity/stock/DataEntityImpl.java
@@ -38,6 +38,7 @@ public class DataEntityImpl extends AbstractEntity implements DataEntity {
 
     @Override
     public void start(Collection<? extends Location> locations) {
+        addLocations(locations);
         connectSensors();
         sensors().set(SERVICE_UP, Boolean.TRUE);
     }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ba8bb7f6/brooklyn-server/core/src/test/java/org/apache/brooklyn/entity/group/DynamicClusterTest.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/core/src/test/java/org/apache/brooklyn/entity/group/DynamicClusterTest.java b/brooklyn-server/core/src/test/java/org/apache/brooklyn/entity/group/DynamicClusterTest.java
index f58ac90..b58c630 100644
--- a/brooklyn-server/core/src/test/java/org/apache/brooklyn/entity/group/DynamicClusterTest.java
+++ b/brooklyn-server/core/src/test/java/org/apache/brooklyn/entity/group/DynamicClusterTest.java
@@ -110,9 +110,9 @@ public class DynamicClusterTest extends BrooklynAppUnitTestSupport {
         try {
             app.createAndManageChild(EntitySpec.create(DynamicCluster.class)
                     .configure("factory", "error"));
-            fail();
+            Asserts.shouldHaveFailedPreviously();
         } catch (Exception e) {
-            if (Exceptions.getFirstThrowableOfType(e, IllegalArgumentException.class) == null) throw e;
+            Asserts.expectedFailure(e);
         }
     }
 
@@ -121,9 +121,9 @@ public class DynamicClusterTest extends BrooklynAppUnitTestSupport {
         DynamicCluster c = app.createAndManageChild(EntitySpec.create(DynamicCluster.class));
         try {
             c.start(ImmutableList.of(loc));
-            fail();
+            Asserts.shouldHaveFailedPreviously();
         } catch (Exception e) {
-            if (Exceptions.getFirstThrowableOfType(e, IllegalStateException.class) == null) throw e;
+            Asserts.expectedFailureOfType(e, IllegalStateException.class);
         }
     }
     
@@ -135,9 +135,9 @@ public class DynamicClusterTest extends BrooklynAppUnitTestSupport {
             cluster.start(ImmutableList.of(loc));
             cluster.stop();
             cluster.start(ImmutableList.of(loc2));
-            fail();
+            Asserts.shouldHaveFailedPreviously();
         } catch (Exception e) {
-            if (Exceptions.getFirstThrowableOfType(e, IllegalStateException.class) == null) throw e;
+            Asserts.expectedFailureOfType(e, IllegalStateException.class);
         }
     }
 
@@ -147,9 +147,9 @@ public class DynamicClusterTest extends BrooklynAppUnitTestSupport {
                 .configure(DynamicCluster.MEMBER_SPEC, EntitySpec.create(TestEntity.class)));
         try {
             cluster.start(ImmutableList.of(loc, loc2));
-            fail();
+            Asserts.shouldHaveFailedPreviously();
         } catch (Exception e) {
-            if (Exceptions.getFirstThrowableOfType(e, IllegalArgumentException.class) == null) throw e;
+            Asserts.expectedFailureContainsIgnoreCase(e, "ambiguous");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ba8bb7f6/brooklyn-server/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SameServerEntityImpl.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SameServerEntityImpl.java b/brooklyn-server/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SameServerEntityImpl.java
index c24201a..8da252f 100644
--- a/brooklyn-server/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SameServerEntityImpl.java
+++ b/brooklyn-server/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SameServerEntityImpl.java
@@ -21,6 +21,7 @@ package org.apache.brooklyn.entity.software.base;
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import java.util.Collection;
+import java.util.List;
 
 import org.apache.brooklyn.api.location.Location;
 import org.apache.brooklyn.api.mgmt.Task;
@@ -28,6 +29,7 @@ import org.apache.brooklyn.core.entity.AbstractEntity;
 import org.apache.brooklyn.core.entity.Entities;
 import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
 import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.ComputeServiceIndicatorsFromChildrenAndMembers;
+import org.apache.brooklyn.core.location.Locations;
 import org.apache.brooklyn.entity.software.base.lifecycle.MachineLifecycleEffectorTasks;
 import org.apache.brooklyn.util.collections.QuorumCheck;
 import org.apache.brooklyn.util.core.config.ConfigBag;
@@ -68,7 +70,10 @@ public class SameServerEntityImpl extends AbstractEntity implements SameServerEn
      * Subclasses should override {@link #doStart} to customise behaviour.
      */
     @Override
-    public final void start(final Collection<? extends Location> locations) {
+    public final void start(Collection<? extends Location> locsO) {
+        addLocations(locsO);
+        final Collection<? extends Location> locations = Locations.getLocationsCheckingAncestors(locsO, this);
+        
         checkNotNull(locations, "locations");
         if (DynamicTasks.getTaskQueuingContext() != null) {
             doStart(locations);

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ba8bb7f6/brooklyn-server/test-framework/src/main/java/org/apache/brooklyn/test/framework/ParallelTestCaseImpl.java
----------------------------------------------------------------------
diff --git a/brooklyn-server/test-framework/src/main/java/org/apache/brooklyn/test/framework/ParallelTestCaseImpl.java b/brooklyn-server/test-framework/src/main/java/org/apache/brooklyn/test/framework/ParallelTestCaseImpl.java
index 469bc3d..3ded474 100644
--- a/brooklyn-server/test-framework/src/main/java/org/apache/brooklyn/test/framework/ParallelTestCaseImpl.java
+++ b/brooklyn-server/test-framework/src/main/java/org/apache/brooklyn/test/framework/ParallelTestCaseImpl.java
@@ -49,7 +49,7 @@ public class ParallelTestCaseImpl extends TargetableTestComponentImpl implements
         try {
             // Get an unsubmitted task for starting all the children of this entity in parallel,
             // at the same location as this entity.
-            final TaskAdaptable<?> taskAdaptable = StartableMethods.startingChildren(this);
+            final TaskAdaptable<?> taskAdaptable = StartableMethods.startingChildren(this, locations);
             logger.trace("{}, TaskAdaptable: {}", this, taskAdaptable);
 
             // Submit the task to the ExecutionManager so that they actually get started