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 2014/08/30 01:01:13 UTC

[13/26] git commit: add SERVICE_PROBLEMS and enrichers in ServiceStateLogic to compute state from that. add enrichers to populate PROBLEMS and NOT_UP_INDICATORS from children and members: this removes a lot of ad hoc service-up computations; some quorum

add SERVICE_PROBLEMS and enrichers in ServiceStateLogic to compute state from that.
add enrichers to populate PROBLEMS and NOT_UP_INDICATORS from children and members: this removes a lot of ad hoc service-up computations; some quorum up logic e.g. for mongo, have changed, but in those cases the previous logic was quite hokey in any case. (to properly support QUORUM_SIZE we should have a key, or use the UP_QUORUM_CHECK config).
also tweak the recent workaround when AbstractEntity.add{Enricher,Policy} detects a duplicate, we now take the more recent one rather than the older one; this allows us to replace them in a principled fashion when using unique tags.


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

Branch: refs/heads/master
Commit: 97eed6bdc159ddcba231fc0183635317101ec7b9
Parents: bc1d19f
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Mon Aug 25 07:23:01 2014 +0100
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Wed Aug 27 02:17:17 2014 -0400

----------------------------------------------------------------------
 .../basic/AbstractBrooklynObjectSpec.java       |  14 +-
 api/src/main/java/brooklyn/entity/Entity.java   |   4 +-
 api/src/main/java/brooklyn/entity/Group.java    |   1 +
 .../brooklyn/entity/proxying/EntitySpec.java    |   7 +-
 .../java/brooklyn/location/LocationSpec.java    |   2 +-
 .../main/java/brooklyn/policy/EnricherSpec.java |  70 ++-
 .../main/java/brooklyn/policy/PolicySpec.java   |   2 +-
 .../main/java/brooklyn/enricher/Enrichers.java  |   3 +
 .../enricher/basic/AbstractAggregator.java      | 215 +++++++++
 .../basic/AbstractMultipleSensorAggregator.java | 144 ++++++
 .../brooklyn/enricher/basic/Aggregator.java     | 158 ++-----
 .../brooklyn/enricher/basic/Propagator.java     |  12 +-
 .../entity/basic/AbstractApplication.java       |  36 +-
 .../brooklyn/entity/basic/AbstractEntity.java   |  84 ++--
 .../brooklyn/entity/basic/AbstractGroup.java    |  10 +
 .../entity/basic/AbstractGroupImpl.java         |  10 +
 .../java/brooklyn/entity/basic/Attributes.java  |  19 +-
 .../java/brooklyn/entity/basic/ConfigKeys.java  |   4 +
 .../brooklyn/entity/basic/DynamicGroup.java     |   1 +
 .../brooklyn/entity/basic/DynamicGroupImpl.java |   2 +-
 .../brooklyn/entity/basic/EntityFunctions.java  |  55 +--
 .../java/brooklyn/entity/basic/Lifecycle.java   |  73 ++-
 .../java/brooklyn/entity/basic/QuorumCheck.java |  74 ++++
 .../entity/basic/ServiceStateLogic.java         | 440 +++++++++++++++++++
 .../entity/basic/ServiceStatusLogic.java        |  92 ----
 .../brooklyn/entity/group/DynamicCluster.java   |   2 +-
 .../entity/group/DynamicClusterImpl.java        |  82 ++--
 .../brooklyn/entity/group/DynamicFabric.java    |   2 +-
 .../entity/group/DynamicFabricImpl.java         |  13 +-
 .../brooklyn/event/basic/BasicConfigKey.java    |   6 +-
 .../event/basic/DependentConfiguration.java     |   4 +-
 .../policy/basic/AbstractEntityAdjunct.java     |   1 +
 .../enricher/basic/BasicEnricherTest.java       |  14 +-
 .../basic/DependentConfigurationTest.java       |   4 +-
 .../brooklyn/entity/basic/DynamicGroupTest.java |   2 -
 .../brooklyn/entity/basic/EntitySpecTest.java   |   5 +-
 .../entity/basic/PolicyRegistrationTest.java    |   6 +-
 .../entity/group/DynamicClusterTest.java        |   5 +-
 .../entity/rebind/RebindEnricherTest.java       |   4 +-
 .../BrooklynMementoPersisterTestFixture.java    |   2 +-
 .../brooklyn/event/feed/http/HttpFeedTest.java  |   5 +-
 .../EntityCleanupLongevityTestFixture.java      |   2 +-
 .../brooklyn/test/entity/TestEntityImpl.java    |   2 +-
 .../test/entity/TestEntityNoEnrichersImpl.java  |  32 ++
 .../brooklyn/demo/CumulusRDFApplication.java    |  12 +-
 .../demo/WebClusterDatabaseExample.java         |   2 +
 .../demo/WebClusterDatabaseExampleApp.java      |   2 +
 .../basic/AbstractSoftwareProcessDriver.java    |   6 +-
 .../brooklyn/entity/basic/SameServerEntity.java |   2 +-
 .../brooklyn/entity/basic/SoftwareProcess.java  |   2 +-
 ...wareProcessDriverLifecycleEffectorTasks.java |   3 +-
 .../entity/basic/SoftwareProcessImpl.java       |  36 +-
 .../brooklynnode/BrooklynEntityMirrorImpl.java  |  11 +-
 .../software/MachineLifecycleEffectorTasks.java |  30 +-
 .../basic/lifecycle/ScriptHelperTest.java       |   1 +
 .../entity/java/VanillaJavaAppTest.java         |   4 +-
 .../entity/messaging/jms/JMSBrokerImpl.java     |  13 +-
 .../entity/zookeeper/ZooKeeperEnsembleImpl.java |  22 +-
 .../cassandra/CassandraDatacenterImpl.java      |  21 +-
 .../nosql/cassandra/CassandraFabricImpl.java    |   6 +-
 .../nosql/couchbase/CouchbaseClusterImpl.java   |  40 +-
 .../nosql/couchdb/CouchDBClusterImpl.java       |  21 +-
 .../elasticsearch/ElasticSearchClusterImpl.java |  10 -
 .../nosql/mongodb/MongoDBReplicaSetImpl.java    |   3 +-
 .../MongoDBConfigServerClusterImpl.java         |  16 +-
 .../entity/nosql/riak/RiakClusterImpl.java      |  10 +-
 .../entity/proxy/AbstractControllerImpl.java    |   2 +-
 .../entity/proxy/LoadBalancerClusterImpl.java   |  31 --
 .../entity/proxy/nginx/NginxControllerImpl.java |   6 +-
 .../entity/proxy/nginx/NginxSshDriver.java      |   2 +-
 .../webapp/ControlledDynamicWebAppCluster.java  |   2 +-
 .../ControlledDynamicWebAppClusterImpl.java     |  71 +--
 .../entity/webapp/DynamicWebAppClusterImpl.java |  41 --
 .../entity/webapp/jetty/Jetty6ServerImpl.java   |   2 +-
 .../proxy/nginx/NginxRebindIntegrationTest.java |   3 +-
 .../ControlledDynamicWebAppClusterTest.java     |   2 +-
 .../rest/transform/ApplicationTransformer.java  |   2 +-
 .../java/brooklyn/util/guava/Functionals.java   |  31 ++
 78 files changed, 1476 insertions(+), 717 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/api/src/main/java/brooklyn/basic/AbstractBrooklynObjectSpec.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/brooklyn/basic/AbstractBrooklynObjectSpec.java b/api/src/main/java/brooklyn/basic/AbstractBrooklynObjectSpec.java
index dc6642f..0efb0a6 100644
--- a/api/src/main/java/brooklyn/basic/AbstractBrooklynObjectSpec.java
+++ b/api/src/main/java/brooklyn/basic/AbstractBrooklynObjectSpec.java
@@ -33,17 +33,17 @@ public abstract class AbstractBrooklynObjectSpec<T,K extends AbstractBrooklynObj
 
     private static final long serialVersionUID = 3010955277740333030L;
     
-    private final Class<T> type;
+    private final Class<? extends T> type;
     private String displayName;
     private Set<Object> tags = MutableSet.of();
 
-    protected AbstractBrooklynObjectSpec(Class<T> type) {
+    protected AbstractBrooklynObjectSpec(Class<? extends T> type) {
         checkValidType(type);
         this.type = type;
     }
     
     @SuppressWarnings("unchecked")
-    protected final K self() {
+    protected K self() {
         return (K) this;
     }
 
@@ -52,7 +52,7 @@ public abstract class AbstractBrooklynObjectSpec<T,K extends AbstractBrooklynObj
         return Objects.toStringHelper(this).add("type", getType()).toString();
     }
 
-    protected abstract void checkValidType(Class<T> type);
+    protected abstract void checkValidType(Class<? extends T> type);
     
     public K displayName(String val) {
         displayName = val;
@@ -71,14 +71,14 @@ public abstract class AbstractBrooklynObjectSpec<T,K extends AbstractBrooklynObj
     }
 
     /**
-     * @return The type of the enricher
+     * @return The type of the object (or significant interface)
      */
-    public final Class<T> getType() {
+    public Class<? extends T> getType() {
         return type;
     }
     
     /**
-     * @return The display name of the enricher
+     * @return The display name of the object
      */
     public final String getDisplayName() {
         return displayName;

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/api/src/main/java/brooklyn/entity/Entity.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/brooklyn/entity/Entity.java b/api/src/main/java/brooklyn/entity/Entity.java
index bcc8a58..88a10fa 100644
--- a/api/src/main/java/brooklyn/entity/Entity.java
+++ b/api/src/main/java/brooklyn/entity/Entity.java
@@ -214,7 +214,7 @@ public interface Entity extends BrooklynObject {
     /**
      * Adds the given policy to this entity. Also calls policy.setEntity if available.
      */
-    Policy addPolicy(Policy policy);
+    void addPolicy(Policy policy);
     
     /**
      * Adds the given policy to this entity. Also calls policy.setEntity if available.
@@ -230,7 +230,7 @@ public interface Entity extends BrooklynObject {
     /**
      * Adds the given enricher to this entity. Also calls enricher.setEntity if available.
      */
-    Enricher addEnricher(Enricher enricher);
+    void addEnricher(Enricher enricher);
     
     /**
      * Adds the given enricher to this entity. Also calls enricher.setEntity if available.

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/api/src/main/java/brooklyn/entity/Group.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/brooklyn/entity/Group.java b/api/src/main/java/brooklyn/entity/Group.java
index f99b385..f2cd0b8 100644
--- a/api/src/main/java/brooklyn/entity/Group.java
+++ b/api/src/main/java/brooklyn/entity/Group.java
@@ -30,6 +30,7 @@ import brooklyn.entity.proxying.EntitySpec;
  * or dynamic (i.e. contains all entities that match some filter).
  */
 public interface Group extends Entity {
+    
     /**
      * Return the entities that are members of this group.
      */

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/api/src/main/java/brooklyn/entity/proxying/EntitySpec.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/brooklyn/entity/proxying/EntitySpec.java b/api/src/main/java/brooklyn/entity/proxying/EntitySpec.java
index 7410266..4ed953a 100644
--- a/api/src/main/java/brooklyn/entity/proxying/EntitySpec.java
+++ b/api/src/main/java/brooklyn/entity/proxying/EntitySpec.java
@@ -151,8 +151,13 @@ public class EntitySpec<T extends Entity> extends AbstractBrooklynObjectSpec<T,E
         super(type);
     }
     
+    @SuppressWarnings("unchecked")
+    public Class<T> getType() {
+        return (Class<T>)super.getType();
+    }
+    
     @Override
-    protected void checkValidType(Class<T> type) {
+    protected void checkValidType(Class<? extends T> type) {
         // EntitySpec does nothing.  Other specs do check it's an implementation etc.
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/api/src/main/java/brooklyn/location/LocationSpec.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/brooklyn/location/LocationSpec.java b/api/src/main/java/brooklyn/location/LocationSpec.java
index f34d1a2..6cd26ab 100644
--- a/api/src/main/java/brooklyn/location/LocationSpec.java
+++ b/api/src/main/java/brooklyn/location/LocationSpec.java
@@ -83,7 +83,7 @@ public class LocationSpec<T extends Location> extends AbstractBrooklynObjectSpec
         super(type);
     }
      
-    protected void checkValidType(java.lang.Class<T> type) {
+    protected void checkValidType(Class<? extends T> type) {
         checkIsImplementation(type, Location.class);
         checkIsNewStyleImplementation(type);
     }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/api/src/main/java/brooklyn/policy/EnricherSpec.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/brooklyn/policy/EnricherSpec.java b/api/src/main/java/brooklyn/policy/EnricherSpec.java
index f2cfdd4..e2f6c22 100644
--- a/api/src/main/java/brooklyn/policy/EnricherSpec.java
+++ b/api/src/main/java/brooklyn/policy/EnricherSpec.java
@@ -56,7 +56,7 @@ public class EnricherSpec<T extends Enricher> extends AbstractBrooklynObjectSpec
      * 
      * @param type A {@link Enricher} class
      */
-    public static <T extends Enricher> EnricherSpec<T> create(Class<T> type) {
+    public static <T extends Enricher> EnricherSpec<T> create(Class<? extends T> type) {
         return new EnricherSpec<T>(type);
     }
     
@@ -68,18 +68,18 @@ public class EnricherSpec<T extends Enricher> extends AbstractBrooklynObjectSpec
      * @param config The spec's configuration (see {@link EnricherSpec#configure(Map)}).
      * @param type   An {@link Enricher} class
      */
-    public static <T extends Enricher> EnricherSpec<T> create(Map<?,?> config, Class<T> type) {
+    public static <T extends Enricher> EnricherSpec<T> create(Map<?,?> config, Class<? extends T> type) {
         return EnricherSpec.create(type).configure(config);
     }
     
     private final Map<String, Object> flags = Maps.newLinkedHashMap();
     private final Map<ConfigKey<?>, Object> config = Maps.newLinkedHashMap();
 
-    protected EnricherSpec(Class<T> type) {
+    protected EnricherSpec(Class<? extends T> type) {
         super(type);
     }
     
-    protected void checkValidType(Class<T> type) {
+    protected void checkValidType(Class<? extends T> type) {
         checkIsImplementation(type, Enricher.class);
         checkIsNewStyleImplementation(type);
     }
@@ -149,4 +149,66 @@ public class EnricherSpec<T extends Enricher> extends AbstractBrooklynObjectSpec
         return Collections.unmodifiableMap(config);
     }
 
+    public abstract static class ExtensibleEnricherSpec<T extends Enricher,K extends ExtensibleEnricherSpec<T,K>> extends EnricherSpec<T> {
+        private static final long serialVersionUID = -3649347642882809739L;
+        
+        protected ExtensibleEnricherSpec(Class<? extends T> type) {
+            super(type);
+        }
+
+        @SuppressWarnings("unchecked")
+        protected K self() {
+            // we override the AbstractBrooklynObjectSpec method -- it's a different K here because
+            // EnricherSpec does not contain a parametrisable generic return type (Self)
+            return (K) this;
+        }
+        
+        @Override
+        public K uniqueTag(String uniqueTag) {
+            super.uniqueTag(uniqueTag);
+            return self();
+        }
+
+        @Override
+        public K configure(Map<?, ?> val) {
+            super.configure(val);
+            return self();
+        }
+
+        @Override
+        public K configure(CharSequence key, Object val) {
+            super.configure(key, val);
+            return self();
+        }
+
+        @Override
+        public <V> K configure(ConfigKey<V> key, V val) {
+            super.configure(key, val);
+            return self();
+        }
+
+        @Override
+        public <V> K configureIfNotNull(ConfigKey<V> key, V val) {
+            super.configureIfNotNull(key, val);
+            return self();
+        }
+
+        @Override
+        public <V> K configure(ConfigKey<V> key, Task<? extends V> val) {
+            super.configure(key, val);
+            return self();
+        }
+
+        @Override
+        public <V> K configure(HasConfigKey<V> key, V val) {
+            super.configure(key, val);
+            return self();
+        }
+
+        @Override
+        public <V> K configure(HasConfigKey<V> key, Task<? extends V> val) {
+            super.configure(key, val);
+            return self();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/api/src/main/java/brooklyn/policy/PolicySpec.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/brooklyn/policy/PolicySpec.java b/api/src/main/java/brooklyn/policy/PolicySpec.java
index 13206cf..4c0665b 100644
--- a/api/src/main/java/brooklyn/policy/PolicySpec.java
+++ b/api/src/main/java/brooklyn/policy/PolicySpec.java
@@ -79,7 +79,7 @@ public class PolicySpec<T extends Policy> extends AbstractBrooklynObjectSpec<T,P
         super(type);
     }
     
-    protected void checkValidType(Class<T> type) {
+    protected void checkValidType(Class<? extends T> type) {
         checkIsImplementation(type, Policy.class);
         checkIsNewStyleImplementation(type);
     }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/enricher/Enrichers.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/enricher/Enrichers.java b/core/src/main/java/brooklyn/enricher/Enrichers.java
index 23b1b83..73b7a3b 100644
--- a/core/src/main/java/brooklyn/enricher/Enrichers.java
+++ b/core/src/main/java/brooklyn/enricher/Enrichers.java
@@ -125,6 +125,9 @@ public class Enrichers {
         public PropagatorBuilder propagatingAll() {
             return new PropagatorBuilder(true, null);
         }
+        public PropagatorBuilder propagatingAllButUsualAnd(Sensor<?>... vals) {
+            return new PropagatorBuilder(true, ImmutableSet.<Sensor<?>>builder().addAll(Propagator.SENSORS_NOT_USUALLY_PROPAGATED).add(vals).build());
+        }
         public PropagatorBuilder propagatingAllBut(Sensor<?>... vals) {
             return new PropagatorBuilder(true, ImmutableSet.copyOf(vals));
         }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/enricher/basic/AbstractAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/enricher/basic/AbstractAggregator.java b/core/src/main/java/brooklyn/enricher/basic/AbstractAggregator.java
new file mode 100644
index 0000000..ba7762c
--- /dev/null
+++ b/core/src/main/java/brooklyn/enricher/basic/AbstractAggregator.java
@@ -0,0 +1,215 @@
+/*
+ * 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 brooklyn.enricher.basic;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import brooklyn.config.ConfigKey;
+import brooklyn.entity.Entity;
+import brooklyn.entity.Group;
+import brooklyn.entity.basic.AbstractEntity;
+import brooklyn.entity.basic.ConfigKeys;
+import brooklyn.entity.basic.EntityLocal;
+import brooklyn.entity.trait.Changeable;
+import brooklyn.event.Sensor;
+import brooklyn.event.SensorEvent;
+import brooklyn.event.SensorEventListener;
+import brooklyn.util.exceptions.Exceptions;
+import brooklyn.util.guava.Maybe;
+
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.Iterables;
+import com.google.common.reflect.TypeToken;
+
+/** Abstract superclass for enrichers which aggregate from children and/or members */
+@SuppressWarnings("serial")
+public abstract class AbstractAggregator<T,U> extends AbstractEnricher implements SensorEventListener<T> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractAggregator.class);
+
+    public static final ConfigKey<Entity> PRODUCER = ConfigKeys.newConfigKey(Entity.class, "enricher.producer", "The entity whose children/members will be aggregated");
+
+    public static final ConfigKey<Sensor<?>> TARGET_SENSOR = ConfigKeys.newConfigKey(new TypeToken<Sensor<?>>() {}, "enricher.targetSensor");
+
+    // FIXME this is not just for "members" i think -Alex
+    public static final ConfigKey<?> DEFAULT_MEMBER_VALUE = ConfigKeys.newConfigKey(Object.class, "enricher.defaultMemberValue");
+
+    public static final ConfigKey<Set<? extends Entity>> FROM_HARDCODED_PRODUCERS = ConfigKeys.newConfigKey(new TypeToken<Set<? extends Entity>>() {}, "enricher.aggregating.fromHardcodedProducers");
+
+    public static final ConfigKey<Boolean> FROM_MEMBERS = ConfigKeys.newBooleanConfigKey("enricher.aggregating.fromMembers");
+
+    public static final ConfigKey<Boolean> FROM_CHILDREN = ConfigKeys.newBooleanConfigKey("enricher.aggregating.fromChildren");
+
+    public static final ConfigKey<Predicate<? super Entity>> ENTITY_FILTER = ConfigKeys.newConfigKey(new TypeToken<Predicate<? super Entity>>() {}, "enricher.aggregating.entityFilter");
+
+    public static final ConfigKey<Predicate<?>> VALUE_FILTER = ConfigKeys.newConfigKey(new TypeToken<Predicate<?>>() {}, "enricher.aggregating.valueFilter");
+
+    protected Entity producer;
+    protected Sensor<U> targetSensor;
+    protected T defaultMemberValue;
+    protected Set<? extends Entity> fromHardcodedProducers;
+    protected Boolean fromMembers;
+    protected Boolean fromChildren;
+    protected Predicate<? super Entity> entityFilter;
+    protected Predicate<? super T> valueFilter;
+    
+    public AbstractAggregator() {}
+
+    @Override
+    public void setEntity(EntityLocal entity) {
+        super.setEntity(entity);
+        setEntityLoadingConfig();
+
+        if (fromHardcodedProducers == null && producer == null) producer = entity;
+        checkState(fromHardcodedProducers != null ^ producer != null, "must specify one of %s (%s) or %s (%s)", 
+                PRODUCER.getName(), producer, FROM_HARDCODED_PRODUCERS.getName(), fromHardcodedProducers);
+        checkState(producer != null ? (Boolean.TRUE.equals(fromMembers) || Boolean.TRUE.equals(fromChildren)) : true, 
+                "when specifying producer, must specify at least one of fromMembers (%s) or fromChildren (%s)", fromMembers, fromChildren);
+
+        if (fromHardcodedProducers != null) {
+            for (Entity producer : Iterables.filter(fromHardcodedProducers, entityFilter)) {
+                addProducerHardcoded(producer);
+            }
+        }
+        
+        if (Boolean.TRUE.equals(fromMembers)) {
+            setEntityBeforeSubscribingProducerMemberEvents(entity);
+            setEntitySubscribeProducerMemberEvents();
+            setEntityAfterSubscribingProducerMemberEvents();
+        }
+        
+        if (Boolean.TRUE.equals(fromChildren)) {
+            setEntityBeforeSubscribingProducerChildrenEvents();
+            setEntitySubscribingProducerChildrenEvents();
+            setEntityAfterSubscribingProducerChildrenEvents();
+        }
+        
+        onUpdated();
+    }
+
+    @SuppressWarnings({ "unchecked" })
+    protected void setEntityLoadingConfig() {
+        this.producer = getConfig(PRODUCER);
+        this.fromHardcodedProducers= getConfig(FROM_HARDCODED_PRODUCERS);
+        this.defaultMemberValue = (T) getConfig(DEFAULT_MEMBER_VALUE);
+        this.fromMembers = Maybe.fromNullable(getConfig(FROM_MEMBERS)).or(fromMembers);
+        this.fromChildren = Maybe.fromNullable(getConfig(FROM_CHILDREN)).or(fromChildren);
+        this.entityFilter = (Predicate<? super Entity>) (getConfig(ENTITY_FILTER) == null ? Predicates.alwaysTrue() : getConfig(ENTITY_FILTER));
+        this.valueFilter = (Predicate<? super T>) (getConfig(VALUE_FILTER) == null ? Predicates.alwaysTrue() : getConfig(VALUE_FILTER));
+        
+        setEntityLoadingTargetConfig();
+    }
+    @SuppressWarnings({ "unchecked" })
+    protected void setEntityLoadingTargetConfig() {
+        this.targetSensor = (Sensor<U>) getRequiredConfig(TARGET_SENSOR);
+    }
+
+    protected void setEntityBeforeSubscribingProducerMemberEvents(EntityLocal entity) {
+        checkState(producer instanceof Group, "must be a group when fromMembers true: producer=%s; entity=%s; "
+                + "hardcodedProducers=%s", getConfig(PRODUCER), entity, fromHardcodedProducers);
+    }
+
+    protected void setEntitySubscribeProducerMemberEvents() {
+        subscribe(producer, Changeable.MEMBER_ADDED, new SensorEventListener<Entity>() {
+            @Override public void onEvent(SensorEvent<Entity> event) {
+                if (entityFilter.apply(event.getValue())) {
+                    addProducerMember(event.getValue());
+                    onUpdated();
+                }
+            }
+        });
+        subscribe(producer, Changeable.MEMBER_REMOVED, new SensorEventListener<Entity>() {
+            @Override public void onEvent(SensorEvent<Entity> event) {
+                removeProducer(event.getValue());
+                onUpdated();
+            }
+        });
+    }
+
+    protected void setEntityAfterSubscribingProducerMemberEvents() {
+        if (producer instanceof Group) {
+            for (Entity member : Iterables.filter(((Group)producer).getMembers(), entityFilter)) {
+                addProducerMember(member);
+            }
+        }
+    }
+
+    protected void setEntityBeforeSubscribingProducerChildrenEvents() {
+    }
+
+    protected void setEntitySubscribingProducerChildrenEvents() {
+        subscribe(producer, AbstractEntity.CHILD_REMOVED, new SensorEventListener<Entity>() {
+            @Override public void onEvent(SensorEvent<Entity> event) {
+                removeProducer(event.getValue());
+                onUpdated();
+            }
+        });
+        subscribe(producer, AbstractEntity.CHILD_ADDED, new SensorEventListener<Entity>() {
+            @Override public void onEvent(SensorEvent<Entity> event) {
+                if (entityFilter.apply(event.getValue())) {
+                    addProducerChild(event.getValue());
+                    onUpdated();
+                }
+            }
+        });
+    }
+
+    protected void setEntityAfterSubscribingProducerChildrenEvents() {
+        for (Entity child : Iterables.filter(producer.getChildren(), entityFilter)) {
+            addProducerChild(child);
+        }
+    }
+
+    protected abstract void addProducerHardcoded(Entity producer);
+    protected abstract void addProducerMember(Entity producer);
+    protected abstract void addProducerChild(Entity producer);
+    
+    // TODO If producer removed but then get (queued) event from it after this method returns,  
+    protected void removeProducer(Entity producer) {
+        if (LOG.isDebugEnabled()) LOG.debug("{} stopped listening to {}", new Object[] {this, producer });
+        unsubscribe(producer);
+        onProducerRemoved(producer);
+    }
+
+    protected abstract void onProducerAdded(Entity producer);
+
+    protected abstract void onProducerRemoved(Entity producer);
+
+
+    /**
+     * Called whenever the values for the set of producers changes (e.g. on an event, or on a member added/removed).
+     */
+    protected void onUpdated() {
+        try {
+            emit(targetSensor, compute());
+        } catch (Throwable t) {
+            LOG.warn("Error calculating and setting aggregate for enricher "+this, t);
+            throw Exceptions.propagate(t);
+        }
+    }
+
+    protected abstract Object compute();
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/enricher/basic/AbstractMultipleSensorAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/enricher/basic/AbstractMultipleSensorAggregator.java b/core/src/main/java/brooklyn/enricher/basic/AbstractMultipleSensorAggregator.java
new file mode 100644
index 0000000..85c36d8
--- /dev/null
+++ b/core/src/main/java/brooklyn/enricher/basic/AbstractMultipleSensorAggregator.java
@@ -0,0 +1,144 @@
+/*
+ * 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 brooklyn.enricher.basic;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import brooklyn.entity.Entity;
+import brooklyn.event.AttributeSensor;
+import brooklyn.event.Sensor;
+import brooklyn.event.SensorEvent;
+import brooklyn.event.SensorEventListener;
+import brooklyn.util.collections.MutableMap;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+/** Building on {@link AbstractAggregator} for a single source sensor (on multiple children and/or members) */
+public abstract class AbstractMultipleSensorAggregator<U> extends AbstractAggregator<Object,U> implements SensorEventListener<Object> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(AbstractMultipleSensorAggregator.class);
+
+    
+    /** access via {@link #getValues(Sensor)} */
+    private final Map<String, Map<Entity,Object>> values = Collections.synchronizedMap(new LinkedHashMap<String, Map<Entity,Object>>());
+
+    public AbstractMultipleSensorAggregator() {}
+
+    protected abstract Collection<Sensor<?>> getSourceSensors();
+    
+    @Override
+    protected void setEntityLoadingConfig() {
+        super.setEntityLoadingConfig();
+        Preconditions.checkNotNull(getSourceSensors(), "sourceSensors must be set");
+    }
+    
+    protected void setEntityBeforeSubscribingProducerChildrenEvents() {
+        if (LOG.isDebugEnabled()) LOG.debug("{} subscribing to children of {}", new Object[] {this, producer });
+        for (Sensor<?> sourceSensor: getSourceSensors()) {
+            subscribeToChildren(producer, sourceSensor, this);
+        }
+    }
+
+    protected void addProducerHardcoded(Entity producer) {
+        for (Sensor<?> sourceSensor: getSourceSensors()) {
+            subscribe(producer, sourceSensor, this);
+        }
+        onProducerAdded(producer);
+    }
+
+    protected void addProducerChild(Entity producer) {
+        // not required due to subscribeToChildren call
+//        subscribe(producer, sourceSensor, this);
+        onProducerAdded(producer);
+    }
+
+    protected void addProducerMember(Entity producer) {
+        addProducerHardcoded(producer);
+    }
+
+    
+    protected void onProducerAdded(Entity producer) {
+        if (LOG.isDebugEnabled()) LOG.debug("{} listening to {}", new Object[] {this, producer});
+        synchronized (values) {
+            for (Sensor<?> sensor: getSourceSensors()) {
+                Map<Entity,Object> vs = values.get(sensor.getName());
+                if (vs==null) {
+                    vs = new LinkedHashMap<Entity,Object>();
+                    values.put(sensor.getName(), vs);
+                }
+                
+                Object vo = vs.get(producer);
+                if (vo==null) {
+                    Object initialVal;
+                    if (sensor instanceof AttributeSensor) {
+                        initialVal = producer.getAttribute((AttributeSensor<?>)sensor);
+                    } else {
+                        initialVal = null;
+                    }
+                    vs.put(producer, initialVal != null ? initialVal : defaultMemberValue);
+                    // NB: see notes on possible race, in Aggregator#onProducerAdded
+                }
+                
+            }
+        }
+    }
+    
+    protected void onProducerRemoved(Entity producer) {
+        synchronized (values) {
+            for (Sensor<?> sensor: getSourceSensors()) {
+                Map<Entity,Object> vs = values.get(sensor.getName());
+                if (vs!=null)
+                    vs.remove(producer);
+            }
+        }
+        onUpdated();
+    }
+
+    @Override
+    public void onEvent(SensorEvent<Object> event) {
+        Entity e = event.getSource();
+        synchronized (values) {
+            Map<Entity,Object> vs = values.get(event.getSensor().getName());
+            if (vs==null) {
+                LOG.warn("{} has no entry for sensor on "+event);
+            } else {
+                vs.put(e, event.getValue());
+            }
+        }
+        onUpdated();
+    }
+
+    @SuppressWarnings("unchecked")
+    public <T> Map<Entity,T> getValues(Sensor<T> sensor) {
+        synchronized (values) {
+            Map<Entity, T> sv = (Map<Entity, T>) values.get(sensor.getName());
+            if (sv==null) return ImmutableMap.of();
+            return MutableMap.copyOf(sv).asUnmodifiable();
+        }
+    }
+    
+    protected abstract Object compute();
+}

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/enricher/basic/Aggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/enricher/basic/Aggregator.java b/core/src/main/java/brooklyn/enricher/basic/Aggregator.java
index fed403e..034a604 100644
--- a/core/src/main/java/brooklyn/enricher/basic/Aggregator.java
+++ b/core/src/main/java/brooklyn/enricher/basic/Aggregator.java
@@ -18,25 +18,18 @@
  */
 package brooklyn.enricher.basic;
 
-import static com.google.common.base.Preconditions.checkState;
-
 import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import brooklyn.config.ConfigKey;
 import brooklyn.entity.Entity;
-import brooklyn.entity.Group;
-import brooklyn.entity.basic.AbstractEntity;
 import brooklyn.entity.basic.ConfigKeys;
-import brooklyn.entity.basic.EntityLocal;
-import brooklyn.entity.trait.Changeable;
 import brooklyn.event.AttributeSensor;
 import brooklyn.event.Sensor;
 import brooklyn.event.SensorEvent;
@@ -46,46 +39,20 @@ import brooklyn.util.collections.MutableMap;
 import brooklyn.util.exceptions.Exceptions;
 
 import com.google.common.base.Function;
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
 import com.google.common.collect.Iterables;
 import com.google.common.reflect.TypeToken;
 
+/** Building on {@link AbstractAggregator} for a single source sensor (on multiple children and/or members) */
 @SuppressWarnings("serial")
-public class Aggregator<T,U> extends AbstractEnricher implements SensorEventListener<T> {
+public class Aggregator<T,U> extends AbstractAggregator<T,U> implements SensorEventListener<T> {
 
     private static final Logger LOG = LoggerFactory.getLogger(Aggregator.class);
 
-    public static final ConfigKey<Function<? super Collection<?>, ?>> TRANSFORMATION = ConfigKeys.newConfigKey(new TypeToken<Function<? super Collection<?>, ?>>() {}, "enricher.transformation");
-
-    public static final ConfigKey<Entity> PRODUCER = ConfigKeys.newConfigKey(Entity.class, "enricher.producer");
-
     public static final ConfigKey<Sensor<?>> SOURCE_SENSOR = ConfigKeys.newConfigKey(new TypeToken<Sensor<?>>() {}, "enricher.sourceSensor");
+    public static final ConfigKey<Function<? super Collection<?>, ?>> TRANSFORMATION = ConfigKeys.newConfigKey(new TypeToken<Function<? super Collection<?>, ?>>() {}, "enricher.transformation");
 
-    public static final ConfigKey<Sensor<?>> TARGET_SENSOR = ConfigKeys.newConfigKey(new TypeToken<Sensor<?>>() {}, "enricher.targetSensor");
-
-    public static final ConfigKey<?> DEFAULT_MEMBER_VALUE = ConfigKeys.newConfigKey(Object.class, "enricher.defaultMemberValue");
-
-    public static final ConfigKey<Set<? extends Entity>> FROM_HARDCODED_PRODUCERS = ConfigKeys.newConfigKey(new TypeToken<Set<? extends Entity>>() {}, "enricher.aggregating.fromHardcodedProducers");
-
-    public static final ConfigKey<Boolean> FROM_MEMBERS = ConfigKeys.newBooleanConfigKey("enricher.aggregating.fromMembers");
-
-    public static final ConfigKey<Boolean> FROM_CHILDREN = ConfigKeys.newBooleanConfigKey("enricher.aggregating.fromChildren");
-
-    public static final ConfigKey<Predicate<? super Entity>> ENTITY_FILTER = ConfigKeys.newConfigKey(new TypeToken<Predicate<? super Entity>>() {}, "enricher.aggregating.entityFilter");
-
-    public static final ConfigKey<Predicate<?>> VALUE_FILTER = ConfigKeys.newConfigKey(new TypeToken<Predicate<?>>() {}, "enricher.aggregating.valueFilter");
-
-    protected Function<? super Collection<T>, ? extends U> transformation;
-    protected Entity producer;
     protected Sensor<T> sourceSensor;
-    protected Sensor<U> targetSensor;
-    protected T defaultMemberValue;
-    protected Set<? extends Entity> fromHardcodedProducers;
-    protected Boolean fromMembers;
-    protected Boolean fromChildren;
-    protected Predicate<? super Entity> entityFilter;
-    protected Predicate<? super T> valueFilter;
+    protected Function<? super Collection<T>, ? extends U> transformation;
     
     /**
      * Users of values should either on it synchronize when iterating over its entries or use
@@ -94,100 +61,40 @@ public class Aggregator<T,U> extends AbstractEnricher implements SensorEventList
     // We use a synchronizedMap over a ConcurrentHashMap for entities that store null values.
     protected final Map<Entity, T> values = Collections.synchronizedMap(new LinkedHashMap<Entity, T>());
 
-    public Aggregator() {
-    }
+    public Aggregator() {}
 
-    @SuppressWarnings({ "unchecked" })
-    @Override
-    public void setEntity(EntityLocal entity) {
-        super.setEntity(entity);
-        this.transformation = (Function<? super Collection<T>, ? extends U>) getRequiredConfig(TRANSFORMATION);
-        this.producer = getConfig(PRODUCER);
-        this.fromHardcodedProducers= getConfig(FROM_HARDCODED_PRODUCERS);
+    @SuppressWarnings("unchecked")
+    protected void setEntityLoadingConfig() {
+        super.setEntityLoadingConfig();
         this.sourceSensor = (Sensor<T>) getRequiredConfig(SOURCE_SENSOR);
-        this.targetSensor = (Sensor<U>) getRequiredConfig(TARGET_SENSOR);
-        this.defaultMemberValue = (T) getConfig(DEFAULT_MEMBER_VALUE);
-        this.fromMembers = getConfig(FROM_MEMBERS);
-        this.fromChildren = getConfig(FROM_CHILDREN);
-        this.entityFilter = (Predicate<? super Entity>) (getConfig(ENTITY_FILTER) == null ? Predicates.alwaysTrue() : getConfig(ENTITY_FILTER));
-        this.valueFilter = (Predicate<? super T>) (getConfig(VALUE_FILTER) == null ? Predicates.alwaysTrue() : getConfig(VALUE_FILTER));
-
-        if (fromHardcodedProducers == null && producer == null) producer = entity;
-        checkState(fromHardcodedProducers != null ^ producer != null, "must specify one of %s (%s) or %s (%s)", 
-                PRODUCER.getName(), producer, FROM_HARDCODED_PRODUCERS.getName(), fromHardcodedProducers);
-        checkState(producer != null ? (Boolean.TRUE.equals(fromMembers) ^ Boolean.TRUE.equals(fromChildren)) : true, 
-                "when specifying producer, must specify one of fromMembers (%s) or fromChildren (%s)", fromMembers, fromChildren);
-
-        if (fromHardcodedProducers != null) {
-            for (Entity producer : Iterables.filter(fromHardcodedProducers, entityFilter)) {
-                addProducer(producer);
-            }
-            onUpdated();
-        }
-        
-        if (Boolean.TRUE.equals(fromMembers)) {
-            checkState(producer instanceof Group, "must be a group when fromMembers true: producer=%s; entity=%s; "
-                    + "hardcodedProducers=%s", getConfig(PRODUCER), entity, fromHardcodedProducers);
-
-            subscribe(producer, Changeable.MEMBER_ADDED, new SensorEventListener<Entity>() {
-                @Override public void onEvent(SensorEvent<Entity> event) {
-                    if (entityFilter.apply(event.getValue())) addProducer(event.getValue());
-                }
-            });
-            subscribe(producer, Changeable.MEMBER_REMOVED, new SensorEventListener<Entity>() {
-                @Override public void onEvent(SensorEvent<Entity> event) {
-                    removeProducer(event.getValue());
-                }
-            });
-            
-            if (producer instanceof Group) {
-                for (Entity member : Iterables.filter(((Group)producer).getMembers(), entityFilter)) {
-                    addProducer(member);
-                }
-            }
-            onUpdated();
-        }
+        this.transformation = (Function<? super Collection<T>, ? extends U>) getRequiredConfig(TRANSFORMATION);
+    }
         
-        if (Boolean.TRUE.equals(fromChildren)) {
-            if (LOG.isDebugEnabled()) LOG.debug("{} linked (children of {}, {}) to {}", new Object[] {this, producer, sourceSensor, targetSensor});
-            subscribeToChildren(producer, sourceSensor, this);
-
-            subscribe(producer, AbstractEntity.CHILD_REMOVED, new SensorEventListener<Entity>() {
-                @Override public void onEvent(SensorEvent<Entity> event) {
-                    onProducerRemoved(event.getValue());
-                }
-            });
-            subscribe(producer, AbstractEntity.CHILD_ADDED, new SensorEventListener<Entity>() {
-                @Override public void onEvent(SensorEvent<Entity> event) {
-                    if (entityFilter.apply(event.getValue())) onProducerAdded(event.getValue());
-                }
-            });
 
-            for (Entity child : Iterables.filter(producer.getChildren(), entityFilter)) {
-                onProducerAdded(child, false);
-            }
-            onUpdated();
-        }
+    protected void setEntityBeforeSubscribingProducerChildrenEvents() {
+        if (LOG.isDebugEnabled()) LOG.debug("{} subscribing to children of {}", new Object[] {this, producer });
+        subscribeToChildren(producer, sourceSensor, this);
     }
 
-    protected void addProducer(Entity producer) {
-        if (LOG.isDebugEnabled()) LOG.debug("{} linked ({}, {}) to {}", new Object[] {this, producer, sourceSensor, targetSensor});
+    protected void addProducerHardcoded(Entity producer) {
         subscribe(producer, sourceSensor, this);
         onProducerAdded(producer);
     }
-    
-    // TODO If producer removed but then get (queued) event from it after this method returns,  
-    protected T removeProducer(Entity producer) {
-        if (LOG.isDebugEnabled()) LOG.debug("{} unlinked ({}, {}) from {}", new Object[] {this, producer, sourceSensor, targetSensor});
-        unsubscribe(producer);
-        return onProducerRemoved(producer);
+
+    protected void addProducerChild(Entity producer) {
+        // not required due to subscribeToChildren call
+//        subscribe(producer, sourceSensor, this);
+        onProducerAdded(producer);
     }
 
-    protected void onProducerAdded(Entity producer) {
-        onProducerAdded(producer, true);
+    protected void addProducerMember(Entity producer) {
+        subscribe(producer, sourceSensor, this);
+        onProducerAdded(producer);
     }
+
     
-    protected void onProducerAdded(Entity producer, boolean update) {
+    protected void onProducerAdded(Entity producer) {
+        if (LOG.isDebugEnabled()) LOG.debug("{} listening to {}", new Object[] {this, producer});
         synchronized (values) {
             T vo = values.get(producer);
             if (vo==null) {
@@ -206,18 +113,13 @@ public class Aggregator<T,U> extends AbstractEnricher implements SensorEventList
                 if (LOG.isDebugEnabled()) LOG.debug("{} already had value ({}) for producer ({}); but that producer has just been added", new Object[] {this, vo, producer});
             }
         }
-        if (update) {
-            onUpdated();
-        }
     }
     
-    // TODO If producer removed but then get (queued) event from it after this method returns,  
-    protected T onProducerRemoved(Entity producer) {
-        T removed = values.remove(producer);
+    protected void onProducerRemoved(Entity producer) {
+        values.remove(producer);
         onUpdated();
-        return removed;
     }
-    
+
     @Override
     public void onEvent(SensorEvent<T> event) {
         Entity e = event.getSource();
@@ -231,9 +133,6 @@ public class Aggregator<T,U> extends AbstractEnricher implements SensorEventList
         onUpdated();
     }
 
-    /**
-     * Called whenever the values for the set of producers changes (e.g. on an event, or on a member added/removed).
-     */
     protected void onUpdated() {
         try {
             emit(targetSensor, compute());
@@ -257,4 +156,5 @@ public class Aggregator<T,U> extends AbstractEnricher implements SensorEventList
             return Collections.unmodifiableMap(MutableMap.copyOf(values));
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/enricher/basic/Propagator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/enricher/basic/Propagator.java b/core/src/main/java/brooklyn/enricher/basic/Propagator.java
index 267ba88..7aca9ca 100644
--- a/core/src/main/java/brooklyn/enricher/basic/Propagator.java
+++ b/core/src/main/java/brooklyn/enricher/basic/Propagator.java
@@ -18,16 +18,16 @@
  */
 package brooklyn.enricher.basic;
 
-import static com.google.common.base.Preconditions.checkState;
-
 import java.util.Collection;
 import java.util.Map;
+import java.util.Set;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import brooklyn.config.ConfigKey;
 import brooklyn.entity.Entity;
+import brooklyn.entity.basic.Attributes;
 import brooklyn.entity.basic.ConfigKeys;
 import brooklyn.entity.basic.EntityLocal;
 import brooklyn.event.AttributeSensor;
@@ -36,9 +36,11 @@ import brooklyn.event.SensorEvent;
 import brooklyn.event.SensorEventListener;
 import brooklyn.util.flags.SetFromFlag;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 import com.google.common.reflect.TypeToken;
@@ -48,6 +50,10 @@ public class Propagator extends AbstractEnricher implements SensorEventListener<
 
     private static final Logger LOG = LoggerFactory.getLogger(Propagator.class);
 
+    public static final Set<Sensor<?>> SENSORS_NOT_USUALLY_PROPAGATED = ImmutableSet.<Sensor<?>>of(
+        Attributes.SERVICE_UP, Attributes.SERVICE_NOT_UP_INDICATORS, 
+        Attributes.SERVICE_STATE_ACTUAL, Attributes.SERVICE_STATE_EXPECTED, Attributes.SERVICE_PROBLEMS);
+
     @SetFromFlag("producer")
     public static ConfigKey<Entity> PRODUCER = ConfigKeys.newConfigKey(Entity.class, "enricher.producer");
 
@@ -112,7 +118,7 @@ public class Propagator extends AbstractEnricher implements SensorEventListener<
             };
         }
             
-        checkState(propagatingAll ^ sensorMapping.size() > 0,
+        Preconditions.checkState(propagatingAll ^ sensorMapping.size() > 0,
                 "Exactly one must be set of propagatingAll (%s, excluding %s), sensorMapping (%s)", propagatingAll, getConfig(PROPAGATING_ALL_BUT), sensorMapping);
 
         if (propagatingAll) {

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/AbstractApplication.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/AbstractApplication.java b/core/src/main/java/brooklyn/entity/basic/AbstractApplication.java
index b7edf59..da33208 100644
--- a/core/src/main/java/brooklyn/entity/basic/AbstractApplication.java
+++ b/core/src/main/java/brooklyn/entity/basic/AbstractApplication.java
@@ -27,6 +27,8 @@ import org.slf4j.LoggerFactory;
 import brooklyn.config.BrooklynProperties;
 import brooklyn.entity.Application;
 import brooklyn.entity.Entity;
+import brooklyn.entity.basic.ServiceStateLogic.ServiceNotUpLogic;
+import brooklyn.entity.basic.ServiceStateLogic.ServiceProblemsLogic;
 import brooklyn.entity.trait.StartableMethods;
 import brooklyn.location.Location;
 import brooklyn.management.ManagementContext;
@@ -75,11 +77,6 @@ public abstract class AbstractApplication extends AbstractEntity implements Star
     }
 
     @Override
-    public void init() {
-        log.warn("Deprecated: AbstractApplication.init() will be declared abstract in a future release; please override (without calling super) for code instantiating child entities");
-    }
-
-    @Override
     public Application getApplication() {
         if (application!=null) {
             if (application.getId().equals(getId()))
@@ -115,6 +112,15 @@ public abstract class AbstractApplication extends AbstractEntity implements Star
         return this;
     }
     
+    /** as {@link AbstractEntity#initEnrichers()} but also adding default service not-up and problem indicators from children */
+    @Override
+    protected void initEnrichers() {
+        super.initEnrichers();
+        
+        // default app logic; easily overridable by adding a different enricher with the same tag
+        ServiceStateLogic.newEnricherFromChildren().checkChildrenAndMembers().addTo(this);
+    }
+    
     /**
      * Default start will start all Startable children (child.start(Collection<? extends Location>)),
      * calling preStart(locations) first and postStart(locations) afterwards.
@@ -123,23 +129,25 @@ public abstract class AbstractApplication extends AbstractEntity implements Star
     public void start(Collection<? extends Location> locations) {
         this.addLocations(locations);
         Collection<? extends Location> locationsToUse = getLocations();
-        setAttribute(Attributes.SERVICE_STATE, Lifecycle.STARTING);
+        ServiceProblemsLogic.clearProblemsIndicator(this, START);
+        ServiceStateLogic.setExpectedState(this, Lifecycle.STARTING);
         recordApplicationEvent(Lifecycle.STARTING);
         try {
             preStart(locationsToUse);
             doStart(locationsToUse);
             postStart(locationsToUse);
         } catch (Exception e) {
-            setAttribute(Attributes.SERVICE_STATE, Lifecycle.ON_FIRE);
+            // TODO should probably remember these problems then clear?  if so, do it here or on all effectors?
+//            ServiceProblemsLogic.updateProblemsIndicator(this, START, e);
+            
             recordApplicationEvent(Lifecycle.ON_FIRE);
             // no need to log here; the effector invocation should do that
             throw Exceptions.propagate(e);
+        } finally {
+            ServiceStateLogic.setExpectedState(this, Lifecycle.RUNNING);
         }
-
-        setAttribute(SERVICE_UP, true);
-        setAttribute(Attributes.SERVICE_STATE, Lifecycle.RUNNING);
+        
         deployed = true;
-
         recordApplicationEvent(Lifecycle.RUNNING);
 
         logApplicationLifecycle("Started");
@@ -175,17 +183,17 @@ public abstract class AbstractApplication extends AbstractEntity implements Star
         logApplicationLifecycle("Stopping");
 
         setAttribute(SERVICE_UP, false);
-        setAttribute(Attributes.SERVICE_STATE, Lifecycle.STOPPING);
+        ServiceStateLogic.setExpectedState(this, Lifecycle.STOPPING);
         recordApplicationEvent(Lifecycle.STOPPING);
         try {
             doStop();
         } catch (Exception e) {
-            setAttribute(Attributes.SERVICE_STATE, Lifecycle.ON_FIRE);
+            ServiceStateLogic.setExpectedState(this, Lifecycle.ON_FIRE);
             recordApplicationEvent(Lifecycle.ON_FIRE);
             log.warn("Error stopping application " + this + " (rethrowing): "+e);
             throw Exceptions.propagate(e);
         }
-        setAttribute(Attributes.SERVICE_STATE, Lifecycle.STOPPED);
+        ServiceStateLogic.setExpectedState(this, Lifecycle.STOPPED);
         recordApplicationEvent(Lifecycle.STOPPED);
 
         synchronized (this) {

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/AbstractEntity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/AbstractEntity.java b/core/src/main/java/brooklyn/entity/basic/AbstractEntity.java
index 97d7b4e..eb5f099 100644
--- a/core/src/main/java/brooklyn/entity/basic/AbstractEntity.java
+++ b/core/src/main/java/brooklyn/entity/basic/AbstractEntity.java
@@ -39,9 +39,9 @@ import brooklyn.entity.Effector;
 import brooklyn.entity.Entity;
 import brooklyn.entity.EntityType;
 import brooklyn.entity.Group;
+import brooklyn.entity.basic.ServiceStateLogic.ServiceNotUpLogic;
 import brooklyn.entity.proxying.EntitySpec;
 import brooklyn.entity.rebind.BasicEntityRebindSupport;
-import brooklyn.entity.rebind.RebindManagerImpl;
 import brooklyn.entity.rebind.RebindSupport;
 import brooklyn.event.AttributeSensor;
 import brooklyn.event.Sensor;
@@ -1026,6 +1026,31 @@ public abstract class AbstractEntity extends AbstractBrooklynObject implements E
 //            .add("name", getDisplayName());
     }
     
+    // -------- INITIALIZATION --------------
+
+    /**
+     * Default entity initialization, just calls {@link #initEnrichers()}.
+     */
+    public void init() {
+        super.init();
+        initEnrichers();
+    }
+    
+    /**
+     * By default, adds enrichers to populate {@link Attributes#SERVICE_UP} and {@link Attributes#SERVICE_STATE}
+     * based on {@link Attributes#SERVICE_NOT_UP_INDICATORS}, 
+     * {@link Attributes#SERVICE_STATE_EXPECTED} and {@link Attributes#SERVICE_PROBLEMS}
+     * (doing nothing if these sensors are not used).
+     * <p>
+     * Subclasses may go further and populate the {@link Attributes#SERVICE_NOT_UP_INDICATORS} 
+     * and {@link Attributes#SERVICE_PROBLEMS} from children and members or other sources.
+     */
+    // these enrichers do nothing unless Attributes.SERVICE_NOT_UP_INDICATORS are used
+    // and/or SERVICE_STATE_EXPECTED 
+    protected void initEnrichers() {
+        addEnricher(ServiceNotUpLogic.newEnricherForServiceUpIfNoNotUpIndicators());
+        addEnricher(ServiceStateLogic.newEnricherForServiceStateFromProblemsAndUp());
+    }
     
     // -------- POLICIES --------------------
 
@@ -1035,35 +1060,32 @@ public abstract class AbstractEntity extends AbstractBrooklynObject implements E
     }
 
     @Override
-    public Policy addPolicy(Policy policy) {
-        List<Policy> old = MutableList.<Policy>copyOf(policies);
-
+    public void addPolicy(Policy policy) {
+        Policy old = findApparentlyEqualAndWarnIfNotSameUniqueTag(policies, policy);
+        if (old!=null) {
+            LOG.debug("Removing "+old+" when adding "+policy+" to "+this);
+            removePolicy(old);
+        }
+        
         policies.add((AbstractPolicy)policy);
         ((AbstractPolicy)policy).setEntity(this);
         
         getManagementSupport().getEntityChangeListener().onPolicyAdded(policy);
         emit(AbstractEntity.POLICY_ADDED, new PolicyDescriptor(policy));
-        
-        Policy actual = findApparentlyEqualsAndWarn(old, policy);
-        if (actual!=null) {
-            removePolicy(policy);
-            return actual;
-        }
-        return policy;
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public <T extends Policy> T addPolicy(PolicySpec<T> spec) {
         T policy = getManagementContext().getEntityManager().createPolicy(spec);
-        return (T) addPolicy(policy);
+        addPolicy(policy);
+        return policy;
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public <T extends Enricher> T addEnricher(EnricherSpec<T> spec) {
         T enricher = getManagementContext().getEntityManager().createEnricher(spec);
-        return (T) addEnricher(enricher);
+        addEnricher(enricher);
+        return enricher;
     }
 
     @Override
@@ -1094,37 +1116,31 @@ public abstract class AbstractEntity extends AbstractBrooklynObject implements E
     }
 
     @Override
-    public Enricher addEnricher(Enricher enricher) {
-        List<Enricher> old = MutableList.<Enricher>copyOf(enrichers);
+    public void addEnricher(Enricher enricher) {
+        Enricher old = findApparentlyEqualAndWarnIfNotSameUniqueTag(enrichers, enricher);
+        if (old!=null) {
+            LOG.debug("Removing "+old+" when adding "+enricher+" to "+this);
+            removeEnricher(old);
+        }
         
         enrichers.add((AbstractEnricher) enricher);
         ((AbstractEnricher)enricher).setEntity(this);
         
         getManagementSupport().getEntityChangeListener().onEnricherAdded(enricher);
         // TODO Could add equivalent of AbstractEntity.POLICY_ADDED for enrichers; no use-case for that yet
-        
-        Enricher actual = findApparentlyEqualsAndWarn(old, enricher);
-        if (actual!=null) {
-            removeEnricher(enricher);
-            return actual;
-        }
-        return enricher;
     }
     
-    private <T extends EntityAdjunct> T findApparentlyEqualsAndWarn(Collection<? extends T> items, T newItem) {
-        T oldItem = findApparentlyEquals(items, newItem);
+    private <T extends EntityAdjunct> T findApparentlyEqualAndWarnIfNotSameUniqueTag(Collection<? extends T> items, T newItem) {
+        T oldItem = findApparentlyEqual(items, newItem);
         
         if (oldItem!=null) {
             String newItemTag = newItem.getUniqueTag();
             if (newItemTag!=null) {
-                // old item has same tag; don't add
-                LOG.warn("Adding to "+this+", "+newItem+" has identical uniqueTag as existing "+oldItem+"; will remove after adding. "
-                    + "Underlying addition should be modified so it is not added twice.");
                 return oldItem;
             }
             if (isRebinding()) {
-                LOG.warn("Adding to "+this+", "+newItem+" appears identical to existing "+oldItem+"; will remove after adding. "
-                    + "Underlying addition should be modified so it is not added twice during rebind.");
+                LOG.warn("Adding to "+this+", "+newItem+" appears identical to existing "+oldItem+"; will replace. "
+                    + "Underlying addition should be modified so it is not added twice during rebind or unique tag should be used to indicate it is identical.");
                 return oldItem;
             } else {
                 LOG.warn("Adding to "+this+", "+newItem+" appears identical to existing "+oldItem+"; may get removed on rebind. "
@@ -1135,8 +1151,8 @@ public abstract class AbstractEntity extends AbstractBrooklynObject implements E
             return null;
         }
     }
-    private <T extends EntityAdjunct> T findApparentlyEquals(Collection<? extends T> itemsCopy, T newItem) {
-        // FIXME workaround for issue where enrichers can get added multiple times on rebind,
+    private <T extends EntityAdjunct> T findApparentlyEqual(Collection<? extends T> itemsCopy, T newItem) {
+        // TODO workaround for issue where enrichers can get added multiple times on rebind,
         // if it's added in onBecomingManager or connectSensors; the right fix will be more disciplined about how/where these are added
         // (easier done when sensor feeds are persisted)
         Class<?> beforeEntityAdjunct = newItem.getClass();
@@ -1157,6 +1173,8 @@ public abstract class AbstractEntity extends AbstractBrooklynObject implements E
                         "transformation",
                         // from averager
                         "values", "timestamps", "lastAverage")) {
+                    
+                    
                     return oldItem;
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/AbstractGroup.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/AbstractGroup.java b/core/src/main/java/brooklyn/entity/basic/AbstractGroup.java
index e82825d..67e4cf5 100644
--- a/core/src/main/java/brooklyn/entity/basic/AbstractGroup.java
+++ b/core/src/main/java/brooklyn/entity/basic/AbstractGroup.java
@@ -23,6 +23,8 @@ import java.util.Collection;
 import brooklyn.config.ConfigKey;
 import brooklyn.entity.Entity;
 import brooklyn.entity.Group;
+import brooklyn.entity.basic.QuorumCheck.QuorumChecks;
+import brooklyn.entity.basic.ServiceStateLogic.ComputeServiceIndicatorsFromChildrenAndMembers;
 import brooklyn.entity.trait.Changeable;
 import brooklyn.event.AttributeSensor;
 import brooklyn.event.basic.Sensors;
@@ -43,6 +45,7 @@ import com.google.common.reflect.TypeToken;
  */
 public interface AbstractGroup extends Entity, Group, Changeable {
 
+    @SuppressWarnings("serial")
     AttributeSensor<Collection<Entity>> GROUP_MEMBERS = Sensors.newSensor(
             new TypeToken<Collection<Entity>>() { }, "group.members", "Members of the group");
 
@@ -52,6 +55,13 @@ public interface AbstractGroup extends Entity, Group, Changeable {
     ConfigKey<String> MEMBER_DELEGATE_NAME_FORMAT = ConfigKeys.newStringConfigKey(
             "group.members.delegate.nameFormat", "Delegate members name format string (Use %s for the original entity display name)", "%s");
 
+    public static final ConfigKey<QuorumCheck> UP_QUORUM_CHECK = ConfigKeys.newConfigKeyWithDefault(ComputeServiceIndicatorsFromChildrenAndMembers.UP_QUORUM_CHECK, 
+        "Up check, applied by default to members, requiring at least one present and up",
+        QuorumChecks.atLeastOne());
+    public static final ConfigKey<QuorumCheck> RUNNING_QUORUM_CHECK = ConfigKeys.newConfigKeyWithDefault(ComputeServiceIndicatorsFromChildrenAndMembers.RUNNING_QUORUM_CHECK,
+        "Problems check from children actual states (lifecycle), applied by default to members and children, not checking upness, but requiring by default that none are on-fire",
+        QuorumChecks.all());
+
     void setMembers(Collection<Entity> m);
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/AbstractGroupImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/AbstractGroupImpl.java b/core/src/main/java/brooklyn/entity/basic/AbstractGroupImpl.java
index 765d166..e3520f6 100644
--- a/core/src/main/java/brooklyn/entity/basic/AbstractGroupImpl.java
+++ b/core/src/main/java/brooklyn/entity/basic/AbstractGroupImpl.java
@@ -89,6 +89,16 @@ public abstract class AbstractGroupImpl extends AbstractEntity implements Abstra
         setAttribute(GROUP_MEMBERS, ImmutableList.<Entity>of());
     }
 
+    @Override
+    protected void initEnrichers() {
+        super.initEnrichers();
+        
+        // problem if any children or members are on fire
+        ServiceStateLogic.newEnricherFromChildrenState().checkChildrenAndMembers().requireRunningChildren(getConfig(RUNNING_QUORUM_CHECK)).addTo(this);
+        // defaults to requiring at least one member or child who is up
+        ServiceStateLogic.newEnricherFromChildrenUp().checkChildrenAndMembers().requireUpChildren(getConfig(UP_QUORUM_CHECK)).addTo(this);
+    }
+
     /**
      * Adds the given entity as a member of this group <em>and</em> this group as one of the groups of the child
      */

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/Attributes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/Attributes.java b/core/src/main/java/brooklyn/entity/basic/Attributes.java
index 4453cad..eaa6143 100644
--- a/core/src/main/java/brooklyn/entity/basic/Attributes.java
+++ b/core/src/main/java/brooklyn/entity/basic/Attributes.java
@@ -45,6 +45,7 @@ public interface Attributes {
     BasicAttributeSensorAndConfigKey<String> DOWNLOAD_URL = new BasicAttributeSensorAndConfigKey<String>(
             String.class, "download.url", "URL pattern for downloading the installer (will substitute things like ${version} automatically)");
 
+    @SuppressWarnings({ "unchecked", "rawtypes" })
     BasicAttributeSensorAndConfigKey<Map<String,String>> DOWNLOAD_ADDON_URLS = new BasicAttributeSensorAndConfigKey(
             Map.class, "download.addon.urls", "URL patterns for downloading named add-ons (will substitute things like ${version} automatically)");
 
@@ -53,9 +54,11 @@ public interface Attributes {
      * Port number attributes.
      */
 
+    @SuppressWarnings({ "unchecked", "rawtypes" })
     AttributeSensor<List<Integer>> PORT_NUMBERS = new BasicAttributeSensor(
             List.class, "port.list", "List of port numbers");
     
+    @SuppressWarnings({ "unchecked", "rawtypes" })
     AttributeSensor<List<Sensor<Integer>>> PORT_SENSORS = new BasicAttributeSensor(
             List.class, "port.list.sensors", "List of port number attributes");
 
@@ -98,8 +101,20 @@ public interface Attributes {
         "service.notUp.indicators", 
         "A map of namespaced indicators that the service is not up");
     
-    AttributeSensor<Lifecycle> SERVICE_STATE = Sensors.newSensor(Lifecycle.class,
-            "service.state", "Expected lifecycle state of the service");
+    @SuppressWarnings("serial")
+    AttributeSensor<Map<String,Object>> SERVICE_PROBLEMS = Sensors.newSensor(
+        new TypeToken<Map<String,Object>>() {},
+        "service.problems", 
+        "A map of namespaced indicators of problems with a service");
+
+    AttributeSensor<Lifecycle> SERVICE_STATE_ACTUAL = Sensors.newSensor(Lifecycle.class,
+            "service.state", "Actual lifecycle state of the service");
+    AttributeSensor<Lifecycle.Transition> SERVICE_STATE_EXPECTED = Sensors.newSensor(Lifecycle.Transition.class,
+            "service.state.expected", "Last controlled change to service state, indicating what the expected state should be");
+    
+    /** @deprecated since 0.7.0 use {@link #SERVICE_STATE_ACTUAL} or {@link #SERVICE_STATE_EXPECTED} as appropriate. */
+    @Deprecated
+    AttributeSensor<Lifecycle> SERVICE_STATE = SERVICE_STATE_ACTUAL;
 
     /*
      * Other metadata (optional)

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/ConfigKeys.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/ConfigKeys.java b/core/src/main/java/brooklyn/entity/basic/ConfigKeys.java
index b7f6362..b79f4b7 100644
--- a/core/src/main/java/brooklyn/entity/basic/ConfigKeys.java
+++ b/core/src/main/java/brooklyn/entity/basic/ConfigKeys.java
@@ -132,6 +132,10 @@ public class ConfigKeys {
         return new BasicConfigKeyOverwriting<T>(parent, defaultValue);
     }
 
+    public static <T> ConfigKey<T> newConfigKeyWithDefault(ConfigKey<T> parent, String newDescription, T defaultValue) {
+        return new BasicConfigKeyOverwriting<T>(parent, newDescription, defaultValue);
+    }
+
     public static <T> ConfigKey<T> newConfigKeyRenamed(String newName, ConfigKey<T> key) {
         return new BasicConfigKey<T>(key.getTypeToken(), newName, key.getDescription(), key.getDefaultValue());
     }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/DynamicGroup.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/DynamicGroup.java b/core/src/main/java/brooklyn/entity/basic/DynamicGroup.java
index 537189d..bd2f87e 100644
--- a/core/src/main/java/brooklyn/entity/basic/DynamicGroup.java
+++ b/core/src/main/java/brooklyn/entity/basic/DynamicGroup.java
@@ -35,6 +35,7 @@ import com.google.common.reflect.TypeToken;
 @ImplementedBy(DynamicGroupImpl.class)
 public interface DynamicGroup extends AbstractGroup {
 
+    @SuppressWarnings("serial")
     @SetFromFlag("entityFilter")
     ConfigKey<Predicate<? super Entity>> ENTITY_FILTER = ConfigKeys.newConfigKey(new TypeToken<Predicate<? super Entity>>() { },
             "dynamicgroup.entityfilter", "Filter for entities which will automatically be in the group");

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/DynamicGroupImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/DynamicGroupImpl.java b/core/src/main/java/brooklyn/entity/basic/DynamicGroupImpl.java
index c69c210..2d5a76f 100644
--- a/core/src/main/java/brooklyn/entity/basic/DynamicGroupImpl.java
+++ b/core/src/main/java/brooklyn/entity/basic/DynamicGroupImpl.java
@@ -60,7 +60,7 @@ public class DynamicGroupImpl extends AbstractGroupImpl implements DynamicGroup
         super.init();
         setAttribute(RUNNING, true);
     }
-
+    
     @Override
     public void setEntityFilter(Predicate<? super Entity> filter) {
         // TODO Sould this be "evenIfOwned"?

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/EntityFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/EntityFunctions.java b/core/src/main/java/brooklyn/entity/basic/EntityFunctions.java
index af9454c..7ad41f6 100644
--- a/core/src/main/java/brooklyn/entity/basic/EntityFunctions.java
+++ b/core/src/main/java/brooklyn/entity/basic/EntityFunctions.java
@@ -30,49 +30,55 @@ import brooklyn.entity.trait.Identifiable;
 import brooklyn.event.AttributeSensor;
 import brooklyn.management.ManagementContext;
 import brooklyn.util.flags.TypeCoercions;
+import brooklyn.util.guava.Functionals;
 
 import com.google.common.base.Function;
 import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
 
 public class EntityFunctions {
 
     public static <T> Function<Entity, T> attribute(final AttributeSensor<T> attribute) {
-        return new Function<Entity, T>() {
+        class GetEntityAttributeFunction implements Function<Entity, T> {
             @Override public T apply(Entity input) {
                 return (input == null) ? null : input.getAttribute(attribute);
             }
         };
+        return new GetEntityAttributeFunction();
     }
     
     public static <T> Function<Entity, T> config(final ConfigKey<T> key) {
-        return new Function<Entity, T>() {
+        class GetEntityConfigFunction implements Function<Entity, T> {
             @Override public T apply(Entity input) {
                 return (input == null) ? null : input.getConfig(key);
             }
         };
+        return new GetEntityConfigFunction();
     }
     
     public static Function<Entity, String> displayName() {
-        return new Function<Entity, String>() {
+        class GetEntityDisplayName implements Function<Entity, String> {
             @Override public String apply(Entity input) {
                 return (input == null) ? null : input.getDisplayName();
             }
         };
+        return new GetEntityDisplayName();
     }
     
     public static Function<Identifiable, String> id() {
-        return new Function<Identifiable, String>() {
+        class GetIdFunction implements Function<Identifiable, String> {
             @Override public String apply(Identifiable input) {
                 return (input == null) ? null : input.getId();
             }
         };
+        return new GetIdFunction();
     }
 
     /** returns a function which sets the given sensors on the entity passed in,
      * with {@link Entities#UNCHANGED} and {@link Entities#REMOVE} doing those actions. */
     public static Function<Entity,Void> settingSensorsConstant(final Map<AttributeSensor<?>,Object> values) {
         checkNotNull(values, "values");
-        return new Function<Entity,Void>() {
+        class SettingSensorsConstantFunction implements Function<Entity, Void> {
             @SuppressWarnings({ "unchecked", "rawtypes" })
             @Override public Void apply(Entity input) {
                 for (Map.Entry<AttributeSensor<?>,Object> entry : values.entrySet()) {
@@ -89,42 +95,37 @@ public class EntityFunctions {
                 }
                 return null;
             }
-        };
+        }
+        return new SettingSensorsConstantFunction();
     }
 
     /** as {@link #settingSensorsConstant(Map)} but as a {@link Runnable} */
-    public static Runnable settingSensorsConstantRunnable(final Entity entity, final Map<AttributeSensor<?>,Object> values) {
+    public static Runnable settingSensorsConstant(final Entity entity, final Map<AttributeSensor<?>,Object> values) {
         checkNotNull(entity, "entity");
         checkNotNull(values, "values");
-        return new Runnable() {
-            @Override
-            public void run() {
-                settingSensorsConstant(values).apply(entity);
-            }
-        };
+        return Functionals.runnable(Suppliers.compose(settingSensorsConstant(values), Suppliers.ofInstance(entity)));
     }
 
-
-    /** as {@link #settingSensorsConstant(Map)} but creating a {@link Function} which ignores its input,
-     * suitable for use with sensor feeds where the input is ignored */
-    public static <T> Function<T,Void> settingSensorsConstantFunction(final Entity entity, final Map<AttributeSensor<?>,Object> values) {
-        checkNotNull(entity, "entity");
-        checkNotNull(values, "values");
-        return new Function<T,Void>() {
-            @Override
-            public Void apply(T input) {
-                return settingSensorsConstant(values).apply(entity);
+    public static <K,V> Function<Entity, Void> updatingSensorMapEntry(final AttributeSensor<Map<K,V>> mapSensor, final K key, final Supplier<? extends V> valueSupplier) {
+        class UpdatingSensorMapEntryFunction implements Function<Entity, Void> {
+            @Override public Void apply(Entity input) {
+                ServiceStateLogic.updateMapSensorEntry((EntityLocal)input, mapSensor, key, valueSupplier.get());
+                return null;
             }
-        };
+        }
+        return new UpdatingSensorMapEntryFunction();
+    }
+    public static <K,V> Runnable updatingSensorMapEntry(final Entity entity, final AttributeSensor<Map<K,V>> mapSensor, final K key, final Supplier<? extends V> valueSupplier) {
+        return Functionals.runnable(Suppliers.compose(updatingSensorMapEntry(mapSensor, key, valueSupplier), Suppliers.ofInstance(entity)));
     }
 
     public static Supplier<Collection<Application>> applications(final ManagementContext mgmt) {
-        return new Supplier<Collection<Application>>() {
+        class AppsSupplier implements Supplier<Collection<Application>> {
             @Override
             public Collection<Application> get() {
                 return mgmt.getApplications();
             }
-        };
+        }
+        return new AppsSupplier();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/Lifecycle.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/Lifecycle.java b/core/src/main/java/brooklyn/entity/basic/Lifecycle.java
index f0f0a9e..5e7d0fe 100644
--- a/core/src/main/java/brooklyn/entity/basic/Lifecycle.java
+++ b/core/src/main/java/brooklyn/entity/basic/Lifecycle.java
@@ -18,26 +18,15 @@
  */
 package brooklyn.entity.basic;
 
+import java.io.Serializable;
+import java.util.Date;
+
 import com.google.common.base.CaseFormat;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
 
 /**
  * An enumeration representing the status of an {@link brooklyn.entity.Entity}.
- *
- * @startuml img/entity-lifecycle.png
- * title Entity Lifecycle
- * 
- * (*) ->  "CREATED"
- *     if "Exception" then
- *     ->  "ON_FIRE"
- *     else
- *     --> "STARTING"
- *     --> "RUNNING"
- *     ->  "STOPPING"
- *     --> "STOPPED"
- *     --> "RUNNING"
- *     --> "DESTROYED"
- *     -left-> (*)
- * @enduml
  */
 public enum Lifecycle {
     /**
@@ -51,12 +40,21 @@ public enum Lifecycle {
 
     /**
      * The entity is starting.
-     *
-     * This stage is entered when the {@link brooklyn.entity.trait.Startable#START} {@link brooklyn.entity.Effector} is called. 
-     * The entity will have its location set and and setup helper object created.
+     * <p>
+     * This stage is typically entered when the {@link brooklyn.entity.trait.Startable#START} {@link brooklyn.entity.Effector} 
+     * is called, to undertake the startup operations from the management plane.
+     * When this completes the entity will normally transition to 
+     * {@link Lifecycle#RUNNING}. 
      */
+//    * {@link Lifecycle#STARTED} or 
     STARTING,
 
+//    /**
+//     * The entity has been started and no further start-up steps are needed from the management plane,
+//     * but the entity has not yet been confirmed as running.
+//     */
+//    STARTED,
+//
     /**
      * The entity service is expected to be running. In healthy operation, {@link Attributes#SERVICE_UP} will be true,
      * or will shortly be true if all service start actions have been completed and we are merely waiting for it to be running. 
@@ -121,4 +119,41 @@ public enum Lifecycle {
           return ON_FIRE;
        }
     }
+    
+    public static class Transition implements Serializable {
+        private static final long serialVersionUID = 603419184398753502L;
+        
+        final Lifecycle state;
+        final long timestampUtc;
+        
+        public Transition(Lifecycle state, Date timestamp) {
+            this.state = Preconditions.checkNotNull(state, "state");
+            this.timestampUtc = Preconditions.checkNotNull(timestamp, "timestamp").getTime();
+        }
+        
+        public Lifecycle getState() {
+            return state;
+        }
+        public Date getTimestamp() {
+            return new Date(timestampUtc);
+        }
+        
+        @Override
+        public int hashCode() {
+            return Objects.hashCode(state, timestampUtc);
+        }
+        
+        @Override
+        public boolean equals(Object obj) {
+            if (!(obj instanceof Transition)) return false;
+            if (!state.equals(((Transition)obj).getState())) return false;
+            if (timestampUtc != ((Transition)obj).timestampUtc) return false;
+            return true;
+        }
+        
+        @Override
+        public String toString() {
+            return state+" @ "+new Date(timestampUtc);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/97eed6bd/core/src/main/java/brooklyn/entity/basic/QuorumCheck.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/entity/basic/QuorumCheck.java b/core/src/main/java/brooklyn/entity/basic/QuorumCheck.java
new file mode 100644
index 0000000..bac4515
--- /dev/null
+++ b/core/src/main/java/brooklyn/entity/basic/QuorumCheck.java
@@ -0,0 +1,74 @@
+/*
+ * 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 brooklyn.entity.basic;
+
+import java.io.Serializable;
+
+public interface QuorumCheck {
+
+    public boolean isQuorate(int sizeHealthy, int totalSize);
+
+    public static class QuorumChecks {
+        public static QuorumCheck all() {
+            return new NumericQuorumCheck(0, 1.0, false);
+        }
+        public static QuorumCheck allAndAtLeastOne() {
+            return new NumericQuorumCheck(1, 1.0, false);
+        }
+        public static QuorumCheck atLeastOne() {
+            return new NumericQuorumCheck(1, 0.0, false);
+        }
+        /** require at least one to be up if the total size is non-zero;
+         * ie okay if empty, or if non-empty and something is healthy, but not okay if not-empty and nothing is healthy */
+        public static QuorumCheck atLeastOneUnlessEmpty() {
+            return new NumericQuorumCheck(1, 0.0, true);
+        }
+        public static QuorumCheck newInstance(int minRequiredSize, double minRequiredRatio, boolean allowEmpty) {
+            return new NumericQuorumCheck(minRequiredSize, minRequiredRatio, allowEmpty);
+        }
+    }
+    
+    public static class NumericQuorumCheck implements QuorumCheck, Serializable {
+        private static final long serialVersionUID = -5090669237460159621L;
+        
+        protected final int minRequiredSize;
+        protected final double minRequiredRatio;
+        protected final boolean allowEmpty;
+
+        public NumericQuorumCheck(int minRequiredSize, double minRequiredRatio, boolean allowEmpty) {
+            this.minRequiredSize = minRequiredSize;
+            this.minRequiredRatio = minRequiredRatio;
+            this.allowEmpty = allowEmpty;
+        }
+        
+        @Override
+        public boolean isQuorate(int sizeHealthy, int totalSize) {
+            if (allowEmpty && totalSize==0) return true;
+            if (sizeHealthy < minRequiredSize) return false;
+            if (sizeHealthy < totalSize*minRequiredRatio-0.000000001) return false;
+            return true;
+        }
+        
+        @Override
+        public String toString() {
+            return "QuorumCheck[require="+minRequiredSize+","+((int)100*minRequiredRatio)+"%]";
+        }
+    }
+    
+}