You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by ge...@apache.org on 2017/07/19 16:25:47 UTC

[17/39] brooklyn-server git commit: roughly working to validate types in a separate pass, fixing the big problem

roughly working to validate types in a separate pass, fixing the big problem

quite a few holes to fill, but confirmed working in a few places.
also makes sure not to detect and delete empty wrapper bundles until we've finished adding them.


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

Branch: refs/heads/master
Commit: bf31100c2969a848a45d24452baee78d15a7e37a
Parents: d2e6a53
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Fri Jun 30 01:14:32 2017 +0100
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Fri Jun 30 14:08:25 2017 +0100

----------------------------------------------------------------------
 .../mementos/BrooklynMementoPersister.java      |  1 +
 .../CatalogYamlEntityOsgiTypeRegistryTest.java  |  7 ++-
 .../brooklyn/catalog/CatalogYamlRebindTest.java | 54 ++++++++---------
 .../brooklyn/test/lite/CampYamlLiteTest.java    |  2 +-
 .../catalog/internal/BasicBrooklynCatalog.java  | 20 +++---
 .../catalog/internal/CatalogBundleLoader.java   | 62 ++++++++++++++++---
 .../catalog/internal/CatalogBundleTracker.java  | 25 +++-----
 .../core/catalog/internal/CatalogUtils.java     |  2 +-
 .../core/mgmt/ha/OsgiArchiveInstaller.java      | 20 +++---
 .../mgmt/ha/OsgiBundleInstallationResult.java   |  7 ++-
 .../brooklyn/core/mgmt/ha/OsgiManager.java      | 49 ++++++++++++---
 .../core/mgmt/rebind/RebindContextImpl.java     |  7 ++-
 .../mgmt/rebind/RebindContextLookupContext.java |  7 ++-
 .../core/mgmt/rebind/RebindIteration.java       | 64 +++++++++++++++++---
 .../typereg/RegisteredTypeLoadingContexts.java  |  3 +-
 15 files changed, 230 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoPersister.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoPersister.java b/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoPersister.java
index e352200..4bd013e 100644
--- a/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoPersister.java
+++ b/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoPersister.java
@@ -38,6 +38,7 @@ import org.apache.brooklyn.api.policy.Policy;
 import org.apache.brooklyn.api.sensor.Enricher;
 import org.apache.brooklyn.api.sensor.Feed;
 import org.apache.brooklyn.api.typereg.ManagedBundle;
+import org.apache.brooklyn.api.typereg.RegisteredType;
 import org.apache.brooklyn.util.time.Duration;
 
 import com.google.common.annotations.Beta;

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityOsgiTypeRegistryTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityOsgiTypeRegistryTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityOsgiTypeRegistryTest.java
index 78df657..cde5159 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityOsgiTypeRegistryTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityOsgiTypeRegistryTest.java
@@ -58,9 +58,10 @@ public class CatalogYamlEntityOsgiTypeRegistryTest extends CatalogYamlEntityTest
                 new ZipEntry(BasicBrooklynCatalog.CATALOG_BOM), new ByteArrayInputStream(catalogYaml.getBytes())));
             ReferenceWithError<OsgiBundleInstallationResult> b = ((ManagementContextInternal)mgmt()).getOsgiManager().get().installDeferredStart(
                 new BasicManagedBundle(bundleName(), bundleVersion(), null), 
-                new FileInputStream(bf));
-            // bundle not started (no need), and BOM not installed above; do it explicitly below
-            // testing the type registry approach instead
+                new FileInputStream(bf),
+                false);
+            // bundle not started (no need), and BOM not installed nor validated above; 
+            // do BOM install and validation below manually to test the type registry approach
             mgmt().getCatalog().addTypesFromBundleBom(catalogYaml, b.get().getMetadata(), isForceUpdate());
             Map<RegisteredType, Collection<Throwable>> validation = mgmt().getCatalog().validateTypes( mgmt().getTypeRegistry().getMatching(RegisteredTypePredicates.containingBundle(b.get().getVersionedName())) );
             if (!validation.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlRebindTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlRebindTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlRebindTest.java
index d92c2b8..a962dbe 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlRebindTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlRebindTest.java
@@ -118,33 +118,33 @@ public class CatalogYamlRebindTest extends AbstractYamlRebindTest {
     public Object[][] dataProvider() {
         return new Object[][] {
             {RebindWithCatalogTestMode.NO_OP, OsgiMode.NONE},
-//            {RebindWithCatalogTestMode.NO_OP, OsgiMode.LIBRARY},
-//            {RebindWithCatalogTestMode.NO_OP, OsgiMode.PREFIX},
-//            
-//            {RebindWithCatalogTestMode.STRIP_DEPRECATION_AND_ENABLEMENT_FROM_CATALOG_ITEM, OsgiMode.NONE},
-//            {RebindWithCatalogTestMode.STRIP_DEPRECATION_AND_ENABLEMENT_FROM_CATALOG_ITEM, OsgiMode.LIBRARY},
-//            {RebindWithCatalogTestMode.STRIP_DEPRECATION_AND_ENABLEMENT_FROM_CATALOG_ITEM, OsgiMode.PREFIX},
-//            
-//            {RebindWithCatalogTestMode.DEPRECATE_CATALOG, OsgiMode.NONE},
-//            {RebindWithCatalogTestMode.DEPRECATE_CATALOG, OsgiMode.LIBRARY},
-//            {RebindWithCatalogTestMode.DEPRECATE_CATALOG, OsgiMode.PREFIX},
-//            
-//            {RebindWithCatalogTestMode.DISABLE_CATALOG, OsgiMode.NONE},
-//            {RebindWithCatalogTestMode.DISABLE_CATALOG, OsgiMode.LIBRARY},
-//            {RebindWithCatalogTestMode.DISABLE_CATALOG, OsgiMode.PREFIX},
-//            
-//            // For DELETE_CATALOG, see https://issues.apache.org/jira/browse/BROOKLYN-149.
-//            // Deletes the catalog item before rebind, but the referenced types are still on the 
-//            // default classpath. Will fallback to loading from classpath.
-//            //
-//            // Does not work for OSGi, because our bundle will no longer be available.
-//            {RebindWithCatalogTestMode.DELETE_CATALOG, OsgiMode.NONE},
-//            
-//            // Upgrades the catalog item before rebind, deleting the old version.
-//            // Will automatically upgrade. Test will enable "FEATURE_AUTO_FIX_CATALOG_REF_ON_REBIND"
-//            {RebindWithCatalogTestMode.REPLACE_CATALOG_WITH_NEWER_VERSION, OsgiMode.NONE},
-//            {RebindWithCatalogTestMode.REPLACE_CATALOG_WITH_NEWER_VERSION, OsgiMode.LIBRARY},
-//            {RebindWithCatalogTestMode.REPLACE_CATALOG_WITH_NEWER_VERSION, OsgiMode.PREFIX},
+            {RebindWithCatalogTestMode.NO_OP, OsgiMode.LIBRARY},
+            {RebindWithCatalogTestMode.NO_OP, OsgiMode.PREFIX},
+            
+            {RebindWithCatalogTestMode.STRIP_DEPRECATION_AND_ENABLEMENT_FROM_CATALOG_ITEM, OsgiMode.NONE},
+            {RebindWithCatalogTestMode.STRIP_DEPRECATION_AND_ENABLEMENT_FROM_CATALOG_ITEM, OsgiMode.LIBRARY},
+            {RebindWithCatalogTestMode.STRIP_DEPRECATION_AND_ENABLEMENT_FROM_CATALOG_ITEM, OsgiMode.PREFIX},
+            
+            {RebindWithCatalogTestMode.DEPRECATE_CATALOG, OsgiMode.NONE},
+            {RebindWithCatalogTestMode.DEPRECATE_CATALOG, OsgiMode.LIBRARY},
+            {RebindWithCatalogTestMode.DEPRECATE_CATALOG, OsgiMode.PREFIX},
+            
+            {RebindWithCatalogTestMode.DISABLE_CATALOG, OsgiMode.NONE},
+            {RebindWithCatalogTestMode.DISABLE_CATALOG, OsgiMode.LIBRARY},
+            {RebindWithCatalogTestMode.DISABLE_CATALOG, OsgiMode.PREFIX},
+            
+            // For DELETE_CATALOG, see https://issues.apache.org/jira/browse/BROOKLYN-149.
+            // Deletes the catalog item before rebind, but the referenced types are still on the 
+            // default classpath. Will fallback to loading from classpath.
+            //
+            // Does not work for OSGi, because our bundle will no longer be available.
+            {RebindWithCatalogTestMode.DELETE_CATALOG, OsgiMode.NONE},
+            
+            // Upgrades the catalog item before rebind, deleting the old version.
+            // Will automatically upgrade. Test will enable "FEATURE_AUTO_FIX_CATALOG_REF_ON_REBIND"
+            {RebindWithCatalogTestMode.REPLACE_CATALOG_WITH_NEWER_VERSION, OsgiMode.NONE},
+            {RebindWithCatalogTestMode.REPLACE_CATALOG_WITH_NEWER_VERSION, OsgiMode.LIBRARY},
+            {RebindWithCatalogTestMode.REPLACE_CATALOG_WITH_NEWER_VERSION, OsgiMode.PREFIX},
         };
     }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/CampYamlLiteTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/CampYamlLiteTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/CampYamlLiteTest.java
index 7e6f176..07a8098 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/CampYamlLiteTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/CampYamlLiteTest.java
@@ -214,7 +214,7 @@ public class CampYamlLiteTest {
         // install bundle for class access but without loading its catalog.bom, 
         // since we only have mock matchers here
         // (if we don't do this, the default routines install it and try to process the catalog.bom, failing)
-        ((ManagementContextInternal)mgmt).getOsgiManager().get().installDeferredStart(new BasicManagedBundle(null, null, bundleUrl), null).get();
+        ((ManagementContextInternal)mgmt).getOsgiManager().get().installDeferredStart(new BasicManagedBundle(null, null, bundleUrl), null, false).get();
     }
 
     private void assertMgmtHasSampleMyCatalogApp(String symbolicName, String bundleUrl) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/core/src/main/java/org/apache/brooklyn/core/catalog/internal/BasicBrooklynCatalog.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/catalog/internal/BasicBrooklynCatalog.java b/core/src/main/java/org/apache/brooklyn/core/catalog/internal/BasicBrooklynCatalog.java
index 4fc78a0..323c70f 100644
--- a/core/src/main/java/org/apache/brooklyn/core/catalog/internal/BasicBrooklynCatalog.java
+++ b/core/src/main/java/org/apache/brooklyn/core/catalog/internal/BasicBrooklynCatalog.java
@@ -140,7 +140,7 @@ public class BasicBrooklynCatalog implements BrooklynCatalog {
             loader.set(val);
         }
         
-        // TODO Stack, for recursive calls?
+        // If needed, could use stack; see ClassLoaderFromStack...
         public static void unsetLoader(BrooklynClassLoadingContext val) {
             loader.set(null);
         }
@@ -931,7 +931,8 @@ public class BasicBrooklynCatalog implements BrooklynCatalog {
         return !containingBundle.getVersionedName().equalsOsgi(li.getVersionedName());
     }
 
-    private static boolean isNoBundleOrSimpleWrappingBundle(ManagementContext mgmt, ManagedBundle b) {
+    @Beta
+    public static boolean isNoBundleOrSimpleWrappingBundle(ManagementContext mgmt, ManagedBundle b) {
         if (b==null) return true;
         Maybe<OsgiManager> osgi = ((ManagementContextInternal)mgmt).getOsgiManager();
         if (osgi.isAbsent()) {
@@ -988,7 +989,7 @@ public class BasicBrooklynCatalog implements BrooklynCatalog {
         CatalogDto dto = CatalogDto.newNamedInstance("Bundles Scanned Catalog", "All annotated Brooklyn entities detected in bundles", "scanning-bundles-classpath-"+libraries.hashCode());
         List<String> urls = MutableList.of();
         for (OsgiBundleWithUrl b: libraries) {
-            // TODO currently does not support pre-installed bundles identified by name:version 
+            // does not support pre-installed bundles identified by name:version 
             // (ie where URL not supplied)
             if (Strings.isNonBlank(b.getUrl())) {
                 urls.add(b.getUrl());
@@ -1358,13 +1359,9 @@ public class BasicBrooklynCatalog implements BrooklynCatalog {
                 throw new IllegalStateException(result.getMessage());
             }
             return toItems(result.getCatalogItemsInstalled());
-            
-            // TODO check if we've overridden all items pertaining to an older anonymous catalog.bom bundle
-            // we could remove references to that anonymous bundle; 
-            // without this currently we leak bundles as bom's are replaced
-            // (because we persist each item as well as the bundle, and we use the item XML on rebind, 
-            // rather than rereading the catalog.bom from the bundle, there isn't currently a risk of loading
-            // any of those overwritten items; however probably wise in future to require a bundle ID)
+
+            // if all items pertaining to an older anonymous catalog.bom bundle have been overridden
+            // we delete those later; see list of wrapper bundles kept in OsgiManager
         }
         // fallback to non-OSGi for tests and other environments
         return addItems(yaml, null, forceUpdate);
@@ -1374,7 +1371,6 @@ public class BasicBrooklynCatalog implements BrooklynCatalog {
     private List<CatalogItem<?,?>> toItems(Iterable<String> itemIds) {
         List<CatalogItem<?,?>> result = MutableList.of();
         for (String id: itemIds) {
-            // TODO prefer to use RegisteredType, but that's an API change here
             result.add(CatalogUtils.getCatalogItemOptionalVersion(mgmt, id));
         }
         return result;
@@ -1812,7 +1808,7 @@ public class BasicBrooklynCatalog implements BrooklynCatalog {
         synchronized (uninstallingEmptyLock) {
             Maybe<OsgiManager> osgi = ((ManagementContextInternal)mgmt).getOsgiManager();
             if (osgi.isAbsent()) return;
-            for (ManagedBundle b: osgi.get().getManagedBundles().values()) {
+            for (ManagedBundle b: osgi.get().getInstalledWrapperBundles()) {
                 if (isNoBundleOrSimpleWrappingBundle(mgmt, b)) {
                     Iterable<RegisteredType> typesInBundle = osgi.get().getTypesFromBundle(b.getVersionedName());
                     if (Iterables.isEmpty(typesInBundle)) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleLoader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleLoader.java b/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleLoader.java
index f42dac1..bd9c7fa 100644
--- a/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleLoader.java
+++ b/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleLoader.java
@@ -24,12 +24,16 @@ import static org.apache.brooklyn.api.catalog.CatalogItem.CatalogItemType.TEMPLA
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URL;
+import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.brooklyn.api.catalog.CatalogItem;
 import org.apache.brooklyn.api.mgmt.ManagementContext;
 import org.apache.brooklyn.api.typereg.ManagedBundle;
+import org.apache.brooklyn.api.typereg.RegisteredType;
 import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
+import org.apache.brooklyn.core.typereg.RegisteredTypePredicates;
 import org.apache.brooklyn.util.collections.MutableList;
 import org.apache.brooklyn.util.exceptions.Exceptions;
 import org.apache.brooklyn.util.osgi.VersionedName;
@@ -41,6 +45,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.Beta;
 import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
 
 @Beta
 public class CatalogBundleLoader {
@@ -56,6 +61,10 @@ public class CatalogBundleLoader {
         this.managementContext = managementContext;
     }
 
+    public void scanForCatalog(Bundle bundle, boolean force, boolean validate) {
+        scanForCatalogInternal(bundle, force, validate, false);
+    }
+    
     /**
      * Scan the given bundle for a catalog.bom and adds it to the catalog.
      *
@@ -63,11 +72,15 @@ public class CatalogBundleLoader {
      * @return A list of items added to the catalog
      * @throws RuntimeException if the catalog items failed to be added to the catalog
      */
-    public Iterable<? extends CatalogItem<?, ?>> scanForCatalog(Bundle bundle) {
-        return scanForCatalog(bundle, false);
+    public Iterable<? extends CatalogItem<?, ?>> scanForCatalogLegacy(Bundle bundle) {
+        return scanForCatalogLegacy(bundle, false);
+    }
+    
+    public Iterable<? extends CatalogItem<?, ?>> scanForCatalogLegacy(Bundle bundle, boolean force) {
+        return scanForCatalogInternal(bundle, force, true, true);
     }
     
-    public Iterable<? extends CatalogItem<?, ?>> scanForCatalog(Bundle bundle, boolean force) {
+    private Iterable<? extends CatalogItem<?, ?>> scanForCatalogInternal(Bundle bundle, boolean force, boolean validate, boolean legacy) {
         ManagedBundle mb = ((ManagementContextInternal)managementContext).getOsgiManager().get().getManagedBundle(
             new VersionedName(bundle));
 
@@ -77,23 +90,50 @@ public class CatalogBundleLoader {
         if (null != bom) {
             LOG.debug("Found catalog BOM in {} {} {}", CatalogUtils.bundleIds(bundle));
             String bomText = readBom(bom);
-            // TODO use addTypesFromBundleBom; but when should we do validation? after all bundles are loaded?
-            // OR maybe deprecate/remove this experiment in favour of explicitly installed and managed bundles?
-            catalogItems = this.managementContext.getCatalog().addItems(bomText, mb, force);
-            for (CatalogItem<?, ?> item : catalogItems) {
-                LOG.debug("Added to catalog: {}, {}", item.getSymbolicName(), item.getVersion());
+            if (legacy) {
+                catalogItems = this.managementContext.getCatalog().addItems(bomText, mb, force);
+                for (CatalogItem<?, ?> item : catalogItems) {
+                    LOG.debug("Added to catalog: {}, {}", item.getSymbolicName(), item.getVersion());
+                }
+            } else {
+                this.managementContext.getCatalog().addTypesFromBundleBom(bomText, mb, force);
+                if (validate) {
+                    Map<RegisteredType, Collection<Throwable>> validationErrors = this.managementContext.getCatalog().validateTypes(
+                        this.managementContext.getTypeRegistry().getMatching(RegisteredTypePredicates.containingBundle(mb.getVersionedName())) );
+                    if (!validationErrors.isEmpty()) {
+                        throw Exceptions.propagate("Failed to install "+mb.getVersionedName()+", types "+validationErrors.keySet()+" gave errors",
+                            Iterables.concat(validationErrors.values()));
+                    }
+                }
+            }
+            
+            if (BasicBrooklynCatalog.isNoBundleOrSimpleWrappingBundle(managementContext, mb)) {
+                ((ManagementContextInternal)managementContext).getOsgiManager().get().addInstalledWrapperBundle(mb);
             }
         } else {
             LOG.debug("No BOM found in {} {} {}", CatalogUtils.bundleIds(bundle));
         }
 
         if (!applicationsPermitted.apply(bundle)) {
-            catalogItems = removeApplications(catalogItems);
+            if (legacy) {
+                catalogItems = removeApplications(catalogItems);
+            } else {
+                removeApplications(mb);
+            }
         }
 
         return catalogItems;
     }
 
+    private void removeApplications(ManagedBundle mb) {
+        for (RegisteredType t: managementContext.getTypeRegistry().getMatching(RegisteredTypePredicates.containingBundle(mb.getVersionedName()))) {
+            // TODO support templates, and remove them here
+//            if (t.getKind() == RegisteredTypeKind.TEMPLATE) {
+//                ((BasicBrooklynTypeRegistry) managementContext.getTypeRegistry()).delete(t);
+//            }
+        }
+    }
+
     /**
      * Remove the given items from the catalog.
      *
@@ -109,6 +149,10 @@ public class CatalogBundleLoader {
             }, " "), e);
         }
     }
+    
+    public void removeFromCatalog(VersionedName n) {
+        ((ManagementContextInternal)managementContext).getOsgiManager().get().uninstallCatalogItemsFromBundle(n);
+    }
 
     private String readBom(URL bom) {
         try (final InputStream ins = bom.openStream()) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleTracker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleTracker.java b/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleTracker.java
index 074d7f7..1ec4cd2 100644
--- a/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleTracker.java
+++ b/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogBundleTracker.java
@@ -19,7 +19,7 @@
 
 package org.apache.brooklyn.core.catalog.internal;
 
-import org.apache.brooklyn.api.catalog.CatalogItem;
+import org.apache.brooklyn.util.osgi.VersionedName;
 import org.osgi.framework.Bundle;
 import org.osgi.framework.BundleContext;
 import org.osgi.framework.BundleEvent;
@@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.Beta;
 
 @Beta
-public class CatalogBundleTracker extends BundleTracker<Iterable<? extends CatalogItem<?, ?>>> {
+public class CatalogBundleTracker extends BundleTracker<Object> {
 
     private static final Logger LOG = LoggerFactory.getLogger(CatalogBundleTracker.class);
 
@@ -46,13 +46,13 @@ public class CatalogBundleTracker extends BundleTracker<Iterable<? extends Catal
      *
      * @param bundle      The bundle being added to the bundle context.
      * @param bundleEvent The event of the addition.
-     * @return The items added to the catalog; these will be tracked by the {@link BundleTracker} mechanism
-     * and supplied to the {@link #removedBundle(Bundle, BundleEvent, Iterable)} method.
+     * @return null
      * @throws RuntimeException if the catalog items failed to be added to the catalog
      */
     @Override
-    public Iterable<? extends CatalogItem<?, ?>> addingBundle(Bundle bundle, BundleEvent bundleEvent) {
-        return catalogBundleLoader.scanForCatalog(bundle);
+    public Object addingBundle(Bundle bundle, BundleEvent bundleEvent) {
+        catalogBundleLoader.scanForCatalog(bundle, false, true);
+        return null;
     }
 
     /**
@@ -60,19 +60,12 @@ public class CatalogBundleTracker extends BundleTracker<Iterable<? extends Catal
      *
      * @param bundle      The bundle being removed to the bundle context.
      * @param bundleEvent The event of the removal.
-     * @param items       The items being removed
+     * @param callback    Ignored
      * @throws RuntimeException if the catalog items failed to be added to the catalog
      */
     @Override
-    public void removedBundle(Bundle bundle, BundleEvent bundleEvent, Iterable<? extends CatalogItem<?, ?>> items) {
-        if (!items.iterator().hasNext()) {
-            return;
-        }
+    public void removedBundle(Bundle bundle, BundleEvent bundleEvent, Object callback) {
         LOG.debug("Unloading catalog BOM entries from {} {} {}", CatalogUtils.bundleIds(bundle));
-        for (CatalogItem<?, ?> item : items) {
-            LOG.debug("Unloading {} {} from catalog", item.getSymbolicName(), item.getVersion());
-
-            catalogBundleLoader.removeFromCatalog(item);
-        }
+        catalogBundleLoader.removeFromCatalog(new VersionedName(bundle));
     }
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogUtils.java b/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogUtils.java
index bc744a0..473fa6b 100644
--- a/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogUtils.java
+++ b/core/src/main/java/org/apache/brooklyn/core/catalog/internal/CatalogUtils.java
@@ -175,7 +175,7 @@ public class CatalogUtils {
             Stopwatch timer = Stopwatch.createStarted();
             List<OsgiBundleInstallationResult> results = MutableList.of();
             for (CatalogBundle bundleUrl : libraries) {
-                OsgiBundleInstallationResult result = osgi.get().installDeferredStart(BasicManagedBundle.of(bundleUrl), null).get();
+                OsgiBundleInstallationResult result = osgi.get().installDeferredStart(BasicManagedBundle.of(bundleUrl), null, true).get();
                 if (log.isDebugEnabled()) {
                     logDebugOrTraceIfRebinding(log, "Installation of library "+bundleUrl+": "+result);
                 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiArchiveInstaller.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiArchiveInstaller.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiArchiveInstaller.java
index 1497b42..a634230 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiArchiveInstaller.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiArchiveInstaller.java
@@ -23,18 +23,18 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.List;
 import java.util.jar.Attributes;
 import java.util.jar.Manifest;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 
-import org.apache.brooklyn.api.catalog.CatalogItem;
 import org.apache.brooklyn.api.typereg.ManagedBundle;
+import org.apache.brooklyn.api.typereg.RegisteredType;
 import org.apache.brooklyn.core.catalog.internal.BasicBrooklynCatalog;
 import org.apache.brooklyn.core.mgmt.ha.OsgiBundleInstallationResult.ResultCode;
 import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
 import org.apache.brooklyn.core.typereg.BasicManagedBundle;
+import org.apache.brooklyn.core.typereg.RegisteredTypePredicates;
 import org.apache.brooklyn.util.collections.MutableList;
 import org.apache.brooklyn.util.core.ResourceUtils;
 import org.apache.brooklyn.util.core.osgi.BundleMaker;
@@ -70,6 +70,7 @@ class OsgiArchiveInstaller {
     private boolean loadCatalogBom = true;
     private boolean force = false;
     private boolean deferredStart = false;
+    private boolean validateTypes = true;
     
     private File zipFile;
     private Manifest discoveredManifest;
@@ -100,7 +101,11 @@ class OsgiArchiveInstaller {
 
     public void setDeferredStart(boolean deferredStart) {
         this.deferredStart = deferredStart;
-    }    
+    }
+    
+    public void setValidateTypes(boolean validateTypes) {
+        this.validateTypes = validateTypes;
+    }
 
     private ManagementContextInternal mgmt() {
         return (ManagementContextInternal) osgiManager.mgmt;
@@ -318,7 +323,7 @@ class OsgiArchiveInstaller {
                     // if it's non-brooklyn installed then fail
                     // (e.g. someone trying to install brooklyn or guice through this mechanism!)
                     result.bundle = b.get();
-                    result.code = OsgiBundleInstallationResult.ResultCode.ERROR_INSTALLING_BUNDLE;
+                    result.code = OsgiBundleInstallationResult.ResultCode.ERROR_LAUNCHING_BUNDLE;
                     throw new IllegalStateException("Bundle "+result.getMetadata().getVersionedName()+" already installed in framework but not managed by Brooklyn; cannot install or update through Brooklyn");
                 }
                 // normal install
@@ -379,9 +384,10 @@ class OsgiArchiveInstaller {
                             osgiManager.uninstallCatalogItemsFromBundle( result.getVersionedName() );
                             // (ideally removal and addition would be atomic)
                         }
-                        List<? extends CatalogItem<?, ?>> items = osgiManager.loadCatalogBom(result.bundle, force);
+                        osgiManager.loadCatalogBom(result.bundle, force, validateTypes);
+                        Iterable<RegisteredType> items = mgmt().getTypeRegistry().getMatching(RegisteredTypePredicates.containingBundle(result.getMetadata()));
                         log.debug("Adding items from bundle "+result.getVersionedName()+": "+items);
-                        for (CatalogItem<?,?> ci: items) {
+                        for (RegisteredType ci: items) {
                             result.catalogItemsInstalled.add(ci.getId());
                         }
                     }
@@ -407,7 +413,7 @@ class OsgiArchiveInstaller {
             
         } catch (Exception e) {
             Exceptions.propagateIfFatal(e);
-            result.code = startedInstallation ? OsgiBundleInstallationResult.ResultCode.ERROR_INSTALLING_BUNDLE : OsgiBundleInstallationResult.ResultCode.ERROR_PREPARING_BUNDLE;
+            result.code = startedInstallation ? OsgiBundleInstallationResult.ResultCode.ERROR_LAUNCHING_BUNDLE : OsgiBundleInstallationResult.ResultCode.ERROR_PREPARING_BUNDLE;
             result.message = "Bundle "+inferredMetadata+" failed "+
                 (startedInstallation ? "installation" : "preparation") + ": " + Exceptions.collapseText(e);
             return ReferenceWithError.newInstanceThrowingError(result, new IllegalStateException(result.message, e));

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiBundleInstallationResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiBundleInstallationResult.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiBundleInstallationResult.java
index 947de1c..c611553 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiBundleInstallationResult.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/ha/OsgiBundleInstallationResult.java
@@ -45,9 +45,10 @@ public class OsgiBundleInstallationResult {
         IGNORING_BUNDLE_AREADY_INSTALLED(false),
         /** bundle could not be made insto a state where it could be installed; bundle is not installed, even if forced */
         ERROR_PREPARING_BUNDLE(true),
-        /** bundle could be installed as OSGi but there was an item later, such as catalog items loading;
-         * bundle may be installed */  // TODO behaviour TBC
-        ERROR_INSTALLING_BUNDLE(true);
+        /** bundle successfully installed to OSGi container but there was an error launching it, 
+         * either the OSGi bundle start, catalog items load, or (most commonly) validating the catalog items;
+         * bundle may be installed (currently it is in most/all places, but behaviour TBC) so caller may have to uninstall it */
+        ERROR_LAUNCHING_BUNDLE(true);
         
         final boolean isError;
         ResultCode(boolean isError) { this.isError = isError; }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/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 6fe1068..9b92a8c 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
@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.InputStream;
 import java.net.URL;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Enumeration;
 import java.util.List;
@@ -44,6 +45,7 @@ import org.apache.brooklyn.core.BrooklynFeatureEnablement;
 import org.apache.brooklyn.core.BrooklynVersion;
 import org.apache.brooklyn.core.catalog.internal.CatalogBundleLoader;
 import org.apache.brooklyn.core.config.ConfigKeys;
+import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
 import org.apache.brooklyn.core.server.BrooklynServerConfig;
 import org.apache.brooklyn.core.server.BrooklynServerPaths;
 import org.apache.brooklyn.core.typereg.RegisteredTypePredicates;
@@ -104,6 +106,7 @@ public class OsgiManager {
     private Set<Bundle> bundlesAtStartup;
     private File osgiCacheDir;
     final ManagedBundlesRecord managedBundlesRecord = new ManagedBundlesRecord();
+    final Map<VersionedName,ManagedBundle> wrapperBundles = MutableMap.of();
     
     static class ManagedBundlesRecord {
         private Map<String, ManagedBundle> managedBundlesByUid = MutableMap.of();
@@ -277,9 +280,10 @@ public class OsgiManager {
     }
 
     /** See {@link OsgiArchiveInstaller#install()}, but deferring the start and catalog load */
-    public ReferenceWithError<OsgiBundleInstallationResult> installDeferredStart(@Nullable ManagedBundle knownBundleMetadata, @Nullable InputStream zipIn) {
+    public ReferenceWithError<OsgiBundleInstallationResult> installDeferredStart(@Nullable ManagedBundle knownBundleMetadata, @Nullable InputStream zipIn, boolean validateTypes) {
         OsgiArchiveInstaller installer = new OsgiArchiveInstaller(this, knownBundleMetadata, zipIn);
         installer.setDeferredStart(true);
+        installer.setValidateTypes(validateTypes);
         
         return installer.install();
     }
@@ -318,6 +322,7 @@ public class OsgiManager {
             }
             managedBundlesRecord.managedBundlesUidByVersionedName.remove(bundleMetadata.getVersionedName());
             managedBundlesRecord.managedBundlesUidByUrl.remove(bundleMetadata.getUrl());
+            removeInstalledWrapperBundle(bundleMetadata);
         }
         mgmt.getRebindManager().getChangeListener().onUnmanaged(bundleMetadata);
 
@@ -373,16 +378,22 @@ public class OsgiManager {
     // it probably works even if that is true, but we should consider what to do;
     // possibly remove that other capability, so that bundles with BOMs _have_ to be installed via this method.
     // (load order gets confusing with auto-scanning...)
-    public List<? extends CatalogItem<?,?>> loadCatalogBom(Bundle bundle) {
-        return loadCatalogBom(bundle, false);
+    public List<? extends CatalogItem<?,?>> loadCatalogBomLegacy(Bundle bundle) {
+        return loadCatalogBomLegacy(bundle, false);
     }
     
     @Beta  // as above
-    public List<? extends CatalogItem<?,?>> loadCatalogBom(Bundle bundle, boolean force) {
-        return MutableList.copyOf(loadCatalogBom(mgmt, bundle, force));
+    public List<? extends CatalogItem<?,?>> loadCatalogBomLegacy(Bundle bundle, boolean force) {
+        return MutableList.copyOf(loadCatalogBomInternal(mgmt, bundle, force, true, true));
     }
     
-    private static Iterable<? extends CatalogItem<?, ?>> loadCatalogBom(ManagementContext mgmt, Bundle bundle, boolean force) {
+    // since 0.12.0 no longer returns items; it installs non-persisted RegisteredTypes to the type registry instead 
+    @Beta
+    public void loadCatalogBom(Bundle bundle, boolean force, boolean validate) {
+        loadCatalogBomInternal(mgmt, bundle, force, validate, false);
+    }
+    
+    private static Iterable<? extends CatalogItem<?, ?>> loadCatalogBomInternal(ManagementContext mgmt, Bundle bundle, boolean force, boolean validate, boolean legacy) {
         Iterable<? extends CatalogItem<?, ?>> catalogItems = MutableList.of();
         if (!BrooklynFeatureEnablement.isEnabled(BrooklynFeatureEnablement.FEATURE_LOAD_BUNDLE_CATALOG_BOM)) {
             // if the above feature is not enabled, let's do it manually (as a contract of this method)
@@ -392,7 +403,13 @@ public class OsgiManager {
                 // here to get back the predicate from it.
                 final Predicate<Bundle> applicationsPermitted = Predicates.<Bundle>alwaysTrue();
 
-                catalogItems = new CatalogBundleLoader(applicationsPermitted, mgmt).scanForCatalog(bundle, force);
+                CatalogBundleLoader cl = new CatalogBundleLoader(applicationsPermitted, mgmt);
+                if (legacy) {
+                    catalogItems = cl.scanForCatalogLegacy(bundle, force);
+                } else {
+                    cl.scanForCatalog(bundle, force, validate);
+                    catalogItems = null;
+                }
             } catch (RuntimeException ex) {
                 // TODO confirm -- as of May 2017 we no longer uninstall the bundle if install of catalog items fails;
                 // caller needs to upgrade, or uninstall then reinstall
@@ -603,4 +620,22 @@ public class OsgiManager {
     public Framework getFramework() {
         return framework;
     }
+
+    // track wrapper bundles lifecvcle specially, to avoid removing it while it's installing
+    public void addInstalledWrapperBundle(ManagedBundle mb) {
+        synchronized (wrapperBundles) {
+            wrapperBundles.put(mb.getVersionedName(), mb);
+        }
+    }
+    public void removeInstalledWrapperBundle(ManagedBundle mb) {
+        synchronized (wrapperBundles) {
+            wrapperBundles.remove(mb.getVersionedName());
+        }
+    }
+    public Collection<ManagedBundle> getInstalledWrapperBundles() {
+        synchronized (wrapperBundles) {
+            return MutableSet.copyOf(wrapperBundles.values());
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextImpl.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextImpl.java
index 6bea476..17f0349 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextImpl.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextImpl.java
@@ -91,10 +91,11 @@ public class RebindContextImpl implements RebindContext {
         catalogItems.put(id, catalogItem);
     }
 
-    // we don't track register/unregister of bundles; it isn't needed as it happens so early
-    // but we do need to know which ones to start subsequently
+    /** install the bundles into brooklyn and osgi, but do not start nor validate;
+     * caller (rebind) will do that manually, doing each step across all bundles before proceeding 
+     * to prevent reference errors */
     public OsgiBundleInstallationResult installBundle(ManagedBundle bundle, InputStream zipInput) {
-        return ((ManagementContextInternal)mgmt).getOsgiManager().get().installDeferredStart(bundle, zipInput).get();
+        return ((ManagementContextInternal)mgmt).getOsgiManager().get().installDeferredStart(bundle, zipInput, false).get();
     }
     public void startBundle(OsgiBundleInstallationResult br) throws BundleException {
         if (br.getDeferredStart()!=null) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextLookupContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextLookupContext.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextLookupContext.java
index f463047..c1ebc5e 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextLookupContext.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/RebindContextLookupContext.java
@@ -116,14 +116,17 @@ public class RebindContextLookupContext implements LookupContext {
 
     @SuppressWarnings("deprecation")
     @Override
+    // only used for persisted xml catalog items; not used for registered types
     public CatalogItem<?, ?> lookupCatalogItem(String id) {
         CatalogItem<?, ?> result = rebindContext.getCatalogItem(id);
         if (result == null) {
-            // TODO-type-registry
-//          result = managementContext.getTypeRegistry().get(id, null, null);
             result = CatalogUtils.getCatalogItemOptionalVersion(managementContext, id);
         }
         if (result == null) {
+            if (managementContext.getTypeRegistry().get(id)!=null) {
+                // don't treat as dangling; caller should now recognise null as meaning it's known in the type registry
+                return null;
+            }
             result = exceptionHandler.onDanglingCatalogItemRef(id);
         }
         return result;

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/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 a80b14a..6ee3dc7 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
@@ -100,13 +100,16 @@ import org.apache.brooklyn.core.objs.proxy.InternalPolicyFactory;
 import org.apache.brooklyn.core.policy.AbstractPolicy;
 import org.apache.brooklyn.core.typereg.BasicManagedBundle;
 import org.apache.brooklyn.core.typereg.RegisteredTypeNaming;
+import org.apache.brooklyn.core.typereg.RegisteredTypePredicates;
 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.ClassLoaderUtils;
 import org.apache.brooklyn.util.core.flags.FlagUtils;
 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.VersionedName;
 import org.apache.brooklyn.util.text.Strings;
 import org.apache.brooklyn.util.time.Duration;
 import org.apache.brooklyn.util.time.Time;
@@ -117,6 +120,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
@@ -337,13 +341,40 @@ public abstract class RebindIteration {
                 }
             }
             // Start them all after we've installed them
+            Set<RegisteredType> installedTypes = MutableSet.of();
             for (OsgiBundleInstallationResult br: installs) {
                 try {
                     rebindContext.startBundle(br);
+                    Iterables.addAll(installedTypes, managementContext.getTypeRegistry().getMatching(
+                        RegisteredTypePredicates.containingBundle(br.getVersionedName())));
                 } catch (Exception e) {
                     exceptionHandler.onCreateFailed(BrooklynObjectType.MANAGED_BUNDLE, br.getMetadata().getId(), br.getMetadata().getSymbolicName(), e);
                 }
             }
+            // Now validate all types
+            Map<RegisteredType, Collection<Throwable>> validationErrors = this.managementContext.getCatalog().validateTypes( installedTypes );
+            if (!validationErrors.isEmpty()) {
+                Map<VersionedName, Map<RegisteredType,Collection<Throwable>>> errorsByBundle = MutableMap.of();
+                for (RegisteredType t: validationErrors.keySet()) {
+                    VersionedName vn = VersionedName.fromString(t.getContainingBundle());
+                    Map<RegisteredType, Collection<Throwable>> errorsInBundle = errorsByBundle.get(vn);
+                    if (errorsInBundle==null) {
+                        errorsInBundle = MutableMap.of();
+                        errorsByBundle.put(vn, errorsInBundle);
+                    }
+                    errorsInBundle.put(t, validationErrors.get(t));
+                }
+                for (VersionedName vn: errorsByBundle.keySet()) {
+                    Map<RegisteredType, Collection<Throwable>> errorsInBundle = errorsByBundle.get(vn);
+                    ManagedBundle b = managementContext.getOsgiManager().get().getManagedBundle(vn);
+                    exceptionHandler.onCreateFailed(BrooklynObjectType.MANAGED_BUNDLE, 
+                        b!=null ? b.getId() : /* just in case it was uninstalled concurrently somehow */ vn.toString(),
+                        vn.getSymbolicName(), 
+                        Exceptions.create("Failed to install "+vn+", types "+errorsInBundle.keySet()+" gave errors",
+                            Iterables.concat(errorsInBundle.values())));
+                }
+            }
+
         } else {
             logRebindingDebug("Not rebinding bundles; feature disabled: {}", mementoManifest.getBundleIds());
         }
@@ -1021,20 +1052,37 @@ public abstract class RebindIteration {
             List<String> reboundSearchPath = MutableList.of();
             if (searchPath != null && !searchPath.isEmpty()) {
                 for (String searchItemId : searchPath) {
-                    CatalogItem<?, ?> searchItem = findCatalogItemInReboundCatalog(bType, searchItemId, contextSuchAsId);
-                    if (searchItem != null) {
-                        reboundSearchPath.add(searchItem.getCatalogItemId());
+                    String fixedSearchItemId = null;
+                    RegisteredType t1 = managementContext.getTypeRegistry().get(searchItemId);
+                    if (t1!=null) fixedSearchItemId = t1.getId();
+                    if (fixedSearchItemId==null) {
+                        CatalogItem<?, ?> ci = findCatalogItemInReboundCatalog(bType, searchItemId, contextSuchAsId);
+                        if (ci!=null) fixedSearchItemId = ci.getCatalogItemId();
+                    }
+                    if (fixedSearchItemId != null) {
+                        reboundSearchPath.add(fixedSearchItemId);
                     } else {
                         LOG.warn("Unable to load catalog item "+ searchItemId
-                            +" for "+contextSuchAsId + " (" + bType.getSimpleName()+"); attempting load nevertheless");
+                            +" for search path of "+contextSuchAsId + " (" + bType.getSimpleName()+"); attempting load nevertheless");
                     }
                 }
             }
 
             if (catalogItemId != null) {
-                CatalogItem<?, ?> catalogItem = findCatalogItemInReboundCatalog(bType, catalogItemId, contextSuchAsId);
-                if (catalogItem != null) {
-                    String transformedCatalogItemId = catalogItem.getCatalogItemId();
+                String transformedCatalogItemId = null;
+                
+                Maybe<RegisteredType> registeredType = managementContext.getTypeRegistry().getMaybe(catalogItemId,
+                    // ignore bType; catalog item ID gives us the search path, but doesn't need to be of the requested type
+                    null );
+                if (registeredType.isPresent()) {
+                    transformedCatalogItemId = registeredType.get().getId();
+                } else {
+                    CatalogItem<?, ?> catalogItem = findCatalogItemInReboundCatalog(bType, catalogItemId, contextSuchAsId);
+                    if (catalogItem != null) {
+                        transformedCatalogItemId = catalogItem.getCatalogItemId();
+                    }
+                }
+                if (transformedCatalogItemId!=null) {
                     try {
                         BrooklynClassLoadingContextSequential loader =
                             new BrooklynClassLoadingContextSequential(managementContext);
@@ -1046,7 +1094,7 @@ public abstract class RebindIteration {
                         LOG.warn("Unable to load "+jType+" using loader; will try reflections");
                     }
                 } else {
-                    LOG.warn("Unable to load catalog item "+catalogItemId+" for " + contextSuchAsId +
+                    LOG.warn("Unable to load catalog item "+catalogItemId+" ("+bType+") for " + contextSuchAsId +
                       " ("+bType.getSimpleName()+"); will try reflection");
                 }
             }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/bf31100c/core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java b/core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java
index 0368ba3..f6ee347 100644
--- a/core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java
+++ b/core/src/main/java/org/apache/brooklyn/core/typereg/RegisteredTypeLoadingContexts.java
@@ -128,7 +128,8 @@ public class RegisteredTypeLoadingContexts {
     public static RegisteredTypeLoadingContext spec(Class<? extends BrooklynObject> javaSuperType) {
         return of(RegisteredTypeKind.SPEC, javaSuperType);
     }
-    
+
+    /** Adds the required supertype for beans, or supertype of the spec target (e.g. Entity not EntitySpec) for specs */
     public static <T> RegisteredTypeLoadingContext withBeanSuperType(@Nullable RegisteredTypeLoadingContext source, @Nullable Class<T> beanSuperType) {
         Class<T> superType = beanSuperType;
         BasicRegisteredTypeLoadingContext constraint = new BasicRegisteredTypeLoadingContext(source);