You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by sv...@apache.org on 2016/07/11 12:45:48 UTC

[2/8] brooklyn-server git commit: OSGi classloading for persistence

OSGi classloading for persistence

* classnames are written out with the OSGi bundle\u2019s symbolic name
  as a prefix in persisted state (unless from a white-listed brooklyn
  bundle)
* stream custom classloading to handle the bundle-name prefix
* adds SimpleObject to osgi jar (and adds config keys to entity)
* adds tests for rebind with OSGi


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

Branch: refs/heads/master
Commit: a555beac618c931d0a1a682f34f1158cb23b3df2
Parents: 25605ca
Author: Aled Sage <al...@gmail.com>
Authored: Thu Jul 7 00:12:30 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Sat Jul 9 10:58:36 2016 +0100

----------------------------------------------------------------------
 .../brooklyn/camp/brooklyn/RebindOsgiTest.java  | 398 +++++++++++++++++++
 .../brooklyn/core/mgmt/ha/OsgiManager.java      |   1 -
 .../core/mgmt/persist/XmlMementoSerializer.java | 216 ++++++----
 .../core/mgmt/rebind/RebindIteration.java       |   4 +-
 .../brooklyn/util/core/ClassLoaderUtils.java    |  39 +-
 .../apache/brooklyn/util/core/osgi/Osgis.java   |   6 +
 ...entoSerializerDelegatingClassLoaderTest.java | 140 +++++++
 .../mgmt/persist/XmlMementoSerializerTest.java  | 115 +++++-
 .../util/core/ClassLoaderUtilsTest.java         |  12 +
 .../test/osgi/entities/SimpleApplication.java   |   1 -
 .../osgi/entities/SimpleApplicationImpl.java    |   1 -
 .../test/osgi/entities/SimpleEntity.java        |   9 +-
 .../test/osgi/entities/SimpleEntityImpl.java    |   1 -
 .../test/osgi/entities/SimpleObject.java        |  56 +++
 .../test/osgi/entities/SimplePolicy.java        |   6 +-
 .../brooklyn/util/osgi/VersionedName.java       |  11 +-
 .../brooklyn/util/osgi/OsgiTestResources.java   |   3 +
 17 files changed, 909 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/RebindOsgiTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/RebindOsgiTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/RebindOsgiTest.java
new file mode 100644
index 0000000..752b066
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/RebindOsgiTest.java
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.brooklyn.camp.brooklyn;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.mgmt.ManagementContext;
+import org.apache.brooklyn.api.policy.Policy;
+import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
+import org.apache.brooklyn.core.config.ConfigKeys;
+import org.apache.brooklyn.core.entity.Entities;
+import org.apache.brooklyn.core.entity.StartableApplication;
+import org.apache.brooklyn.core.mgmt.ha.OsgiManager;
+import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
+import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
+import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
+import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.core.test.entity.TestEntity;
+import org.apache.brooklyn.util.core.osgi.Osgis;
+import org.apache.brooklyn.util.exceptions.Exceptions;
+import org.apache.brooklyn.util.guava.Maybe;
+import org.apache.brooklyn.util.javalang.Reflections;
+import org.apache.brooklyn.util.osgi.OsgiTestResources;
+import org.jclouds.compute.domain.OsFamily;
+import org.osgi.framework.Bundle;
+import org.osgi.framework.launch.Framework;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.DataProvider;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+public class RebindOsgiTest extends AbstractYamlRebindTest {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RebindOsgiTest.class);
+
+    private static final String OSGI_BUNDLE_URL = OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL;
+    private static final String OSGI_BUNDLE_SYMBOLIC_NAME = "org.apache.brooklyn.test.resources.osgi.brooklyn-test-osgi-entities";
+    private static final String OSGI_ENTITY_TYPE = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY;
+    private static final String OSGI_POLICY_TYPE = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_POLICY;
+    private static final String OSGI_OBJECT_TYPE = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_OBJECT;
+    private static final String OSGI_ENTITY_CONFIG_NAME = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY_CONFIG_NAME;
+    private static final String OSGI_ENTITY_SENSOR_NAME = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY_SENSOR_NAME;
+
+    private List<String> bundleUrlsToInstallOnRebind;
+    
+    @BeforeMethod(alwaysRun = true)
+    @Override
+    public void setUp() throws Exception {
+        bundleUrlsToInstallOnRebind = Lists.newArrayList();
+        super.setUp();
+    }
+
+    @Override
+    protected boolean useOsgi() {
+        return true;
+    }
+    
+    @Override
+    protected LocalManagementContext createNewManagementContext(File mementoDir) {
+        LocalManagementContext result = super.createNewManagementContext(mementoDir);
+        for (String bundleUrl : bundleUrlsToInstallOnRebind) {
+            try {
+                installBundle(result, bundleUrl);
+            } catch (Exception e) {
+                throw Exceptions.propagate(e);
+            }
+        }
+        return result;
+    }
+    
+    @DataProvider(name = "valInEntityDataProvider")
+    public Object[][] valInEntityDataProvider() {
+        return new Object[][] {
+            {Predicates.alwaysTrue(), false},
+            {Predicates.alwaysTrue(), true},
+            {OsFamily.CENTOS, false},
+            {OsFamily.CENTOS, true},
+        };
+    }
+ 
+    @Test(dataProvider = "valInEntityDataProvider")
+    public void testValInEntity(Object val, boolean useOsgi) throws Exception {
+        String appSymbolicName = "my.catalog.app.id.load";
+        String appVersion = "0.1.0";
+        String appCatalogFormat;
+        if (useOsgi) {
+            appCatalogFormat = Joiner.on("\n").join(
+                    "brooklyn.catalog:",
+                    "  id: " + appSymbolicName,
+                    "  version: " + appVersion,
+                    "  itemType: entity",
+                    "  libraries:",
+                    "  - " + OSGI_BUNDLE_URL,
+                    "  item:",
+                    "    type: " + OSGI_ENTITY_TYPE);
+        } else {
+            appCatalogFormat = Joiner.on("\n").join(
+                    "brooklyn.catalog:",
+                    "  id: " + appSymbolicName,
+                    "  version: " + appVersion,
+                    "  itemType: entity",
+                    "  item:",
+                    "    type: " + TestEntity.class.getName());
+        }
+        
+        // Create the catalog items
+        Iterables.getOnlyElement(addCatalogItems(String.format(appCatalogFormat, appVersion)));
+        
+        // Create an app, using that catalog item
+        String appBlueprintYaml = Joiner.on("\n").join(
+                "location: localhost\n",
+                "services:",
+                "- type: " + CatalogUtils.getVersionedId(appSymbolicName, appVersion));
+        origApp = (StartableApplication) createAndStartApplication(appBlueprintYaml);
+        Entity origEntity = Iterables.getOnlyElement(origApp.getChildren());
+        origEntity.config().set(TestEntity.CONF_OBJECT, val);
+        
+        // Rebind
+        rebind();
+
+        Entity newEntity = Iterables.getOnlyElement(newApp.getChildren());
+        assertEquals(newEntity.config().get(TestEntity.CONF_OBJECT), val);
+    }
+
+    @Test
+    public void testValInEntityFromOtherBundle() throws Exception {
+        installBundle(mgmt(), OSGI_BUNDLE_URL);
+        bundleUrlsToInstallOnRebind.add(OSGI_BUNDLE_URL);
+        
+        // Create an app, using that catalog item
+        String appBlueprintYaml = Joiner.on("\n").join(
+                "services:",
+                "- type: " + TestEntity.class.getName());
+        origApp = (StartableApplication) createAndStartApplication(appBlueprintYaml);
+        Entity origEntity = Iterables.getOnlyElement(origApp.getChildren());
+
+        Object configVal = newOsgiSimpleObject("myEntityConfigVal");
+        origEntity.config().set(ConfigKeys.newConfigKey(Object.class, OSGI_ENTITY_CONFIG_NAME), configVal);
+        
+        // Rebind
+        rebind();
+
+        // Ensure app is still there, and that it is usable - e.g. "stop" effector functions as expected
+        Entity newEntity = Iterables.getOnlyElement(newApp.getChildren());
+
+        Object newConfigVal = newEntity.config().get(ConfigKeys.newConfigKey(Object.class, OSGI_ENTITY_CONFIG_NAME));
+        assertOsgiSimpleObjectsEqual(newConfigVal, configVal);
+    }
+    
+    @Test
+    public void testEntityAndPolicyFromCatalogOsgi() throws Exception {
+        String appSymbolicName = "my.catalog.app.id.load";
+        String appVersion = "0.1.0";
+        String appCatalogFormat = Joiner.on("\n").join(
+                "brooklyn.catalog:",
+                "  id: " + appSymbolicName,
+                "  version: " + appVersion,
+                "  itemType: entity",
+                "  libraries:",
+                "  - " + OSGI_BUNDLE_URL,
+                "  item:",
+                "    type: " + OSGI_ENTITY_TYPE,
+                "    brooklyn.policies:",
+                "    - type: " + OSGI_POLICY_TYPE);
+        
+        // Create the catalog items
+        Iterables.getOnlyElement(addCatalogItems(String.format(appCatalogFormat, appVersion)));
+        
+        // Create an app, using that catalog item
+        String appBlueprintYaml = Joiner.on("\n").join(
+                "location: localhost\n",
+                "services:",
+                "- type: " + CatalogUtils.getVersionedId(appSymbolicName, appVersion));
+        origApp = (StartableApplication) createAndStartApplication(appBlueprintYaml);
+        Entity origEntity = Iterables.getOnlyElement(origApp.getChildren());
+        Policy origPolicy = Iterables.getOnlyElement(origEntity.policies());
+
+        // Rebind
+        rebind();
+
+        // Ensure app is still there, and that it is usable - e.g. "stop" effector functions as expected
+        Entity newEntity = Iterables.getOnlyElement(newApp.getChildren());
+        Policy newPolicy = Iterables.getOnlyElement(newEntity.policies());
+        assertEquals(newEntity.getCatalogItemId(), appSymbolicName+":"+appVersion);
+        assertEquals(newPolicy.getId(), origPolicy.getId());
+
+        // Ensure stop works as expected
+        newApp.stop();
+        assertFalse(Entities.isManaged(newApp));
+        assertFalse(Entities.isManaged(newEntity));
+        
+        // Ensure can still use catalog item to deploy a new entity
+        StartableApplication app2 = (StartableApplication) createAndStartApplication(appBlueprintYaml);
+        Entity entity2 = Iterables.getOnlyElement(app2.getChildren());
+        assertEquals(entity2.getCatalogItemId(), appSymbolicName+":"+appVersion);
+    }
+
+    @Test
+    public void testJavaPojoFromCatalogOsgi() throws Exception {
+        String appSymbolicName = "my.catalog.app.id.load";
+        String appVersion = "0.1.0";
+        String appCatalogFormat = Joiner.on("\n").join(
+                "brooklyn.catalog:",
+                "  id: " + appSymbolicName,
+                "  version: " + appVersion,
+                "  itemType: entity",
+                "  libraries:",
+                "  - " + OSGI_BUNDLE_URL,
+                "  item:",
+                "    type: " + OSGI_ENTITY_TYPE);
+        
+        // Create the catalog items
+        Iterables.getOnlyElement(addCatalogItems(String.format(appCatalogFormat, appVersion)));
+        
+        // Create an app, using that catalog item
+        String appBlueprintYaml = Joiner.on("\n").join(
+                "location: localhost\n",
+                "services:",
+                "- type: " + CatalogUtils.getVersionedId(appSymbolicName, appVersion));
+        origApp = (StartableApplication) createAndStartApplication(appBlueprintYaml);
+        Entity origEntity = Iterables.getOnlyElement(origApp.getChildren());
+
+        Object configVal = newOsgiSimpleObject("myEntityConfigVal");
+        Object sensorVal = newOsgiSimpleObject("myEntitySensorVal");
+        origEntity.config().set(ConfigKeys.newConfigKey(Object.class, OSGI_ENTITY_CONFIG_NAME), configVal);
+        origEntity.sensors().set(Sensors.newSensor(Object.class, OSGI_ENTITY_SENSOR_NAME), sensorVal);
+        
+        // Rebind
+        rebind();
+
+        // Ensure app is still there, and that it is usable - e.g. "stop" effector functions as expected
+        Entity newEntity = Iterables.getOnlyElement(newApp.getChildren());
+
+        Object newConfigVal = newEntity.config().get(ConfigKeys.newConfigKey(Object.class, OSGI_ENTITY_CONFIG_NAME));
+        Object newSensorVal = newEntity.sensors().get(Sensors.newSensor(Object.class, OSGI_ENTITY_SENSOR_NAME));
+        assertOsgiSimpleObjectsEqual(newConfigVal, configVal);
+        assertOsgiSimpleObjectsEqual(newSensorVal, sensorVal);
+    }
+    
+    @Test
+    public void testBrooklynObjectDslFromCatalogOsgi() throws Exception {
+        String appSymbolicName = "my.catalog.app.id.load";
+        String appVersion = "0.1.0";
+        String appCatalogFormat = Joiner.on("\n").join(
+                "brooklyn.catalog:",
+                "  id: " + appSymbolicName,
+                "  version: " + appVersion,
+                "  itemType: entity",
+                "  libraries:",
+                "  - " + OSGI_BUNDLE_URL,
+                "  item:",
+                "    type: " + OSGI_ENTITY_TYPE,
+                "    brooklyn.config:",
+                "      " + OSGI_ENTITY_CONFIG_NAME + ":",
+                "        $brooklyn:object:",
+                "          type: " + OSGI_OBJECT_TYPE,
+                "          object.fields:",
+                "            val: myEntityVal");
+        
+        // Create the catalog items
+        Iterables.getOnlyElement(addCatalogItems(String.format(appCatalogFormat, appVersion)));
+        
+        // Create an app, using that catalog item
+        String appBlueprintYaml = Joiner.on("\n").join(
+                "location: localhost\n",
+                "services:",
+                "- type: " + CatalogUtils.getVersionedId(appSymbolicName, appVersion));
+        origApp = (StartableApplication) createAndStartApplication(appBlueprintYaml);
+        Entity origEntity = Iterables.getOnlyElement(origApp.getChildren());
+
+        Object configVal = origEntity.config().get(ConfigKeys.newConfigKey(Object.class, OSGI_ENTITY_CONFIG_NAME));
+        assertEquals(getOsgiSimpleObjectsVal(configVal), "myEntityVal");
+        
+        // Rebind
+        rebind();
+
+        // Ensure app is still there, and that it is usable - e.g. "stop" effector functions as expected
+        Entity newEntity = Iterables.getOnlyElement(newApp.getChildren());
+
+        Object newConfigVal = newEntity.config().get(ConfigKeys.newConfigKey(Object.class, OSGI_ENTITY_CONFIG_NAME));
+        assertOsgiSimpleObjectsEqual(newConfigVal, configVal);
+        
+        // Ensure stop works as expected
+        newApp.stop();
+        assertFalse(Entities.isManaged(newApp));
+        assertFalse(Entities.isManaged(newEntity));
+        
+        // Ensure can still use catalog item to deploy a new entity
+        StartableApplication app2 = (StartableApplication) createAndStartApplication(appBlueprintYaml);
+        Entity entity2 = Iterables.getOnlyElement(app2.getChildren());
+        assertEquals(entity2.getCatalogItemId(), appSymbolicName+":"+appVersion);
+    }
+    
+    // TODO Does not do rebind; the config isn't there after rebind.
+    // Need to reproduce that in a simpler use-case.
+    @Test
+    public void testBrooklynObjectDslFromCatalogOsgiInPolicy() throws Exception {
+        String appSymbolicName = "my.catalog.app.id.load";
+        String appVersion = "0.1.0";
+        String appCatalogFormat = Joiner.on("\n").join(
+                "brooklyn.catalog:",
+                "  id: " + appSymbolicName,
+                "  version: " + appVersion,
+                "  itemType: entity",
+                "  libraries:",
+                "  - " + OSGI_BUNDLE_URL,
+                "  item:",
+                "    type: " + OSGI_ENTITY_TYPE,
+                "    brooklyn.policies:",
+                "    - type: " + OSGI_POLICY_TYPE,
+                "      brooklyn.config:",
+                "        " + OSGI_ENTITY_CONFIG_NAME + ":",
+                "          $brooklyn:object:",
+                "            type: " + OSGI_OBJECT_TYPE,
+                "            object.fields:",
+                "              val: myPolicyVal");
+        
+        // Create the catalog items
+        Iterables.getOnlyElement(addCatalogItems(String.format(appCatalogFormat, appVersion)));
+        
+        // Create an app, using that catalog item
+        String appBlueprintYaml = Joiner.on("\n").join(
+                "location: localhost\n",
+                "services:",
+                "- type: " + CatalogUtils.getVersionedId(appSymbolicName, appVersion));
+        origApp = (StartableApplication) createAndStartApplication(appBlueprintYaml);
+        Entity origEntity = Iterables.getOnlyElement(origApp.getChildren());
+        Policy origPolicy = Iterables.getOnlyElement(origEntity.policies());
+
+        Object policyConfigVal = origPolicy.config().get(ConfigKeys.newConfigKey(Object.class, OSGI_ENTITY_CONFIG_NAME));
+        assertEquals(getOsgiSimpleObjectsVal(policyConfigVal), "myPolicyVal");
+    }
+    
+    private Bundle getBundle(ManagementContext mgmt, final String symbolicName) throws Exception {
+        OsgiManager osgiManager = ((ManagementContextInternal)mgmt).getOsgiManager().get();
+        Framework framework = osgiManager.getFramework();
+        Maybe<Bundle> result = Osgis.bundleFinder(framework)
+                .symbolicName(symbolicName)
+                .find();
+        return result.get();
+    }
+    
+    private Object newOsgiSimpleObject(String val) throws Exception {
+        Class<?> osgiObjectClazz = getBundle(mgmt(), OSGI_BUNDLE_SYMBOLIC_NAME).loadClass(OSGI_OBJECT_TYPE);
+        return Reflections.invokeConstructorWithArgs(osgiObjectClazz, val).get();
+    }
+    
+    private void assertOsgiSimpleObjectsEqual(Object val1, Object val2) throws Exception {
+        if (val2 == null) {
+            assertNull(val1);
+        } else {
+            assertNotNull(val1);
+        }
+        assertEquals(val1.getClass().getName(), val2.getClass().getName());
+        assertEquals(getOsgiSimpleObjectsVal(val1), getOsgiSimpleObjectsVal(val2));
+    }
+
+    private String getOsgiSimpleObjectsVal(Object val) throws Exception {
+        assertNotNull(val);
+        return (String) Reflections.invokeMethodWithArgs(val, "getVal", ImmutableList.of()).get();
+    }
+    
+    private Bundle installBundle(ManagementContext mgmt, String bundleUrl) throws Exception {
+        OsgiManager osgiManager = ((ManagementContextInternal)mgmt).getOsgiManager().get();
+        Framework framework = osgiManager.getFramework();
+        return Osgis.install(framework, bundleUrl);
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiManager.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiManager.java
index 28b20b3..2c68c8a 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiManager.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiManager.java
@@ -45,7 +45,6 @@ import org.apache.brooklyn.util.exceptions.Exceptions;
 import org.apache.brooklyn.util.guava.Maybe;
 import org.apache.brooklyn.util.os.Os;
 import org.apache.brooklyn.util.os.Os.DeletionResult;
-import org.apache.brooklyn.util.osgi.SystemFramework;
 import org.apache.brooklyn.util.repeat.Repeater;
 import org.apache.brooklyn.util.text.Strings;
 import org.apache.brooklyn.util.time.Duration;

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializer.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializer.java
index 58df79f..15b3346 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializer.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializer.java
@@ -61,22 +61,29 @@ import org.apache.brooklyn.core.mgmt.rebind.dto.BasicPolicyMemento;
 import org.apache.brooklyn.core.mgmt.rebind.dto.MutableBrooklynMemento;
 import org.apache.brooklyn.core.sensor.BasicAttributeSensor;
 import org.apache.brooklyn.util.core.ClassLoaderUtils;
+import org.apache.brooklyn.util.core.osgi.Osgis;
 import org.apache.brooklyn.util.core.xstream.XmlSerializer;
 import org.apache.brooklyn.util.exceptions.Exceptions;
 import org.apache.brooklyn.util.text.Strings;
+import org.osgi.framework.Bundle;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.thoughtworks.xstream.MarshallingStrategy;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
 import com.thoughtworks.xstream.converters.Converter;
 import com.thoughtworks.xstream.converters.MarshallingContext;
 import com.thoughtworks.xstream.converters.SingleValueConverter;
 import com.thoughtworks.xstream.converters.UnmarshallingContext;
 import com.thoughtworks.xstream.converters.reflection.ReflectionConverter;
+import com.thoughtworks.xstream.core.ClassLoaderReference;
 import com.thoughtworks.xstream.core.ReferencingMarshallingContext;
+import com.thoughtworks.xstream.core.util.Primitives;
 import com.thoughtworks.xstream.io.HierarchicalStreamReader;
 import com.thoughtworks.xstream.io.HierarchicalStreamWriter;
 import com.thoughtworks.xstream.io.path.PathTrackingReader;
+import com.thoughtworks.xstream.mapper.CannotResolveClassException;
+import com.thoughtworks.xstream.mapper.DefaultMapper;
 import com.thoughtworks.xstream.mapper.Mapper;
 import com.thoughtworks.xstream.mapper.MapperWrapper;
 
@@ -88,30 +95,17 @@ public class XmlMementoSerializer<T> extends XmlSerializer<T> implements Memento
 
     private static final Logger LOG = LoggerFactory.getLogger(XmlMementoSerializer.class);
 
-    private final ClassLoader classLoader;
+    private final OsgiClassLoader delegatingClassLoader;
     private LookupContext lookupContext;
-
+    
     public XmlMementoSerializer(ClassLoader classLoader) {
         this(classLoader, DeserializingClassRenamesProvider.loadDeserializingClassRenames());
     }
     
-    private static class CustomClassLoader extends ClassLoader {
-        private ClassLoaderUtils loader;
-        private CustomClassLoader(ClassLoader cl) {
-            loader = new ClassLoaderUtils(cl);
-        }
-
-        @Override
-        protected Class<?> findClass(String name) throws ClassNotFoundException {
-            return loader.loadClass(name);
-        }
-        
-    }
-    
     public XmlMementoSerializer(ClassLoader classLoader, Map<String, String> deserializingClassRenames) {
         super(deserializingClassRenames);
-        this.classLoader = checkNotNull(classLoader, "classLoader");
-        xstream.setClassLoader(new CustomClassLoader(this.classLoader));
+        this.delegatingClassLoader = new OsgiClassLoader(classLoader);
+        xstream.setClassLoader(this.delegatingClassLoader);
         
         // old (deprecated in 070? or earlier) single-file persistence uses this keyword; TODO remove soon in 080 ?
         xstream.alias("brooklyn", MutableBrooklynMemento.class);
@@ -156,6 +150,7 @@ public class XmlMementoSerializer<T> extends XmlSerializer<T> implements Memento
     @Override
     protected MapperWrapper wrapMapperForNormalUsage(Mapper next) {
         MapperWrapper mapper = super.wrapMapperForNormalUsage(next);
+        mapper = new OsgiClassnameMapper(mapper);
         mapper = new CustomMapper(mapper, Entity.class, "entityProxy");
         mapper = new CustomMapper(mapper, Location.class, "locationProxy");
         mapper = new UnwantedStateLoggingMapper(mapper);
@@ -175,6 +170,7 @@ public class XmlMementoSerializer<T> extends XmlSerializer<T> implements Memento
     @Override
     public void setLookupContext(LookupContext lookupContext) {
         this.lookupContext = checkNotNull(lookupContext, "lookupContext");
+        delegatingClassLoader.setManagementContext(lookupContext.lookupManagementContext());
     }
 
     @Override
@@ -410,6 +406,47 @@ public class XmlMementoSerializer<T> extends XmlSerializer<T> implements Memento
         }
     }
 
+    public class OsgiClassnameMapper extends MapperWrapper {
+        private final ClassLoaderUtils whiteListRetriever;
+        
+        OsgiClassnameMapper(MapperWrapper mapper) {
+            super(mapper);
+            whiteListRetriever = new ClassLoaderUtils(getClass());
+        }
+        
+        @Override
+        public String serializedClass(Class type) {
+            // TODO What if previous stages have already renamed it?
+            // For example the "outer class renaming stuff"?!
+            String superResult = super.serializedClass(type);
+            if (type != null && type.getName().equals(superResult)) {
+                Optional<Bundle> bundle  = Osgis.getBundleOf(type);
+                if (bundle.isPresent() && !whiteListRetriever.isBundleWhiteListed(bundle.get())) {
+                    return bundle.get().getSymbolicName() + ":" + superResult;
+                }
+            }
+            return superResult;
+        }
+        
+        @Override
+        public Class realClass(String elementName) {
+            CannotResolveClassException tothrow;
+            try {
+                return super.realClass(elementName);
+            } catch (CannotResolveClassException e) {
+                tothrow = e;
+            }
+
+            // Class.forName(elementName, false, classLader) does not seem to like us returned a 
+            // class whose name does not match that passed in. Therefore fallback to using loadClass.
+            try {
+                return xstream.getClassLoaderReference().getReference().loadClass(elementName);
+            } catch (ClassNotFoundException e) {
+                throw new CannotResolveClassException(elementName + " via loadClass", tothrow);
+            }
+        }
+    }
+    
     /** When reading/writing specs, it checks whether there is a catalog item id set and uses it to load */
     public class SpecConverter extends ReflectionConverter {
         SpecConverter() {
@@ -462,7 +499,7 @@ public class XmlMementoSerializer<T> extends XmlSerializer<T> implements Memento
                     RegisteredType cat = lookupContext.lookupManagementContext().getTypeRegistry().get(catalogItemId);
                     if (cat==null) throw new NoSuchElementException("catalog item: "+catalogItemId);
                     BrooklynClassLoadingContext clcNew = CatalogUtils.newClassLoadingContext(lookupContext.lookupManagementContext(), cat);
-                    pushXstreamCustomClassLoader(clcNew);
+                    delegatingClassLoader.pushXstreamCustomClassLoader(clcNew);
                     customLoaderSet = true;
                 }
                 
@@ -473,7 +510,7 @@ public class XmlMementoSerializer<T> extends XmlSerializer<T> implements Memento
             } finally {
                 context.put("SpecConverter.instance", null);
                 if (customLoaderSet) {
-                    popXstreamCustomClassLoader();
+                    delegatingClassLoader.popXstreamCustomClassLoader();
                 }
             }
         }
@@ -497,68 +534,99 @@ public class XmlMementoSerializer<T> extends XmlSerializer<T> implements Memento
         }
     }
     
-    Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
-    Stack<ClassLoader> cls = new Stack<ClassLoader>();
-    AtomicReference<Thread> xstreamLockOwner = new AtomicReference<Thread>();
-    int lockCount;
-    
-    /** Must be accompanied by a corresponding {@link #popXstreamCustomClassLoader()} when finished. */
-    @SuppressWarnings("deprecation")
-    protected void pushXstreamCustomClassLoader(BrooklynClassLoadingContext clcNew) {
-        acquireXstreamLock();
-        BrooklynClassLoadingContext oldClc;
-        if (!contexts.isEmpty()) {
-            oldClc = contexts.peek();
-        } else {
-            // TODO XmlMementoSerializer should take a BCLC instead of a CL
-            oldClc = JavaBrooklynClassLoadingContext.create(lookupContext.lookupManagementContext(), xstream.getClassLoader());
-        }
-        BrooklynClassLoadingContextSequential clcMerged = new BrooklynClassLoadingContextSequential(lookupContext.lookupManagementContext(),
-            oldClc, clcNew);
-        contexts.push(clcMerged);
-        cls.push(xstream.getClassLoader());
-        ClassLoader newCL = ClassLoaderFromBrooklynClassLoadingContext.of(clcMerged);
-        xstream.setClassLoader(newCL);
-    }
+    @VisibleForTesting
+    static class OsgiClassLoader extends ClassLoader {
+        private final Stack<BrooklynClassLoadingContext> contexts = new Stack<BrooklynClassLoadingContext>();
+        private final Stack<ClassLoader> cls = new Stack<ClassLoader>();
+        private final AtomicReference<Thread> xstreamLockOwner = new AtomicReference<Thread>();
+        private ManagementContext mgmt;
+        private ClassLoader currentClassLoader;
+        private AtomicReference<ClassLoaderUtils> currentLoader = new AtomicReference<>();
+        private int lockCount;
+        
+        protected OsgiClassLoader(ClassLoader classLoader) {
+            setCurrentClassLoader(classLoader);
+        }
+        
+        protected void setManagementContext(ManagementContext mgmt) {
+            this.mgmt = checkNotNull(mgmt, "mgmt");
+            currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
+        }
 
-    protected void popXstreamCustomClassLoader() {
-        synchronized (xstreamLockOwner) {
-            releaseXstreamLock();
-            xstream.setClassLoader(cls.pop());
-            contexts.pop();
+        @Override
+        protected Class<?> findClass(String name) throws ClassNotFoundException {
+            return currentLoader.get().loadClass(name);
         }
-    }
-    
-    protected void acquireXstreamLock() {
-        synchronized (xstreamLockOwner) {
-            while (true) {
-                if (xstreamLockOwner.compareAndSet(null, Thread.currentThread()) || 
-                    Thread.currentThread().equals( xstreamLockOwner.get() )) {
-                    break;
-                }
-                try {
-                    xstreamLockOwner.wait(1000);
-                } catch (InterruptedException e) {
-                    throw Exceptions.propagate(e);
+
+        /** Must be accompanied by a corresponding {@link #popXstreamCustomClassLoader()} when finished. */
+        @SuppressWarnings("deprecation")
+        protected void pushXstreamCustomClassLoader(BrooklynClassLoadingContext clcNew) {
+            acquireXstreamLock();
+            BrooklynClassLoadingContext oldClc;
+            if (!contexts.isEmpty()) {
+                oldClc = contexts.peek();
+            } else {
+                // TODO XmlMementoSerializer should take a BCLC instead of a CL
+                oldClc = JavaBrooklynClassLoadingContext.create(mgmt, getCurrentClassLoader());
+            }
+            BrooklynClassLoadingContextSequential clcMerged = new BrooklynClassLoadingContextSequential(mgmt, oldClc, clcNew);
+            ClassLoader newCL = ClassLoaderFromBrooklynClassLoadingContext.of(clcMerged);
+            contexts.push(clcMerged);
+            cls.push(getCurrentClassLoader());
+            setCurrentClassLoader(newCL);
+        }
+
+        protected void popXstreamCustomClassLoader() {
+            synchronized (xstreamLockOwner) {
+                releaseXstreamLock();
+                setCurrentClassLoader(cls.pop());
+                contexts.pop();
+            }
+        }
+        
+        private ClassLoader getCurrentClassLoader() {
+            return currentClassLoader;
+        }
+        
+        private void setCurrentClassLoader(ClassLoader classLoader) {
+            currentClassLoader = checkNotNull(classLoader);
+            if (mgmt != null) {
+                currentLoader.set(new ClassLoaderUtils(currentClassLoader, mgmt));
+            } else {
+                currentLoader.set(new ClassLoaderUtils(currentClassLoader));
+            }
+        }
+        
+        protected void acquireXstreamLock() {
+            synchronized (xstreamLockOwner) {
+                while (true) {
+                    if (xstreamLockOwner.compareAndSet(null, Thread.currentThread()) || 
+                        Thread.currentThread().equals( xstreamLockOwner.get() )) {
+                        break;
+                    }
+                    try {
+                        xstreamLockOwner.wait(1000);
+                    } catch (InterruptedException e) {
+                        throw Exceptions.propagate(e);
+                    }
                 }
+                lockCount++;
             }
-            lockCount++;
         }
-    }
 
-    protected void releaseXstreamLock() {
-        synchronized (xstreamLockOwner) {
-            if (lockCount<=0) {
-                throw new IllegalStateException("xstream not locked");
-            }
-            if (--lockCount == 0) {
-                if (!xstreamLockOwner.compareAndSet(Thread.currentThread(), null)) {
-                    Thread oldOwner = xstreamLockOwner.getAndSet(null);
-                    throw new IllegalStateException("xstream was locked by "+oldOwner+" but unlock attempt by "+Thread.currentThread());
+        protected void releaseXstreamLock() {
+            synchronized (xstreamLockOwner) {
+                if (lockCount<=0) {
+                    throw new IllegalStateException("xstream not locked");
+                }
+                if (--lockCount == 0) {
+                    if (!xstreamLockOwner.compareAndSet(Thread.currentThread(), null)) {
+                        Thread oldOwner = xstreamLockOwner.getAndSet(null);
+                        throw new IllegalStateException("xstream was locked by "+oldOwner+" but unlock attempt by "+Thread.currentThread());
+                    }
+                    xstreamLockOwner.notifyAll();
                 }
-                xstreamLockOwner.notifyAll();
             }
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindIteration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindIteration.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindIteration.java
index 1f1ac2e..4aee9a8 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindIteration.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindIteration.java
@@ -976,9 +976,9 @@ public abstract class RebindIteration {
                         return new LoadedClass<T>((Class<? extends T>) catalogClass.get(), catalogItemId);
                     }
                 }
-                throw new IllegalStateException("No catalogItemId specified for "+contextSuchAsId+" and can't load class from either classpath or catalog items");
+                throw new IllegalStateException("No catalogItemId specified for "+contextSuchAsId+" and can't load class (" + jType + ") from either classpath or catalog items");
             } else {
-                throw new IllegalStateException("No catalogItemId specified for "+contextSuchAsId+" and can't load class from classpath");
+                throw new IllegalStateException("No catalogItemId specified for "+contextSuchAsId+" and can't load class (" + jType + ") from classpath");
             }
         }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/core/src/main/java/org/apache/brooklyn/util/core/ClassLoaderUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/util/core/ClassLoaderUtils.java b/core/src/main/java/org/apache/brooklyn/util/core/ClassLoaderUtils.java
index 00fd05f..bc46b60 100644
--- a/core/src/main/java/org/apache/brooklyn/util/core/ClassLoaderUtils.java
+++ b/core/src/main/java/org/apache/brooklyn/util/core/ClassLoaderUtils.java
@@ -15,6 +15,8 @@
  */
 package org.apache.brooklyn.util.core;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.util.List;
 import java.util.regex.Pattern;
 
@@ -41,6 +43,7 @@ import org.osgi.framework.launch.Framework;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.Beta;
 import com.google.common.base.Predicate;
 
 public class ClassLoaderUtils {
@@ -69,27 +72,33 @@ public class ClassLoaderUtils {
     }
 
     public ClassLoaderUtils(Class<?> callingClass) {
-        this.classLoader = callingClass.getClassLoader();
+        this.classLoader = checkNotNull(callingClass, "callingClass").getClassLoader();
         this.entity = null;
         this.mgmt = null;
     }
 
     public ClassLoaderUtils(ClassLoader cl) {
-        this.classLoader = cl;
+        this.classLoader = checkNotNull(cl, "classLoader");
         this.entity = null;
         this.mgmt = null;
     }
 
+    public ClassLoaderUtils(ClassLoader cl, @Nullable ManagementContext mgmt) {
+        this.classLoader = checkNotNull(cl, "classLoader");
+        this.entity = null;
+        this.mgmt = checkNotNull(mgmt, "mgmt");
+    }
+
     public ClassLoaderUtils(Class<?> callingClass, Entity entity) {
-        this.classLoader = callingClass.getClassLoader();
-        this.entity = entity;
+        this.classLoader = checkNotNull(callingClass, "callingClass").getClassLoader();
+        this.entity = checkNotNull(entity, "entity");
         this.mgmt = ((EntityInternal)entity).getManagementContext();
     }
 
     public ClassLoaderUtils(Class<?> callingClass, @Nullable ManagementContext mgmt) {
-        this.classLoader = callingClass.getClassLoader();
+        this.classLoader = checkNotNull(callingClass, "callingClass").getClassLoader();
         this.entity = null;
-        this.mgmt = mgmt;
+        this.mgmt = checkNotNull(mgmt, "mgmt");
     }
 
     public Class<?> loadClass(String name) throws ClassNotFoundException {
@@ -139,7 +148,9 @@ public class ClassLoaderUtils {
 
         try {
             // Used instead of callingClass.getClassLoader().loadClass(...) as it could be null (only for bootstrap classes)
-            return Class.forName(name, true, classLoader);
+            // Note that Class.forName(name, false, classLoader) doesn't seem to like us returning a 
+            // class with a different name from that intended (e.g. stripping off an OSGi prefix).
+            return classLoader.loadClass(name);
         } catch (ClassNotFoundException e) {
         }
 
@@ -181,6 +192,12 @@ public class ClassLoaderUtils {
         }
     }
 
+    @Beta
+    public boolean isBundleWhiteListed(Bundle bundle) {
+        WhiteListBundlePredicate p = createBundleMatchingPredicate();
+        return p.apply(bundle);
+    }
+
     protected Framework getFramework() {
         if (mgmt != null) {
             Maybe<OsgiManager> osgiManager = ((ManagementContextInternal)mgmt).getOsgiManager();
@@ -206,11 +223,11 @@ public class ClassLoaderUtils {
 
 
     private static class WhiteListBundlePredicate implements Predicate<Bundle> {
-        private Pattern symbolicName;
-        private Pattern version;
+        private final Pattern symbolicName;
+        private final Pattern version;
 
         private WhiteListBundlePredicate(String symbolicName, String version) {
-            this.symbolicName = Pattern.compile(symbolicName);
+            this.symbolicName = Pattern.compile(checkNotNull(symbolicName, "symbolicName"));
             this.version = version != null ? Pattern.compile(version) : null;
         }
 
@@ -219,7 +236,6 @@ public class ClassLoaderUtils {
             return symbolicName.matcher(input.getSymbolicName()).matches() &&
                     (version == null || version.matcher(input.getVersion().toString()).matches());
         }
-    
     }
 
     private Class<?> tryLoadFromBundleWhiteList(String name) {
@@ -253,5 +269,4 @@ public class ClassLoaderUtils {
         }
         return new WhiteListBundlePredicate(symbolicName, version);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/core/src/main/java/org/apache/brooklyn/util/core/osgi/Osgis.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/util/core/osgi/Osgis.java b/core/src/main/java/org/apache/brooklyn/util/core/osgi/Osgis.java
index 82f8723..a33b446 100644
--- a/core/src/main/java/org/apache/brooklyn/util/core/osgi/Osgis.java
+++ b/core/src/main/java/org/apache/brooklyn/util/core/osgi/Osgis.java
@@ -50,6 +50,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.Beta;
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
+import com.google.common.base.Optional;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 
@@ -438,4 +439,9 @@ public class Osgis {
         });
     }
 
+    @Beta
+    public static Optional<Bundle> getBundleOf(Class<?> clazz) {
+        Bundle bundle = org.osgi.framework.FrameworkUtil.getBundle(clazz);
+        return Optional.fromNullable(bundle);
+    }
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/core/src/test/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializerDelegatingClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializerDelegatingClassLoaderTest.java b/core/src/test/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializerDelegatingClassLoaderTest.java
new file mode 100644
index 0000000..deb240a
--- /dev/null
+++ b/core/src/test/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializerDelegatingClassLoaderTest.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.brooklyn.core.mgmt.persist;
+
+import static org.testng.Assert.assertEquals;
+
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.mgmt.ManagementContext;
+import org.apache.brooklyn.core.entity.AbstractEntity;
+import org.apache.brooklyn.core.entity.Entities;
+import org.apache.brooklyn.core.mgmt.ha.OsgiManager;
+import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
+import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
+import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
+import org.apache.brooklyn.core.mgmt.persist.XmlMementoSerializer.OsgiClassLoader;
+import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests;
+import org.apache.brooklyn.util.core.osgi.Osgis;
+import org.apache.brooklyn.util.guava.Maybe;
+import org.apache.brooklyn.util.osgi.OsgiTestResources;
+import org.osgi.framework.Bundle;
+import org.osgi.framework.launch.Framework;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Optional;
+
+public class XmlMementoSerializerDelegatingClassLoaderTest {
+
+    private LocalManagementContext mgmt;
+
+    @BeforeMethod(alwaysRun=true)
+    public void setUp() throws Exception {
+        mgmt = LocalManagementContextForTests.builder(true).disableOsgi(false).build();
+    }
+
+    @AfterMethod(alwaysRun=true)
+    public void tearDown() throws Exception {
+        if (mgmt != null) {
+            Entities.destroyAll(mgmt);
+        }
+    }
+    
+    @Test
+    public void testLoadClassFromBundle() throws Exception {
+        ClassLoader classLoader = getClass().getClassLoader();
+        Bundle apiBundle = getBundle(mgmt, "org.apache.brooklyn.api");
+        Bundle coreBundle = getBundle(mgmt, "org.apache.brooklyn.core");
+        
+        String bundleUrl = OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL;
+        Bundle otherBundle = installBundle(mgmt, bundleUrl);
+        
+        assertLoads(classLoader, Entity.class, Optional.of(apiBundle));
+        assertLoads(classLoader, AbstractEntity.class, Optional.of(coreBundle));
+        assertLoads(classLoader, OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY, Optional.of(otherBundle));
+    }
+    
+    @Test
+    public void testLoadClassVanilla() throws Exception {
+        ClassLoader classLoader = getClass().getClassLoader();
+        
+        assertLoads(classLoader, Entity.class, Optional.<Bundle>absent());
+        assertLoads(classLoader, AbstractEntity.class, Optional.<Bundle>absent());
+    }
+    
+    // Tests we can do funny stuff, like return a differently named class from that expected!
+    @Test
+    public void testLoadClassReturningDifferentlyNamedClass() throws Exception {
+        final String specialClassName = "my.madeup.Clazz";
+        
+        ClassLoader classLoader = new ClassLoader() {
+            @Override
+            protected Class<?> findClass(String name) throws ClassNotFoundException {
+                if (name != null && name.equals(specialClassName)) {
+                    return Entity.class;
+                }
+                return getClass().getClassLoader().loadClass(name);
+            }
+        };
+        
+        OsgiClassLoader ocl = new XmlMementoSerializer.OsgiClassLoader(classLoader);
+        ocl.setManagementContext(mgmt);
+        assertEquals(ocl.loadClass(specialClassName), Entity.class);
+        
+        // TODO The line below fails: java.lang.ClassNotFoundException: my/madeup/Clazz
+        //assertEquals(Class.forName(specialClassName, false, ocl).getName(), Entity.class.getName());
+    }
+    
+    private void assertLoads(ClassLoader delegateClassLoader, Class<?> clazz, Optional<Bundle> bundle) throws Exception {
+        OsgiClassLoader ocl = new XmlMementoSerializer.OsgiClassLoader(delegateClassLoader);
+        ocl.setManagementContext(mgmt);
+        String classname = (bundle.isPresent() ? bundle.get().getSymbolicName() + ":" : "") + clazz.getName();
+        assertEquals(ocl.loadClass(classname), clazz);
+        
+        // TODO The line below fails, e.g.: java.lang.ClassNotFoundException: org/apache/brooklyn/api:org/apache/brooklyn/api/entity/Entity
+        //assertEquals(Class.forName(classname, false, ocl), clazz);
+        
+    }
+
+    private void assertLoads(ClassLoader delegateClassLoader, String clazz, Optional<Bundle> bundle) throws Exception {
+        OsgiClassLoader ocl = new XmlMementoSerializer.OsgiClassLoader(delegateClassLoader);
+        ocl.setManagementContext(mgmt);
+        String classname = (bundle.isPresent() ? bundle.get().getSymbolicName() + ":" : "") + clazz;
+        assertEquals(ocl.loadClass(classname).getName(), clazz);
+        
+        // TODO The line below fails, e.g.: java.lang.ClassNotFoundException: org/apache/brooklyn/test/resources/osgi/brooklyn-test-osgi-entities:org/apache/brooklyn/test/osgi/entities/SimpleEntity
+        //assertEquals(Class.forName(classname, false, ocl).getName(), clazz);
+    }
+
+    private Bundle getBundle(ManagementContext mgmt, final String symbolicName) throws Exception {
+        OsgiManager osgiManager = ((ManagementContextInternal)mgmt).getOsgiManager().get();
+        Framework framework = osgiManager.getFramework();
+        Maybe<Bundle> result = Osgis.bundleFinder(framework)
+                .symbolicName(symbolicName)
+                .find();
+        return result.get();
+    }
+    
+    private Bundle installBundle(ManagementContext mgmt, String bundleUrl) throws Exception {
+        OsgiManager osgiManager = ((ManagementContextInternal)mgmt).getOsgiManager().get();
+        Framework framework = osgiManager.getFramework();
+        return Osgis.install(framework, bundleUrl);
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/core/src/test/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializerTest.java b/core/src/test/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializerTest.java
index aefd623..9251354 100644
--- a/core/src/test/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializerTest.java
+++ b/core/src/test/java/org/apache/brooklyn/core/mgmt/persist/XmlMementoSerializerTest.java
@@ -50,6 +50,9 @@ import org.apache.brooklyn.core.catalog.internal.CatalogItemBuilder;
 import org.apache.brooklyn.core.catalog.internal.CatalogItemDtoAbstract;
 import org.apache.brooklyn.core.entity.Entities;
 import org.apache.brooklyn.core.location.SimulatedLocation;
+import org.apache.brooklyn.core.mgmt.ha.OsgiManager;
+import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
+import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
 import org.apache.brooklyn.core.mgmt.osgi.OsgiVersionMoreEntityTest;
 import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests;
 import org.apache.brooklyn.core.test.entity.TestApplication;
@@ -59,16 +62,22 @@ import org.apache.brooklyn.test.support.TestResourceUnavailableException;
 import org.apache.brooklyn.util.collections.MutableList;
 import org.apache.brooklyn.util.collections.MutableMap;
 import org.apache.brooklyn.util.collections.MutableSet;
+import org.apache.brooklyn.util.core.osgi.Osgis;
+import org.apache.brooklyn.util.javalang.Reflections;
 import org.apache.brooklyn.util.net.Networking;
 import org.apache.brooklyn.util.net.UserAndHostAndPort;
 import org.apache.brooklyn.util.osgi.OsgiTestResources;
+import org.osgi.framework.Bundle;
+import org.osgi.framework.launch.Framework;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 import com.google.api.client.repackaged.com.google.common.base.Joiner;
 import com.google.common.base.Objects;
+import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -84,12 +93,21 @@ public class XmlMementoSerializerTest {
     private static final Logger LOG = LoggerFactory.getLogger(XmlMementoSerializerTest.class);
 
     private XmlMementoSerializer<Object> serializer;
-
+    private ManagementContext mgmt;
+    
     @BeforeMethod(alwaysRun=true)
     public void setUp() throws Exception {
         serializer = new XmlMementoSerializer<Object>(XmlMementoSerializerTest.class.getClassLoader());
     }
 
+    @AfterMethod(alwaysRun=true)
+    private void tearDown() {
+        if (mgmt != null) {
+            Entities.destroyAllCatching(mgmt);
+            mgmt = null;
+        }
+    }
+    
     @Test
     public void testRenamedClass() throws Exception {
         serializer = new XmlMementoSerializer<Object>(XmlMementoSerializerTest.class.getClassLoader(),
@@ -337,23 +355,90 @@ public class XmlMementoSerializerTest {
     @Test
     public void testEntitySpecFromOsgi() throws Exception {
         TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiTestResources.BROOKLYN_TEST_MORE_ENTITIES_V1_PATH);
-        ManagementContext mgmt = LocalManagementContextForTests.builder(true).disableOsgi(false).build();
-        try {
-            RegisteredType ci = OsgiVersionMoreEntityTest.addMoreEntityV1(mgmt, "1.0");
+        mgmt = LocalManagementContextForTests.builder(true).disableOsgi(false).build();
+        
+        RegisteredType ci = OsgiVersionMoreEntityTest.addMoreEntityV1(mgmt, "1.0");
             
-            EntitySpec<DynamicCluster> spec = EntitySpec.create(DynamicCluster.class)
-                .configure(DynamicCluster.INITIAL_SIZE, 1)
-                .configure(DynamicCluster.MEMBER_SPEC, mgmt.getTypeRegistry().createSpec(ci, null, EntitySpec.class));
+        EntitySpec<DynamicCluster> spec = EntitySpec.create(DynamicCluster.class)
+            .configure(DynamicCluster.INITIAL_SIZE, 1)
+            .configure(DynamicCluster.MEMBER_SPEC, mgmt.getTypeRegistry().createSpec(ci, null, EntitySpec.class));
+
+        serializer.setLookupContext(new LookupContextImpl(mgmt,
+            ImmutableList.<Entity>of(), ImmutableList.<Location>of(), ImmutableList.<Policy>of(),
+            ImmutableList.<Enricher>of(), ImmutableList.<Feed>of(), ImmutableList.<CatalogItem<?,?>>of(), true));
+        assertSerializeAndDeserialize(spec);
+    }
     
-            serializer.setLookupContext(new LookupContextImpl(mgmt,
+    @Test
+    public void testOsgiBundleNameNotIncludedForWhiteListed() throws Exception {
+        mgmt = LocalManagementContextForTests.builder(true).disableOsgi(false).build();
+
+        serializer.setLookupContext(new LookupContextImpl(mgmt,
+            ImmutableList.<Entity>of(), ImmutableList.<Location>of(), ImmutableList.<Policy>of(),
+            ImmutableList.<Enricher>of(), ImmutableList.<Feed>of(), ImmutableList.<CatalogItem<?,?>>of(), true));
+        
+        Object obj = PersistMode.AUTO;
+        
+        assertSerializeAndDeserialize(obj);
+
+        // i.e. not pre-pended with "org.apache.brooklyn.core:"
+        String expectedForm = "<"+PersistMode.class.getName()+">AUTO</"+PersistMode.class.getName()+">";
+        String serializedForm = serializer.toString(obj);
+        assertEquals(serializedForm.trim(), expectedForm.trim());
+    }
+
+    @Test
+    public void testOsgiBundleNamePrefixIncluded() throws Exception {
+        String bundlePath = OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH;
+        String bundleUrl = OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL;
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), bundlePath);
+        
+        mgmt = LocalManagementContextForTests.builder(true).disableOsgi(false).build();
+
+        serializer.setLookupContext(new LookupContextImpl(mgmt,
                 ImmutableList.<Entity>of(), ImmutableList.<Location>of(), ImmutableList.<Policy>of(),
                 ImmutableList.<Enricher>of(), ImmutableList.<Feed>of(), ImmutableList.<CatalogItem<?,?>>of(), true));
-            assertSerializeAndDeserialize(spec);
-        } finally {
-            Entities.destroyAllCatching(mgmt);
-        }
+        
+        Bundle bundle = installBundle(mgmt, bundleUrl);
+        
+        String classname = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_OBJECT;
+        Class<?> osgiObjectClazz = bundle.loadClass(classname);
+        Object obj = Reflections.invokeConstructorWithArgs(osgiObjectClazz, "myval").get();
+
+        assertSerializeAndDeserialize(obj);
+
+        // i.e. prepended with bundle name
+        String expectedForm = Joiner.on("\n").join(
+                "<"+bundle.getSymbolicName()+":"+classname+">",
+                "  <val>myval</val>",
+                "</"+bundle.getSymbolicName()+":"+classname+">");
+        String serializedForm = serializer.toString(obj);
+        assertEquals(serializedForm.trim(), expectedForm.trim());
     }
+    
+    // TODO This doesn't get the bundleName - should we expect it to? Is this because of 
+    // how we're using Felix? Would it also be true in Karaf?
+    @Test(groups="Broken")
+    public void testOsgiBundleNamePrefixIncludedForDownstreamDependency() throws Exception {
+        mgmt = LocalManagementContextForTests.builder(true).disableOsgi(false).build();
 
+        serializer.setLookupContext(new LookupContextImpl(mgmt,
+                ImmutableList.<Entity>of(), ImmutableList.<Location>of(), ImmutableList.<Policy>of(),
+                ImmutableList.<Enricher>of(), ImmutableList.<Feed>of(), ImmutableList.<CatalogItem<?,?>>of(), true));
+        
+        // Using a guava type (which is a downstream dependency of Brooklyn)
+        String bundleName = "com.goole.guava";
+        String classname = "com.google.common.base.Predicates_-ObjectPredicate";
+        Object obj = Predicates.alwaysTrue();
+
+        assertSerializeAndDeserialize(obj);
+
+        // i.e. prepended with bundle name
+        String expectedForm = "<"+bundleName+":"+classname+">ALWAYS_TRUE</"+bundleName+":"+classname+">";
+        String serializedForm = serializer.toString(obj);
+        assertEquals(serializedForm.trim(), expectedForm.trim());
+    }
+    
     @Test
     public void testImmutableCollectionsWithDanglingEntityRef() throws Exception {
         // If there's a dangling entity in an ImmutableList etc, then discard it entirely.
@@ -647,4 +732,10 @@ public class XmlMementoSerializerTest {
             return Objects.hashCode(myStaticInnerField);
         }
     }
+    
+    private Bundle installBundle(ManagementContext mgmt, String bundleUrl) throws Exception {
+        OsgiManager osgiManager = ((ManagementContextInternal)mgmt).getOsgiManager().get();
+        Framework framework = osgiManager.getFramework();
+        return Osgis.install(framework, bundleUrl);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/core/src/test/java/org/apache/brooklyn/util/core/ClassLoaderUtilsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/brooklyn/util/core/ClassLoaderUtilsTest.java b/core/src/test/java/org/apache/brooklyn/util/core/ClassLoaderUtilsTest.java
index bfa5adc..3f2a8e1 100644
--- a/core/src/test/java/org/apache/brooklyn/util/core/ClassLoaderUtilsTest.java
+++ b/core/src/test/java/org/apache/brooklyn/util/core/ClassLoaderUtilsTest.java
@@ -19,6 +19,8 @@
 package org.apache.brooklyn.util.core;
 
 import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
 
 import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.mgmt.ManagementContext;
@@ -132,6 +134,16 @@ public class ClassLoaderUtilsTest {
         assertEquals(clu.loadClass(bundle.getSymbolicName() + ":" + bundle.getVersion() + ":" + classname), clazz);
     }
     
+    @Test
+    public void testIsBundleWhiteListed() throws Exception {
+        mgmt = LocalManagementContextForTests.builder(true).disableOsgi(false).build();
+        ClassLoaderUtils clu = new ClassLoaderUtils(getClass(), mgmt);
+        
+        assertTrue(clu.isBundleWhiteListed(getBundle(mgmt, "org.apache.brooklyn.core")));
+        assertTrue(clu.isBundleWhiteListed(getBundle(mgmt, "org.apache.brooklyn.api")));
+        assertFalse(clu.isBundleWhiteListed(getBundle(mgmt, "com.google.guava")));
+    }
+    
     private Bundle installBundle(ManagementContext mgmt, String bundleUrl) throws Exception {
         OsgiManager osgiManager = ((ManagementContextInternal)mgmt).getOsgiManager().get();
         Framework framework = osgiManager.getFramework();

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplication.java
----------------------------------------------------------------------
diff --git a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplication.java b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplication.java
index dcfb495..f561b97 100644
--- a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplication.java
+++ b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplication.java
@@ -18,7 +18,6 @@
  */
 package org.apache.brooklyn.test.osgi.entities;
 
-
 import org.apache.brooklyn.api.entity.ImplementedBy;
 import org.apache.brooklyn.core.entity.StartableApplication;
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplicationImpl.java
----------------------------------------------------------------------
diff --git a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplicationImpl.java b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplicationImpl.java
index fe6f1a2..ae4c69d 100644
--- a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplicationImpl.java
+++ b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleApplicationImpl.java
@@ -18,7 +18,6 @@
  */
 package org.apache.brooklyn.test.osgi.entities;
 
-
 import org.apache.brooklyn.core.entity.AbstractApplication;
 import org.apache.brooklyn.core.entity.StartableApplication;
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntity.java
----------------------------------------------------------------------
diff --git a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntity.java b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntity.java
index ffed15f..f256af7 100644
--- a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntity.java
+++ b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntity.java
@@ -19,13 +19,20 @@
 package org.apache.brooklyn.test.osgi.entities;
 
 
+import org.apache.brooklyn.api.catalog.Catalog;
 import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.entity.ImplementedBy;
-import org.apache.brooklyn.api.catalog.Catalog;
+import org.apache.brooklyn.api.sensor.AttributeSensor;
+import org.apache.brooklyn.config.ConfigKey;
+import org.apache.brooklyn.core.config.ConfigKeys;
+import org.apache.brooklyn.core.sensor.Sensors;
 
 
 @Catalog(name="A Simple Entity", description="Simple entity for test purposes")
 @ImplementedBy(SimpleEntityImpl.class)
 public interface SimpleEntity extends Entity {
+    
+    ConfigKey<Object> SIMPLE_CONFIG = ConfigKeys.newConfigKey(Object.class, "simple.config");
 
+    AttributeSensor<Object> SIMPLE_SENSOR = Sensors.newSensor(Object.class, "simple.sensor");
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntityImpl.java
----------------------------------------------------------------------
diff --git a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntityImpl.java b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntityImpl.java
index 2595c39..16ecd68 100644
--- a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntityImpl.java
+++ b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleEntityImpl.java
@@ -20,7 +20,6 @@ package org.apache.brooklyn.test.osgi.entities;
 
 import org.apache.brooklyn.core.entity.AbstractEntity;
 
-
 public class SimpleEntityImpl extends AbstractEntity implements SimpleEntity {
 
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleObject.java
----------------------------------------------------------------------
diff --git a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleObject.java b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleObject.java
new file mode 100644
index 0000000..dad1ab2
--- /dev/null
+++ b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimpleObject.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.brooklyn.test.osgi.entities;
+
+public class SimpleObject {
+    
+    private String val;
+    
+    public SimpleObject() {
+    }
+
+    public SimpleObject(String val) {
+        this.val = val;
+    }
+
+    public String getVal() {
+        return val;
+    }
+    
+    public void setVal(String val) {
+        this.val = val;
+    }
+    
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof SimpleObject)) return false;
+        String oVal = ((SimpleObject)obj).val;
+        return (val == null) ? oVal == null : val.equals(oVal);
+    }
+    
+    @Override
+    public int hashCode() {
+        return (val == null) ? 0 : val.hashCode();
+    }
+    
+    @Override
+    public String toString() {
+        return "SimpleObject["+val+"]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimplePolicy.java
----------------------------------------------------------------------
diff --git a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimplePolicy.java b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimplePolicy.java
index fdea821..f6589c5 100644
--- a/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimplePolicy.java
+++ b/utils/common/dependencies/osgi/entities/src/main/java/org/apache/brooklyn/test/osgi/entities/SimplePolicy.java
@@ -18,7 +18,6 @@
  */
 package org.apache.brooklyn.test.osgi.entities;
 
-
 import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.core.config.ConfigKeys;
 import org.apache.brooklyn.core.policy.AbstractPolicy;
@@ -33,4 +32,9 @@ public class SimplePolicy extends AbstractPolicy {
 
     @SetFromFlag("config3")
     public static final ConfigKey<String> CONFIG3 = ConfigKeys.newStringConfigKey("config3");
+    
+    @Override
+    protected <T> void doReconfigureConfig(ConfigKey<T> key, T val) {
+        // no-op; allow any config to be set
+    }
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/utils/common/src/main/java/org/apache/brooklyn/util/osgi/VersionedName.java
----------------------------------------------------------------------
diff --git a/utils/common/src/main/java/org/apache/brooklyn/util/osgi/VersionedName.java b/utils/common/src/main/java/org/apache/brooklyn/util/osgi/VersionedName.java
index df36a80..2d89be2 100644
--- a/utils/common/src/main/java/org/apache/brooklyn/util/osgi/VersionedName.java
+++ b/utils/common/src/main/java/org/apache/brooklyn/util/osgi/VersionedName.java
@@ -15,6 +15,10 @@
  */
 package org.apache.brooklyn.util.osgi;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import javax.annotation.Nullable;
+
 import com.google.common.base.Objects;
 import org.apache.brooklyn.util.text.Strings;
 import org.osgi.framework.Bundle;
@@ -29,12 +33,11 @@ public class VersionedName {
     private final Version version;
 
     public VersionedName(Bundle b) {
-        this.symbolicName = b.getSymbolicName();
-        this.version = b.getVersion();
+        this(b.getSymbolicName(), b.getVersion());
     }
 
-    public VersionedName(String symbolicName, Version version) {
-        this.symbolicName = symbolicName;
+    public VersionedName(String symbolicName, @Nullable Version version) {
+        this.symbolicName = checkNotNull(symbolicName, "symbolicName");
         this.version = version;
     }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a555beac/utils/common/src/test/java/org/apache/brooklyn/util/osgi/OsgiTestResources.java
----------------------------------------------------------------------
diff --git a/utils/common/src/test/java/org/apache/brooklyn/util/osgi/OsgiTestResources.java b/utils/common/src/test/java/org/apache/brooklyn/util/osgi/OsgiTestResources.java
index 21827a0..181849d 100644
--- a/utils/common/src/test/java/org/apache/brooklyn/util/osgi/OsgiTestResources.java
+++ b/utils/common/src/test/java/org/apache/brooklyn/util/osgi/OsgiTestResources.java
@@ -44,6 +44,9 @@ public class OsgiTestResources {
     public static final String BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_APPLICATION = "org.apache.brooklyn.test.osgi.entities.SimpleApplication";
     public static final String BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY = "org.apache.brooklyn.test.osgi.entities.SimpleEntity";
     public static final String BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_POLICY = "org.apache.brooklyn.test.osgi.entities.SimplePolicy";
+    public static final String BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_OBJECT = "org.apache.brooklyn.test.osgi.entities.SimpleObject";
+    public static final String BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY_CONFIG_NAME = "simple.config";
+    public static final String BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY_SENSOR_NAME = "simple.sensor";
 
     /**
      * brooklyn-test-osgi-more-entities_0.1.0 -