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/09/22 21:09:38 UTC

[06/22] brooklyn-server git commit: change impl of EntityConfigMap to not cache inherited

change impl of EntityConfigMap to not cache inherited

fix tests, and misc other more fixes and notes


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

Branch: refs/heads/master
Commit: 8847b2eed525f23080e8741ed5d9ba7119507534
Parents: c701fcb
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Mon Sep 19 23:11:11 2016 +0100
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Tue Sep 20 09:58:13 2016 +0100

----------------------------------------------------------------------
 .../brooklyn/ConfigInheritanceYamlTest.java     |  32 +-
 .../core/config/BasicConfigInheritance.java     |  44 ++-
 .../apache/brooklyn/core/config/ConfigKeys.java |  12 +-
 .../config/internal/AbstractConfigMapImpl.java  |   6 +
 .../brooklyn/core/entity/AbstractEntity.java    |  14 +-
 .../brooklyn/core/entity/EntityAsserts.java     |   2 +-
 .../brooklyn/core/entity/EntityFunctions.java   |  44 ++-
 .../core/entity/internal/EntityConfigMap.java   | 305 +++++++------------
 .../core/internal/BrooklynPropertiesImpl.java   |   5 +
 .../core/location/AbstractLocation.java         |   6 +
 .../internal/DeferredBrooklynProperties.java    |   5 +
 .../core/objs/AbstractEntityAdjunct.java        |   5 +
 .../core/objs/BrooklynObjectInternal.java       |  18 +-
 .../entity/software/base/SoftwareProcess.java   |   1 +
 .../SoftwareProcessEntityFeedRebindTest.java    |   2 +-
 .../brooklyn/config/ConfigInheritance.java      |  17 +-
 .../org/apache/brooklyn/config/ConfigMap.java   |  11 +
 17 files changed, 316 insertions(+), 213 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigInheritanceYamlTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigInheritanceYamlTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigInheritanceYamlTest.java
index fd4472c..c3d4ebc 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigInheritanceYamlTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigInheritanceYamlTest.java
@@ -60,6 +60,7 @@ public class ConfigInheritanceYamlTest extends AbstractYamlTest {
 
     private Path emptyFile;
     private Path emptyFile2;
+    private Path emptyFile3;
     
     private ExecutorService executor;
 
@@ -72,6 +73,7 @@ public class ConfigInheritanceYamlTest extends AbstractYamlTest {
         
         emptyFile = Files.createTempFile("ConfigInheritanceYamlTest", ".txt");
         emptyFile2 = Files.createTempFile("ConfigInheritanceYamlTest2", ".txt");
+        emptyFile3 = Files.createTempFile("ConfigInheritanceYamlTest3", ".txt");
         
         addCatalogItems(
                 "brooklyn.catalog:",
@@ -326,53 +328,72 @@ public class ConfigInheritanceYamlTest extends AbstractYamlTest {
                 "- type: org.apache.brooklyn.entity.stock.BasicApplication",
                 "  brooklyn.config:",
                 "    shell.env:",
-                "      ENV1: myEnv1",
+                "      ENV: myEnv",
+                "      ENV3: myEnv",
                 "    templates.preinstall:",
                 "      "+emptyFile.toUri()+": myfile",
+                "      "+emptyFile3.toUri()+": myfile",
                 "    files.preinstall:",
                 "      "+emptyFile.toUri()+": myfile",
+                "      "+emptyFile3.toUri()+": myfile",
                 "    templates.install:",
                 "      "+emptyFile.toUri()+": myfile",
+                "      "+emptyFile3.toUri()+": myfile",
                 "    files.install:",
                 "      "+emptyFile.toUri()+": myfile",
+                "      "+emptyFile3.toUri()+": myfile",
                 "    templates.runtime:",
                 "      "+emptyFile.toUri()+": myfile",
+                "      "+emptyFile3.toUri()+": myfile",
                 "    files.runtime:",
                 "      "+emptyFile.toUri()+": myfile",
+                "      "+emptyFile3.toUri()+": myfile",
                 "    provisioning.properties:",
                 "      mykey: myval",
+                "      mykey3: myval",
                 "      templateOptions:",
                 "        myOptionsKey: myOptionsVal", 
+                "        myOptionsKey3: myOptionsVal", 
                 "  brooklyn.children:",
                 "  - type: org.apache.brooklyn.entity.software.base.EmptySoftwareProcess",
                 "    brooklyn.config:",
                 "      shell.env:",
                 "        ENV2: myEnv2",
+                "        ENV3: myEnv2",
                 "      templates.preinstall:",
                 "        "+emptyFile2.toUri()+": myfile2",
+                "        "+emptyFile3.toUri()+": myfile2",
                 "      files.preinstall:",
                 "        "+emptyFile2.toUri()+": myfile2",
+                "        "+emptyFile3.toUri()+": myfile2",
                 "      templates.install:",
                 "        "+emptyFile2.toUri()+": myfile2",
+                "        "+emptyFile3.toUri()+": myfile2",
                 "      files.install:",
                 "        "+emptyFile2.toUri()+": myfile2",
+                "        "+emptyFile3.toUri()+": myfile2",
                 "      templates.runtime:",
                 "        "+emptyFile2.toUri()+": myfile2",
+                "        "+emptyFile3.toUri()+": myfile2",
                 "      files.runtime:",
                 "        "+emptyFile2.toUri()+": myfile2",
+                "        "+emptyFile3.toUri()+": myfile2",
                 "      provisioning.properties:",
                 "        mykey2: myval2",
+                "        mykey3: myval2",
                 "        templateOptions:",
-                "          myOptionsKey2: myOptionsVal2");
+                "          myOptionsKey2: myOptionsVal2",
+                "          myOptionsKey3: myOptionsVal2");
         
         Entity app = createStartWaitAndLogApplication(yaml);
         Entity entity = Iterables.getOnlyElement(app.getChildren());
         
         assertEmptySoftwareProcessConfig(
                 entity,
-                ImmutableMap.of("ENV2", "myEnv2"),
-                ImmutableMap.of(emptyFile2.toUri().toString(), "myfile2"),
-                ImmutableMap.of("mykey2", "myval2", "templateOptions", ImmutableMap.of("myOptionsKey2", "myOptionsVal2")));
+                ImmutableMap.of("ENV", "myEnv", "ENV2", "myEnv2", "ENV3", "myEnv2"),
+                ImmutableMap.of(emptyFile.toUri().toString(), "myfile", emptyFile2.toUri().toString(), "myfile2", emptyFile3.toUri().toString(), "myfile2"),
+                ImmutableMap.of("mykey", "myval", "mykey2", "myval2", "mykey3", "myval2", 
+                    "templateOptions", ImmutableMap.of("myOptionsKey", "myOptionsVal", "myOptionsKey2", "myOptionsVal2", "myOptionsKey3", "myOptionsVal2")));
     }
     
     @Test
@@ -629,6 +650,7 @@ public class ConfigInheritanceYamlTest extends AbstractYamlTest {
     
     // TODO Has never worked, and probably hard to fix?! We need to figure out that "env" corresponds to the
     // config key. Maybe FlagUtils could respect SetFromFlags when returning Map<String,ConfigKey>?
+    // TODO should be able to fix in this phase of work
     @Test(groups={"WIP", "Broken"})
     public void testExtendsSuperTypeConfigMixingShortOverridingShortName() throws Exception {
         ImmutableMap<String, Object> expectedEnv = ImmutableMap.<String, Object>of("ENV1", "myEnv1", "ENV2", "myEnv2");

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/config/BasicConfigInheritance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/config/BasicConfigInheritance.java b/core/src/main/java/org/apache/brooklyn/core/config/BasicConfigInheritance.java
index c4ac255..764e0d6 100644
--- a/core/src/main/java/org/apache/brooklyn/core/config/BasicConfigInheritance.java
+++ b/core/src/main/java/org/apache/brooklyn/core/config/BasicConfigInheritance.java
@@ -20,6 +20,7 @@ package org.apache.brooklyn.core.config;
 
 import java.util.Iterator;
 import java.util.Map;
+import java.util.NoSuchElementException;
 
 import javax.annotation.Nullable;
 
@@ -80,12 +81,12 @@ public class BasicConfigInheritance implements ConfigInheritance {
             // check whether parent allows us to get inherited value
             ContainerAndKeyValue<T> c = ancestorContainerKeyValues.next();
             ConfigInheritance inh2 = c.getKey()==null ? null : c.getKey().getInheritanceByContext(context);
-            if (inh2==null) inh2 = this;
-            if (!this.isReinherited) {
+            if (inh2!=null && !ConfigKeys.isReinherited(c.getKey(), context)) {
                 // can't inherit
             } else {
                 // get inherited value
-                v2 = inh2.resolveInheriting(c.getKey(), 
+                if (inh2==null) inh2=this;
+                v2 = inh2.resolveInheriting(c.getKey()!=null ? c.getKey() : null, 
                     c.isValueSet() ? Maybe.of(c.getValue()) : Maybe.<T>absent(), c.getContainer(), 
                         ancestorContainerKeyValues, context);
             }
@@ -99,7 +100,7 @@ public class BasicConfigInheritance implements ConfigInheritance {
         v.container = container;
         if (v2==null || !v2.isValueSet()) {
             v.isValueSet = localValue.isPresent();
-            v.value = v.isValueSet() ? localValue.get() : key.getDefaultValue(); 
+            v.value = v.isValueSet() ? localValue.get() : key!=null ? key.getDefaultValue() : null; 
         } else {
             v.value = resolveConflict(key, localValue, Maybe.ofAllowingNull(v2.getValue()));
             v.isValueSet = true;
@@ -190,6 +191,41 @@ public class BasicConfigInheritance implements ConfigInheritance {
         public TValue getDefaultValue() {
             return key.getDefaultValue();
         }
+    }
+    
+    public static class AncestorContainerAndKeyValueIterator<TContainer,TValue> implements Iterator<ContainerAndKeyValue<TValue>> {
+        private TContainer lastContainer;
+        private final Function<TContainer, ConfigKey<TValue>> keyFindingFunction; 
+        private final Function<TContainer, Maybe<TValue>> localEvaluationFunction; 
+        private final Function<TContainer, TContainer> parentFunction;
+        
+        public AncestorContainerAndKeyValueIterator(TContainer childContainer, 
+                Function<TContainer, ConfigKey<TValue>> keyFindingFunction, 
+                Function<TContainer, Maybe<TValue>> localEvaluationFunction, 
+                Function<TContainer, TContainer> parentFunction) {
+            this.lastContainer = childContainer;
+            this.keyFindingFunction = keyFindingFunction;
+            this.localEvaluationFunction = localEvaluationFunction;
+            this.parentFunction = parentFunction;
+        }
+
+        @Override
+        public boolean hasNext() {
+            return parentFunction.apply(lastContainer)!=null;
+        }
         
+        @Override
+        public ContainerAndKeyValue<TValue> next() {
+            TContainer nextContainer = parentFunction.apply(lastContainer);
+            if (nextContainer==null) throw new NoSuchElementException("Cannot search ancestors further than "+lastContainer);
+            lastContainer = nextContainer;
+            return new BasicContainerAndKeyValue<TContainer,TValue>(keyFindingFunction.apply(lastContainer), lastContainer, localEvaluationFunction);
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException("This iterator does not support removal");
+        }
     }
+    
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/config/ConfigKeys.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/config/ConfigKeys.java b/core/src/main/java/org/apache/brooklyn/core/config/ConfigKeys.java
index 5dd7204..795da50 100644
--- a/core/src/main/java/org/apache/brooklyn/core/config/ConfigKeys.java
+++ b/core/src/main/java/org/apache/brooklyn/core/config/ConfigKeys.java
@@ -22,6 +22,7 @@ import java.util.Map;
 
 import javax.annotation.Nonnull;
 
+import org.apache.brooklyn.config.ConfigInheritance;
 import org.apache.brooklyn.config.ConfigInheritance.ConfigInheritanceContext;
 import org.apache.brooklyn.config.ConfigInheritance.ContainerAndKeyValue;
 import org.apache.brooklyn.config.ConfigInheritance.ContainerAndValue;
@@ -290,10 +291,17 @@ public class ConfigKeys {
     }
 
     /** determine whether a key is reinherited, ie its value is exported to container's descendants  
-     * @deprecated since 0.10.0 In order to determine whether a key is inherited we might need to know whether
+     * @deprecated since introduction in 0.10.0 In order to determine whether a key is inherited we might need to know whether
      * it was explicitly defined as a key on a parent; callers should be refactored. */
     @Deprecated
-    public static <T> boolean isReinherited(final ConfigKey<T> key, final InheritanceContext context) {
+    public static <T> boolean isReinherited(final ConfigKey<T> key, final ConfigInheritanceContext context) {
+        if (key==null) return true;
+        ConfigInheritance inh = key.getInheritanceByContext(context);
+        if (inh==null) return true;
+        if (inh instanceof BasicConfigInheritance) {
+            return ((BasicConfigInheritance)inh).isReinherited;
+        }
+        
         // evaluate by faking a parent who sets a value and seeing if it's reinherited
         Iterable<? extends ContainerAndKeyValue<T>> ckvi = MutableList.of(
             new BasicContainerAndKeyValue<Void,T>(key, null, new Function<Void,Maybe<T>>() {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/config/internal/AbstractConfigMapImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/config/internal/AbstractConfigMapImpl.java b/core/src/main/java/org/apache/brooklyn/core/config/internal/AbstractConfigMapImpl.java
index fd02fc1..26e44b6 100644
--- a/core/src/main/java/org/apache/brooklyn/core/config/internal/AbstractConfigMapImpl.java
+++ b/core/src/main/java/org/apache/brooklyn/core/config/internal/AbstractConfigMapImpl.java
@@ -30,6 +30,7 @@ import org.apache.brooklyn.core.config.StructuredConfigKey;
 import org.apache.brooklyn.core.entity.internal.ConfigMapViewWithStringKeys;
 import org.apache.brooklyn.util.core.flags.TypeCoercions;
 import org.apache.brooklyn.util.core.task.DeferredSupplier;
+import org.apache.brooklyn.util.guava.Maybe;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -63,6 +64,11 @@ public abstract class AbstractConfigMapImpl implements ConfigMap {
         return getConfigRaw(key, true).orNull();
     }
     
+    @Override
+    public Maybe<Object> getConfigLocalRaw(ConfigKey<?> key) {
+        return getConfigRaw(key, false);
+    }
+
     protected Object coerceConfigVal(ConfigKey<?> key, Object v) {
         Object val;
         if ((v instanceof Future) || (v instanceof DeferredSupplier)) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java b/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java
index 3c9e18d..0b40e32 100644
--- a/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java
+++ b/core/src/main/java/org/apache/brooklyn/core/entity/AbstractEntity.java
@@ -54,6 +54,7 @@ import org.apache.brooklyn.api.sensor.Sensor;
 import org.apache.brooklyn.api.sensor.SensorEvent;
 import org.apache.brooklyn.api.sensor.SensorEventListener;
 import org.apache.brooklyn.config.ConfigKey;
+import org.apache.brooklyn.config.ConfigMap;
 import org.apache.brooklyn.config.ConfigKey.HasConfigKey;
 import org.apache.brooklyn.core.BrooklynFeatureEnablement;
 import org.apache.brooklyn.core.BrooklynLogging;
@@ -1254,12 +1255,8 @@ public abstract class AbstractEntity extends AbstractBrooklynObject implements E
 
         @Override
         public void refreshInheritedConfig() {
-            if (getParent() != null) {
-                configsInternal.setInheritedConfig(((EntityInternal)getParent()).getAllConfig(), ((EntityInternal)getParent()).config().getBag());
-            } else {
-                configsInternal.clearInheritedConfig();
-            }
-
+            // no-op, for now, because the impl always looks at ancestors
+            // but in a distributed impl it will need to clear any local cache
             refreshInheritedConfigOfChildren();
         }
         
@@ -1290,6 +1287,11 @@ public abstract class AbstractEntity extends AbstractBrooklynObject implements E
         protected ExecutionContext getContext() {
             return AbstractEntity.this.getExecutionContext();
         }
+        
+        @Override
+        public ConfigMap getInternalConfigMap() {
+            return configsInternal;
+        }
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/entity/EntityAsserts.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/entity/EntityAsserts.java b/core/src/main/java/org/apache/brooklyn/core/entity/EntityAsserts.java
index a793c1f..449d498 100644
--- a/core/src/main/java/org/apache/brooklyn/core/entity/EntityAsserts.java
+++ b/core/src/main/java/org/apache/brooklyn/core/entity/EntityAsserts.java
@@ -159,7 +159,7 @@ public class EntityAsserts {
         try {
             Asserts.succeedsEventually(new Runnable() {
                 @Override public void run() {
-                    Asserts.assertTrue(changed.get(), entity + " -> " + attribute + " not changed");
+                    Asserts.assertTrue(changed.get(), entity + " -> " + attribute + " not changed from "+origValue);
                 }});
         } finally {
             entity.subscriptions().unsubscribe(entity, handle);

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/entity/EntityFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/entity/EntityFunctions.java b/core/src/main/java/org/apache/brooklyn/core/entity/EntityFunctions.java
index 7d8fa3d..505c0d6 100644
--- a/core/src/main/java/org/apache/brooklyn/core/entity/EntityFunctions.java
+++ b/core/src/main/java/org/apache/brooklyn/core/entity/EntityFunctions.java
@@ -19,10 +19,13 @@
 package org.apache.brooklyn.core.entity;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.brooklyn.util.groovy.GroovyJavaMethods.elvis;
 
 import java.util.Collection;
 import java.util.Map;
 
+import javax.annotation.Nullable;
+
 import org.apache.brooklyn.api.entity.Application;
 import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.entity.EntityLocal;
@@ -347,7 +350,8 @@ public class EntityFunctions {
     private static class LocationMatching implements Function<Entity, Location> {
         private Predicate<? super Location> filter;
         
-        private LocationMatching() { /* for xstream */
+        @SuppressWarnings("unused")
+        private LocationMatching() { /* for xstream */ 
         }
         public LocationMatching(Predicate<? super Location> filter) {
             this.filter = filter;
@@ -356,4 +360,42 @@ public class EntityFunctions {
             return Iterables.find(input.getLocations(), filter);
         }
     }
+    
+    public static Function<Entity, Entity> parent() {
+        return new EntityParent();
+    }
+    
+    private static class EntityParent implements Function<Entity, Entity> {
+        @Override public Entity apply(Entity input) {
+            return input==null ? null : input.getParent();
+        }
+    }
+
+    /** Returns a function that finds the best match for the given config key on an entity */
+    public static <T> Function<Entity, ConfigKey<T>> configKeyFinder(ConfigKey<T> queryKey, @Nullable ConfigKey<T> defaultValue) {
+        return new EntityKeyFinder<T>(queryKey, defaultValue);
+    }
+
+    /** As {@link #configKeyFinder(ConfigKey,ConfigKey)} using the query key as the default value */
+    public static <T> Function<Entity, ConfigKey<T>> configKeyFinder(ConfigKey<T> queryKey) {
+        return new EntityKeyFinder<T>(queryKey, queryKey);
+    }
+
+    private static class EntityKeyFinder<T> implements Function<Entity, ConfigKey<T>> {
+        private ConfigKey<T> queryKey;
+        private ConfigKey<T> defaultValue;
+
+        @SuppressWarnings("unused")
+        private EntityKeyFinder() { /* for xstream */ }
+        public EntityKeyFinder(ConfigKey<T> queryKey, ConfigKey<T> defaultValue) {
+            this.queryKey = queryKey;
+            this.defaultValue = defaultValue;
+        }
+
+        @SuppressWarnings("unchecked")
+        @Override public ConfigKey<T> apply(Entity entity) {
+            return entity!=null ? (ConfigKey<T>)elvis(entity.getEntityType().getConfigKey(queryKey.getName()), defaultValue) : defaultValue;
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/entity/internal/EntityConfigMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/entity/internal/EntityConfigMap.java b/core/src/main/java/org/apache/brooklyn/core/entity/internal/EntityConfigMap.java
index 72d494c..9014d84 100644
--- a/core/src/main/java/org/apache/brooklyn/core/entity/internal/EntityConfigMap.java
+++ b/core/src/main/java/org/apache/brooklyn/core/entity/internal/EntityConfigMap.java
@@ -19,33 +19,31 @@
 package org.apache.brooklyn.core.entity.internal;
 
 import static com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.brooklyn.util.groovy.GroovyJavaMethods.elvis;
 
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.mgmt.ExecutionContext;
 import org.apache.brooklyn.api.mgmt.Task;
 import org.apache.brooklyn.config.ConfigInheritance;
-import org.apache.brooklyn.config.ConfigInheritance.ContainerAndKeyValue;
 import org.apache.brooklyn.config.ConfigInheritance.ContainerAndValue;
 import org.apache.brooklyn.config.ConfigKey;
+import org.apache.brooklyn.config.ConfigMap;
 import org.apache.brooklyn.core.config.BasicConfigInheritance;
-import org.apache.brooklyn.core.config.BasicConfigInheritance.BasicContainerAndKeyValue;
-import org.apache.brooklyn.core.config.ConfigKeys;
+import org.apache.brooklyn.core.config.BasicConfigInheritance.AncestorContainerAndKeyValueIterator;
 import org.apache.brooklyn.core.config.ConfigKeys.InheritanceContext;
 import org.apache.brooklyn.core.config.Sanitizer;
 import org.apache.brooklyn.core.config.StructuredConfigKey;
 import org.apache.brooklyn.core.config.internal.AbstractConfigMapImpl;
 import org.apache.brooklyn.core.entity.AbstractEntity;
-import org.apache.brooklyn.util.collections.MutableList;
+import org.apache.brooklyn.core.entity.EntityFunctions;
+import org.apache.brooklyn.core.entity.EntityInternal;
+import org.apache.brooklyn.core.objs.BrooklynObjectInternal;
+import org.apache.brooklyn.core.objs.BrooklynObjectInternal.ConfigurationSupportInternal;
 import org.apache.brooklyn.util.collections.MutableMap;
 import org.apache.brooklyn.util.core.config.ConfigBag;
-import org.apache.brooklyn.util.core.flags.FlagUtils;
-import org.apache.brooklyn.util.core.flags.SetFromFlag;
 import org.apache.brooklyn.util.core.flags.TypeCoercions;
 import org.apache.brooklyn.util.core.internal.ConfigKeySelfExtracting;
 import org.apache.brooklyn.util.guava.Maybe;
@@ -55,7 +53,6 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 
 public class EntityConfigMap extends AbstractConfigMapImpl {
 
@@ -63,19 +60,9 @@ public class EntityConfigMap extends AbstractConfigMapImpl {
 
     /** entity against which config resolution / task execution will occur */
     private final AbstractEntity entity;
-
-    /**
-     * Map of configuration information that is defined at start-up time for the entity. These
-     * configuration parameters are shared and made accessible to the "children" of this
-     * entity.
-     */
-    private final Map<ConfigKey<?>,Object> inheritedConfig = Collections.synchronizedMap(new LinkedHashMap<ConfigKey<?>, Object>());
-    // TODO do we really want to have *both* bags and maps for these?  danger that they get out of synch.
-    // have added some logic (Oct 2014) so that the same changes are applied to both, in most places at least;
-    // i (alex) think we should prefer ConfigBag (the input keys don't matter, it is more a question of retrieval keys),
-    // but first we need ConfigBag to support StructuredConfigKeys 
-    private final ConfigBag localConfigBag;
-    private final ConfigBag inheritedConfigBag;
+    /** config bag cache for local anonymous key lookup */
+    // TODO this could be removed and everything made more like AdjunctConfigMap
+    private final ConfigBag ownConfigBag;
 
     public EntityConfigMap(AbstractEntity entity) {
         // Not using ConcurrentMap, because want to (continue to) allow null values.
@@ -86,96 +73,88 @@ public class EntityConfigMap extends AbstractConfigMapImpl {
     public EntityConfigMap(AbstractEntity entity, Map<ConfigKey<?>, Object> storage) {
         this.entity = checkNotNull(entity, "entity must be specified");
         this.ownConfig = checkNotNull(storage, "storage map must be specified");
-        
-        // TODO store ownUnused in backing-storage
-        this.localConfigBag = ConfigBag.newInstance();
-        this.inheritedConfigBag = ConfigBag.newInstance();
+        this.ownConfigBag = ConfigBag.newInstance();
     }
+    
+    /* These flags are included for documentation.
+     * Changes will require testing.
+     * Note that trying to get a raw inherited value can be problematic
+     * if it is merging resolved values.
+     */
+    final static boolean RESOLVE_AND_COERCE_EXTRACTING_KEYS = true;
+    final static boolean COERCE_DEFAULT_VALUES = true;
+    // TODO We're notifying of config-changed because currently persistence needs to know when the
+    // attributeWhenReady is complete (so it can persist the result).
+    // Long term, we'll just persist tasks properly so the call to onConfigChanged will go!
+    final static boolean NOTIFY_ON_TASK_RESOLUTION = true;
 
-    @SuppressWarnings("unchecked")
-    public <T> T getConfig(ConfigKey<T> key, T defaultValue) {
-        // In case this entity class has overridden the given key (e.g. to set default), then retrieve this entity's key
-        // TODO If ask for a config value that's not in our configKeys, should we really continue with rest of method and return key.getDefaultValue?
-        //      e.g. SshBasedJavaAppSetup calls setAttribute(JMX_USER), which calls getConfig(JMX_USER)
-        //           but that example doesn't have a default...
-        ConfigKey<T> ownKey = entity!=null ? (ConfigKey<T>)elvis(entity.getEntityType().getConfigKey(key.getName()), key) : key;
+    protected static class LocalEvaluateKeyValue<T> implements Function<Entity,Maybe<T>> {
+        ConfigKey<T> keyIgnoringInheritance;
         
-        // TODO We're notifying of config-changed because currently persistence needs to know when the
-        // attributeWhenReady is complete (so it can persist the result).
-        // Long term, we'll just persist tasks properly so the call to onConfigChanged will go!
-
-        // Don't use groovy truth: if the set value is e.g. 0, then would ignore set value and return default!
-        if (ownKey instanceof ConfigKeySelfExtracting) {
-            Object rawval = ownConfig.get(key);
-            // TODO this no longer looks at key's inheritance
-            Maybe<T> result = getConfigImpl((ConfigKeySelfExtracting<T>)ownKey);
-
-            if (rawval instanceof Task) {
-                entity.getManagementSupport().getEntityChangeListener().onConfigChanged(key);
+        public LocalEvaluateKeyValue(ConfigKey<T> keyIgnoringInheritance) {
+            this.keyIgnoringInheritance = keyIgnoringInheritance;
+        }
+        
+        @Override
+        public Maybe<T> apply(Entity entity) {
+            ExecutionContext exec = ((EntityInternal)entity).getExecutionContext();
+            ConfigMap configMap = ((ConfigurationSupportInternal)entity.config()).getInternalConfigMap();
+            Map<ConfigKey<?>,Object> ownConfig = ((EntityConfigMap)configMap).ownConfig;
+            ConfigBag ownConfigBag = ((EntityConfigMap)configMap).ownConfigBag;
+            Maybe<Object> rawValue = configMap.getConfigLocalRaw(keyIgnoringInheritance);
+            Maybe<T> ownValue;
+            
+            // Get own value
+            if (RESOLVE_AND_COERCE_EXTRACTING_KEYS && keyIgnoringInheritance instanceof ConfigKeySelfExtracting && ((ConfigKeySelfExtracting<T>)keyIgnoringInheritance).isSet(ownConfig)) {
+                Map<ConfigKey<?>, ?> ownCopy;
+                synchronized (ownConfig) {
+                    // TODO wasteful to make a copy to look up; maybe try once opportunistically?
+                    ownCopy = MutableMap.copyOf(ownConfig);
+                }
+                ownValue = Maybe.of(((ConfigKeySelfExtracting<T>) keyIgnoringInheritance).extractValue(ownCopy, exec));
+            } else if (ownConfigBag.containsKey(keyIgnoringInheritance)) {
+                // TODO configBag.get doesn't handle tasks/attributeWhenReady - it only uses TypeCoercions
+                // Precedence ordering has changed; previously we'd prefer an explicit isSet(inheritedConfig)
+                // over the localConfigBag.get(key).
+                ownValue = Maybe.of(ownConfigBag.get(keyIgnoringInheritance));
+            } else {
+                ownValue = Maybe.<T>absent();
             }
-            if (result.isPresent()) {
-                return result.get();
+            
+            if (NOTIFY_ON_TASK_RESOLUTION && rawValue.isPresent() && (rawValue.get() instanceof Task)) {
+                ((EntityInternal)entity).getManagementSupport().getEntityChangeListener().onConfigChanged(keyIgnoringInheritance);
             }
-        } else {
-            LOG.warn("Config key {} of {} is not a ConfigKeySelfExtracting; cannot retrieve value; returning default", ownKey, this);
+            
+            return ownValue;
         }
-        return TypeCoercions.coerce((defaultValue != null) ? defaultValue : ownKey.getDefaultValue(), key.getTypeToken());
     }
     
-    private <T> Maybe<T> getConfigImpl(final ConfigKeySelfExtracting<T> key) {
-        final ExecutionContext exec = entity.getExecutionContext();
-        Maybe<T> ownValue;
-
-        // Get own value
-        if (((ConfigKeySelfExtracting<T>)key).isSet(ownConfig)) {
-            Map<ConfigKey<?>, ?> ownCopy;
-            synchronized (ownConfig) {
-                ownCopy = MutableMap.copyOf(ownConfig);
-            }
-            ownValue = Maybe.of(((ConfigKeySelfExtracting<T>) key).extractValue(ownCopy, exec));
-        } else if (localConfigBag.containsKey(key)) {
-            // TODO configBag.get doesn't handle tasks/attributeWhenReady - it only uses TypeCoercions
-            // Precedence ordering has changed; previously we'd prefer an explicit isSet(inheritedConfig)
-            // over the localConfigBag.get(key).
-            ownValue = Maybe.of(localConfigBag.get(key));
-        } else {
-            ownValue = Maybe.<T>absent();
-        }
+    public <T> T getConfig(ConfigKey<T> key, T defaultValue) {
+        Function<Entity, ConfigKey<T>> keyFn = EntityFunctions.configKeyFinder(key, null);
         
-        // TODO the current inheritedConfigBag is not good enough to detect the ancestor container
-        // (only goes up one level in hierarchy)
-        Iterable<? extends ContainerAndKeyValue<T>> ckvi = MutableList.of(
-            new BasicContainerAndKeyValue<Entity,T>(key, entity.getParent(), new Function<Entity,Maybe<T>>() {
-                @Override
-                public Maybe<T> apply(Entity input) {
-                    if (((ConfigKeySelfExtracting<T>)key).isSet(inheritedConfig)) {
-                        return Maybe.of( ((ConfigKeySelfExtracting<T>)key).extractValue(inheritedConfig, exec) );
-                    } else if (inheritedConfigBag.containsKey(key)) {
-                        return Maybe.of(inheritedConfigBag.get(key));
-                    } else {
-                        return Maybe.absent();
-                    }
-                }
-            }));
+        // In case this entity class has overridden the given key (e.g. to set default), then retrieve this entity's key
+        ConfigKey<T> ownKey = keyFn.apply(entity);
+        if (ownKey==null) ownKey = key;
+        
+        LocalEvaluateKeyValue<T> evalFn = new LocalEvaluateKeyValue<T>(ownKey);
 
-        ContainerAndValue<T> result = getDefaultRuntimeInheritance().resolveInheriting(key,
-            ownValue, entity,
-            ckvi.iterator(), InheritanceContext.RUNTIME_MANAGEMENT);
+        if (ownKey instanceof ConfigKeySelfExtracting) {
+            Maybe<T> ownExplicitValue = evalFn.apply(entity);
+            
+            AncestorContainerAndKeyValueIterator<Entity, T> ckvi = new AncestorContainerAndKeyValueIterator<Entity,T>(
+                entity, keyFn, evalFn, EntityFunctions.parent());
+            
+            ContainerAndValue<T> result = getDefaultRuntimeInheritance().resolveInheriting(ownKey,
+                ownExplicitValue, entity,
+                ckvi, InheritanceContext.RUNTIME_MANAGEMENT);
         
-        if (result.isValueSet()) return Maybe.of(result.getValue());
-        return Maybe.absent();
+            if (result.getValue()!=null) return result.getValue();
+        } else {
+            LOG.warn("Config key {} of {} is not a ConfigKeySelfExtracting; cannot retrieve value; returning default", ownKey, this);
+        }
+        return COERCE_DEFAULT_VALUES ? TypeCoercions.coerce(defaultValue, key.getTypeToken()) : defaultValue;
     }
 
-    private <T> boolean isInherited(ConfigKey<T> key) {
-        // TODO
-        return ConfigKeys.isReinherited(key, InheritanceContext.RUNTIME_MANAGEMENT);
-    }
-//    private <T> boolean isInherited(ConfigKey<T> key, ConfigInheritance inheritance) {
-//        if (inheritance==null) inheritance = getDefaultRuntimeInheritance();
-//        InheritanceMode mode = inheritance.isInherited(key, entity.getParent(), entity);
-//        return mode != null && mode != InheritanceMode.NONE;
-//    }
-    
     private ConfigInheritance getDefaultRuntimeInheritance() {
         return BasicConfigInheritance.OVERWRITE; 
     }
@@ -183,14 +162,16 @@ public class EntityConfigMap extends AbstractConfigMapImpl {
     @Override
     public Maybe<Object> getConfigRaw(ConfigKey<?> key, boolean includeInherited) {
         if (ownConfig.containsKey(key)) return Maybe.of(ownConfig.get(key));
-        if (includeInherited && inheritedConfig.containsKey(key)) return Maybe.of(inheritedConfig.get(key));
-        return Maybe.absent();
+        if (!includeInherited || entity.getParent()==null) return Maybe.absent(); 
+        return ((ConfigurationSupportInternal)entity.getParent().config()).getRaw(key);
     }
     
     /** an immutable copy of the config visible at this entity, local and inherited (preferring local) */
+    // TODO deprecate because key inheritance not respected
     public Map<ConfigKey<?>,Object> getAllConfig() {
-        Map<ConfigKey<?>,Object> result = new LinkedHashMap<ConfigKey<?>,Object>(inheritedConfig.size()+ownConfig.size());
-        result.putAll(inheritedConfig);
+        Map<ConfigKey<?>,Object> result = new LinkedHashMap<ConfigKey<?>,Object>();
+        if (entity.getParent()!=null)
+            result.putAll( ((BrooklynObjectInternal)entity.getParent()).config().getInternalConfigMap().getAllConfig() );
         result.putAll(ownConfig);
         return Collections.unmodifiableMap(result);
     }
@@ -203,17 +184,20 @@ public class EntityConfigMap extends AbstractConfigMapImpl {
     }
     
     /** Creates an immutable copy of the config visible at this entity, local and inherited (preferring local), including those that did not match config keys */
+    // TODO deprecate because key inheritance not respected
     public ConfigBag getAllConfigBag() {
-        return ConfigBag.newInstanceCopying(localConfigBag)
-                .putAll(ownConfig)
-                .putIfAbsent(inheritedConfig)
-                .putIfAbsent(inheritedConfigBag)
-                .seal();
+        ConfigBag result = ConfigBag.newInstanceCopying(ownConfigBag)
+                .putAll(ownConfig);
+        if (entity.getParent()!=null) {
+            result.putIfAbsent(
+                ((EntityConfigMap) ((BrooklynObjectInternal)entity.getParent()).config().getInternalConfigMap()).getAllConfigBag() );
+        }
+        return result.seal();
     }
 
     /** Creates an immutable copy of the config defined at this entity, ie not inherited, including those that did not match config keys */
     public ConfigBag getLocalConfigBag() {
-        return ConfigBag.newInstanceCopying(localConfigBag)
+        return ConfigBag.newInstanceCopying(ownConfigBag)
                 .putAll(ownConfig)
                 .seal();
     }
@@ -227,10 +211,10 @@ public class EntityConfigMap extends AbstractConfigMapImpl {
             // TODO ConfigBag does not handle structured config keys; quick fix is to remove (and should also remove any subkeys;
             // as it stands if someone set string a.b.c in the config bag then removed structured key a.b, then got a.b.c they'd get a vale);
             // long term fix is to support structured config keys in ConfigBag, at which point i think we could remove ownConfig altogether
-            localConfigBag.remove(key);
+            ownConfigBag.remove(key);
         } else {
             oldVal = ownConfig.put(key, val);
-            localConfigBag.put((ConfigKey<Object>)key, v);
+            ownConfigBag.put((ConfigKey<Object>)key, v);
         }
         entity.config().refreshInheritedConfigOfChildren();
         return oldVal;
@@ -239,95 +223,27 @@ public class EntityConfigMap extends AbstractConfigMapImpl {
     public void setLocalConfig(Map<ConfigKey<?>, ?> vals) {
         synchronized (ownConfig) {
             ownConfig.clear();
-            localConfigBag.clear();
+            ownConfigBag.clear();
             ownConfig.putAll(vals);
-            localConfigBag.putAll(vals);
-        }
-    }
-    
-    public void setInheritedConfig(Map<ConfigKey<?>, ?> valsO, ConfigBag configBagVals) {
-        Map<ConfigKey<?>, ?> vals = filterUninheritable(valsO);
-        
-        inheritedConfig.clear();
-        inheritedConfig.putAll(vals);
-
-        // The configBagVals contains all inherited, including strings that did not match a config key on the parent.
-        // They might match a config-key on this entity though, so need to check that:
-        //   - if it matches one of our keys, set it in inheritedConfig
-        //   - otherwise add it to our inheritedConfigBag
-        Set<String> valKeyNames = Sets.newLinkedHashSet();
-        for (ConfigKey<?> key : vals.keySet()) {
-            valKeyNames.add(key.getName());
-        }
-        Map<String,Object> valsUnmatched = MutableMap.<String,Object>builder()
-                .putAll(configBagVals.getAllConfig())
-                .removeAll(valKeyNames)
-                .build();
-        inheritedConfigBag.clear();
-        Map<ConfigKey<?>, SetFromFlag> annotatedConfigKeys = FlagUtils.getAnnotatedConfigKeys(entity.getClass());
-        Map<String, ConfigKey<?>> renamedConfigKeys = Maps.newLinkedHashMap();
-        for (Map.Entry<ConfigKey<?>, SetFromFlag> entry: annotatedConfigKeys.entrySet()) {
-            String rename = entry.getValue().value();
-            if (rename != null) {
-                renamedConfigKeys.put(rename, entry.getKey());
-            }
-        }
-        for (Map.Entry<String,Object> entry : valsUnmatched.entrySet()) {
-            String name = entry.getKey();
-            Object value = entry.getValue();
-            ConfigKey<?> key = renamedConfigKeys.get(name);
-            if (key == null) key = entity.getEntityType().getConfigKey(name);
-            if (key != null) {
-                if (!isInherited(key)) {
-                    // no-op
-                } else if (inheritedConfig.containsKey(key)) {
-                    LOG.warn("Entity "+entity+" inherited duplicate config for key "+key+", via explicit config and string name "+name+"; using value of key");
-                } else {
-                    inheritedConfig.put(key, value);
-                }
-            } else {
-                // a config bag has discarded the keys, so we must assume default inheritance for things given that way
-                // unless we can infer a key; not a big deal, as we should have the key in inheritedConfig for everything
-                // which originated with a key ... but still, it would be nice to clean up the use of config bag!
-                inheritedConfigBag.putStringKey(name, value);
-            }
+            ownConfigBag.putAll(vals);
         }
     }
-    
-    private Map<ConfigKey<?>, ?> filterUninheritable(Map<ConfigKey<?>, ?> vals) {
-        Map<ConfigKey<?>, Object> result = Maps.newLinkedHashMap();
-        for (Map.Entry<ConfigKey<?>, ?> entry : vals.entrySet()) {
-            if (isInherited(entry.getKey())) {
-                result.put(entry.getKey(), entry.getValue());
-            }
-        }
-        return result;
-    }
-    
+ 
     public void addToLocalBag(Map<String,?> vals) {
-        localConfigBag.putAll(vals);
-        // quick fix for problem that ownConfig can get out of synch
-        ownConfig.putAll(localConfigBag.getAllConfigAsConfigKeyMap());
+        ownConfigBag.putAll(vals);
+        // TODO quick fix for problem that ownConfig can get out of synch
+        ownConfig.putAll(ownConfigBag.getAllConfigAsConfigKeyMap());
     }
 
     public void removeFromLocalBag(String key) {
-        localConfigBag.remove(key);
+        ownConfigBag.remove(key);
         ownConfig.remove(key);
     }
 
-    public void clearInheritedConfig() {
-        inheritedConfig.clear();
-        inheritedConfigBag.clear();
-    }
-
     @Override
+    // TODO deprecate or clarify syntax 
     public EntityConfigMap submap(Predicate<ConfigKey<?>> filter) {
         EntityConfigMap m = new EntityConfigMap(entity, Maps.<ConfigKey<?>, Object>newLinkedHashMap());
-        for (Map.Entry<ConfigKey<?>,Object> entry: inheritedConfig.entrySet()) {
-            if (filter.apply(entry.getKey())) {
-                m.inheritedConfig.put(entry.getKey(), entry.getValue());
-            }
-        }
         synchronized (ownConfig) {
             for (Map.Entry<ConfigKey<?>,Object> entry: ownConfig.entrySet()) {
                 if (filter.apply(entry.getKey())) {
@@ -335,16 +251,29 @@ public class EntityConfigMap extends AbstractConfigMapImpl {
                 }
             }
         }
+        if (entity.getParent()!=null) {
+            merge(m, ((EntityConfigMap) ((ConfigurationSupportInternal)entity.getParent().config()).getInternalConfigMap()).submap(filter));
+        }
+        m.ownConfigBag.putAll(ownConfig);
         return m;
     }
 
+    private void merge(EntityConfigMap local, EntityConfigMap parent) {
+        for (ConfigKey<?> k: parent.ownConfig.keySet()) {
+            // TODO apply inheritance
+            if (!local.ownConfig.containsKey(k)) {
+                local.ownConfig.put(k, parent.ownConfig.get(k));
+            }
+        }
+    }
+
     @Override
     public String toString() {
         Map<ConfigKey<?>, Object> sanitizeConfig;
         synchronized (ownConfig) {
             sanitizeConfig = Sanitizer.sanitize(ownConfig);
         }
-        return super.toString()+"[own="+sanitizeConfig+"; inherited="+Sanitizer.sanitize(inheritedConfig)+"]";
+        return super.toString()+"[local="+sanitizeConfig+"]";
     }
     
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/internal/BrooklynPropertiesImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/internal/BrooklynPropertiesImpl.java b/core/src/main/java/org/apache/brooklyn/core/internal/BrooklynPropertiesImpl.java
index 023b3e3..d4593b3 100644
--- a/core/src/main/java/org/apache/brooklyn/core/internal/BrooklynPropertiesImpl.java
+++ b/core/src/main/java/org/apache/brooklyn/core/internal/BrooklynPropertiesImpl.java
@@ -450,6 +450,11 @@ public class BrooklynPropertiesImpl extends LinkedHashMap implements BrooklynPro
     }
 
     @Override
+    public Maybe<Object> getConfigLocalRaw(ConfigKey<?> key) {
+        return getConfigRaw(key, false);
+    }
+    
+    @Override
     public Map<ConfigKey<?>, Object> getAllConfig() {
         Map<ConfigKey<?>, Object> result = new LinkedHashMap<ConfigKey<?>, Object>();
         for (Object entry: entrySet())

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/location/AbstractLocation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/location/AbstractLocation.java b/core/src/main/java/org/apache/brooklyn/core/location/AbstractLocation.java
index 1d5c71f..20b44ce 100644
--- a/core/src/main/java/org/apache/brooklyn/core/location/AbstractLocation.java
+++ b/core/src/main/java/org/apache/brooklyn/core/location/AbstractLocation.java
@@ -45,6 +45,7 @@ import org.apache.brooklyn.api.sensor.Sensor;
 import org.apache.brooklyn.api.sensor.SensorEventListener;
 import org.apache.brooklyn.config.ConfigInheritance;
 import org.apache.brooklyn.config.ConfigKey;
+import org.apache.brooklyn.config.ConfigMap;
 import org.apache.brooklyn.config.ConfigKey.HasConfigKey;
 import org.apache.brooklyn.core.BrooklynFeatureEnablement;
 import org.apache.brooklyn.core.config.BasicConfigInheritance;
@@ -501,6 +502,11 @@ public abstract class AbstractLocation extends AbstractBrooklynObject implements
         protected ExecutionContext getContext() {
             return AbstractLocation.this.getManagementContext().getServerExecutionContext();
         }
+        
+        @Override
+        public ConfigMap getInternalConfigMap() {
+            throw new UnsupportedOperationException("location does not use config map");
+        }
     }
     
     public class BasicSubscriptionSupport implements SubscriptionSupportInternal {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/DeferredBrooklynProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/DeferredBrooklynProperties.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/DeferredBrooklynProperties.java
index ae0c7a5..4e3d342 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/DeferredBrooklynProperties.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/DeferredBrooklynProperties.java
@@ -137,6 +137,11 @@ public class DeferredBrooklynProperties implements BrooklynProperties {
         Maybe<Object> result = delegate.getConfigRaw(key, includeInherited);
         return (result.isPresent()) ? Maybe.of(transform(key, result.get())) : Maybe.absent();
     }
+    
+    @Override
+    public Maybe<Object> getConfigLocalRaw(ConfigKey<?> key) {
+        return getConfigRaw(key, false);
+    }
 
     @Override
     public Map<ConfigKey<?>, Object> getAllConfig() {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/objs/AbstractEntityAdjunct.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/objs/AbstractEntityAdjunct.java b/core/src/main/java/org/apache/brooklyn/core/objs/AbstractEntityAdjunct.java
index b800334..1f20d6d 100644
--- a/core/src/main/java/org/apache/brooklyn/core/objs/AbstractEntityAdjunct.java
+++ b/core/src/main/java/org/apache/brooklyn/core/objs/AbstractEntityAdjunct.java
@@ -359,6 +359,11 @@ public abstract class AbstractEntityAdjunct extends AbstractBrooklynObject imple
         protected ExecutionContext getContext() {
             return AbstractEntityAdjunct.this.execution;
         }
+        
+        @Override
+        public ConfigMap getInternalConfigMap() {
+            return configsInternal;
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/core/src/main/java/org/apache/brooklyn/core/objs/BrooklynObjectInternal.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/objs/BrooklynObjectInternal.java b/core/src/main/java/org/apache/brooklyn/core/objs/BrooklynObjectInternal.java
index 89c4e32..4baea63 100644
--- a/core/src/main/java/org/apache/brooklyn/core/objs/BrooklynObjectInternal.java
+++ b/core/src/main/java/org/apache/brooklyn/core/objs/BrooklynObjectInternal.java
@@ -26,6 +26,7 @@ import org.apache.brooklyn.api.objs.BrooklynObject;
 import org.apache.brooklyn.api.objs.Configurable;
 import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.config.ConfigKey.HasConfigKey;
+import org.apache.brooklyn.config.ConfigMap;
 import org.apache.brooklyn.util.core.config.ConfigBag;
 import org.apache.brooklyn.util.guava.Maybe;
 
@@ -51,10 +52,16 @@ public interface BrooklynObjectInternal extends BrooklynObject, Rebindable {
          * Returns a read-only view of all the config key/value pairs on this entity, backed by a string-based map,
          * including config names that did not match anything on this entity.
          *
-         * TODO This method gives no information about which config is inherited versus local;
-         * this means {@link ConfigKey#getInheritance()} cannot be respected. This is an unsolvable problem
-         * for "config names that did not match anything on this entity". Therefore consider using
-         * alternative getters.
+         * This method gives no information about which config is inherited versus local;
+         * this means {@link ConfigKey#getInheritanceByContext()} cannot be respected
+         * if an anonymous key (not matching a declared config key) is set but the
+         * strongly typed key is accessed.
+         * <p> 
+         * It does not identify the container where it is defined, meaning URLs and deferred config values 
+         * cannot be resolved in the context of the appropriate ancestor.
+         * <p>
+         * For these reasons it is recommended to use a different accessor,
+         * and callers should be advised this beta method may be removed. 
          */
         @Beta
         ConfigBag getBag();
@@ -128,6 +135,9 @@ public interface BrooklynObjectInternal extends BrooklynObject, Rebindable {
 
         @Beta
         void refreshInheritedConfigOfChildren();
+        
+        @Beta
+        ConfigMap getInternalConfigMap();
     }
     
     @Beta

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SoftwareProcess.java
----------------------------------------------------------------------
diff --git a/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SoftwareProcess.java b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SoftwareProcess.java
index 0225729..d28bdc6 100644
--- a/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SoftwareProcess.java
+++ b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/SoftwareProcess.java
@@ -258,6 +258,7 @@ public interface SoftwareProcess extends Entity, Startable {
             .description("Custom properties to be passed in when provisioning a new machine")
             .defaultValue(ImmutableMap.<String, Object>of())
             .typeInheritance(BasicConfigInheritance.DEEP_MERGE)
+            .runtimeInheritance(BasicConfigInheritance.NOT_REINHERITED_ELSE_DEEP_MERGE)
             .build();
 
     @SetFromFlag("maxRebindSensorsDelay")

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/software/base/src/test/java/org/apache/brooklyn/entity/software/base/SoftwareProcessEntityFeedRebindTest.java
----------------------------------------------------------------------
diff --git a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/SoftwareProcessEntityFeedRebindTest.java b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/SoftwareProcessEntityFeedRebindTest.java
index aaac53d..960c244 100644
--- a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/SoftwareProcessEntityFeedRebindTest.java
+++ b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/SoftwareProcessEntityFeedRebindTest.java
@@ -68,7 +68,7 @@ public class SoftwareProcessEntityFeedRebindTest extends RebindTestFixtureWithAp
         runFeedsDoNotPollUntilManaged(1, Duration.millis(250));
     }
 
-    @Test(groups="Integeration")
+    @Test(groups="Integration")
     public void testFeedsDoNotPollUntilManagedManyEntities() throws Exception {
         runFeedsDoNotPollUntilManaged(100, Duration.ONE_SECOND);
     }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/utils/common/src/main/java/org/apache/brooklyn/config/ConfigInheritance.java
----------------------------------------------------------------------
diff --git a/utils/common/src/main/java/org/apache/brooklyn/config/ConfigInheritance.java b/utils/common/src/main/java/org/apache/brooklyn/config/ConfigInheritance.java
index 6b8fde3..b9b567c 100644
--- a/utils/common/src/main/java/org/apache/brooklyn/config/ConfigInheritance.java
+++ b/utils/common/src/main/java/org/apache/brooklyn/config/ConfigInheritance.java
@@ -29,6 +29,7 @@ import org.apache.brooklyn.util.guava.Maybe;
 import org.apache.brooklyn.util.text.Strings;
 
 import com.google.common.annotations.Beta;
+import com.google.common.base.Supplier;
 
 @SuppressWarnings("serial")
 public interface ConfigInheritance extends Serializable {
@@ -54,6 +55,20 @@ public interface ConfigInheritance extends Serializable {
     @Deprecated
     InheritanceMode isInherited(ConfigKey<?> key, Object from, Object to);
 
+//    // TODO
+//    interface ConfigKeyValueInContext<TContainer,TValue> extends Supplier<TValue> {
+//        TValue get();
+//        Maybe<TValue> asMaybe();
+//        
+//        TContainer getContainer();
+//        
+//        /** if false, the contents of {@link #getValue()} will have come from the default */
+//        boolean isValueExplicitlySet();
+//        
+//        ConfigKey<TValue> getKey();
+//        TValue getDefaultValue();
+//    }
+
     interface ContainerAndValue<T> {
         Object getContainer();
         T getValue();
@@ -160,7 +175,7 @@ public interface ConfigInheritance extends Serializable {
                 v.container = container;
                 if (v2==null || !v2.isValueSet()) {
                     v.isValueSet = localValue.isPresent();
-                    v.value = v.isValueSet() ? localValue.get() : key.getDefaultValue(); 
+                    v.value = v.isValueSet() ? localValue.get() : key!=null ? key.getDefaultValue() : null; 
                 } else {
                     v.value = resolveConflict(key, localValue, Maybe.ofAllowingNull(v2.getValue()));
                     v.isValueSet = true;

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8847b2ee/utils/common/src/main/java/org/apache/brooklyn/config/ConfigMap.java
----------------------------------------------------------------------
diff --git a/utils/common/src/main/java/org/apache/brooklyn/config/ConfigMap.java b/utils/common/src/main/java/org/apache/brooklyn/config/ConfigMap.java
index 665bbf6..7838b15 100644
--- a/utils/common/src/main/java/org/apache/brooklyn/config/ConfigMap.java
+++ b/utils/common/src/main/java/org/apache/brooklyn/config/ConfigMap.java
@@ -66,7 +66,18 @@ public interface ConfigMap {
      * @return raw, unresolved, uncoerced value of key in map,  
      *         but <b>not</b> any default on the key
      */
+    // TODO behaviour of this is undefined if the key specifies a merge
     public Maybe<Object> getConfigRaw(ConfigKey<?> key, boolean includeInherited);
+    
+    /** returns the value stored against the given key, 
+     * <b>not</b> any default,
+     * <b>not</b> inherited (if there is an inheritance hierarchy),
+     * <b>not</b> type-coerced or further resolved (eg a task or supplier, if such rules are applicable)
+     * @param key  key to look up
+     * @return raw, unresolved, uncoerced value of key explicitly in map
+     */
+    // TODO deprecate other methods?  getRaw won't be able to merge for instance
+    public Maybe<Object> getConfigLocalRaw(ConfigKey<?> key);
 
     /** returns a map of all config keys to their raw (unresolved+uncoerced) contents */
     public Map<ConfigKey<?>,Object> getAllConfig();