You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by he...@apache.org on 2017/04/21 22:55:32 UTC

[13/39] brooklyn-server git commit: Add support for catalogItemSuperIds to rebind.

Add support for catalogItemSuperIds to rebind.

Fixes testReboundDeepCatalogItemCanLoadResources.


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

Branch: refs/heads/master
Commit: 4c7ca74f6a0a2092d5c7c6cdc6e711ab5fb5f75c
Parents: 3e05d39
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Authored: Tue Sep 27 11:21:46 2016 +0100
Committer: Geoff Macartney <ge...@cloudsoftcorp.com>
Committed: Thu Apr 20 11:20:35 2017 +0100

----------------------------------------------------------------------
 .../mementos/BrooklynMementoManifest.java       |  30 ++---
 .../api/mgmt/rebind/mementos/Memento.java       |  33 +++--
 .../brooklyn/camp/brooklyn/RebindOsgiTest.java  |   6 +-
 .../core/catalog/internal/CatalogUtils.java     |  30 ++++-
 .../internal/AbstractManagementContext.java     |  25 +---
 .../BrooklynMementoPersisterToObjectStore.java  |  58 +++++++--
 .../core/mgmt/rebind/RebindIteration.java       | 125 ++++++++++---------
 .../core/mgmt/rebind/dto/AbstractMemento.java   |  16 ++-
 .../rebind/dto/BrooklynMementoManifestImpl.java |   5 +-
 .../rebind/dto/EntityMementoManifestImpl.java   |  17 ++-
 .../mgmt/rebind/dto/MementosGenerators.java     |   7 +-
 .../brooklyn/util/core/xstream/XmlUtil.java     |  26 ++--
 12 files changed, 229 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoManifest.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoManifest.java b/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoManifest.java
index a96601f..d3c3cc3 100644
--- a/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoManifest.java
+++ b/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/BrooklynMementoManifest.java
@@ -20,6 +20,7 @@ package org.apache.brooklyn.api.mgmt.rebind.mementos;
 
 import java.io.Serializable;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.brooklyn.api.objs.Identifiable;
@@ -32,30 +33,31 @@ import org.apache.brooklyn.api.objs.Identifiable;
 public interface BrooklynMementoManifest extends Serializable {
     public interface EntityMementoManifest extends Identifiable{
         @Override
-        public String getId();
-        public String getType();
-        public String getParent();
-        public String getCatalogItemId();
+        String getId();
+        String getType();
+        String getParent();
+        String getCatalogItemId();
+        List<String> getCatalogItemSuperIds();
     }
 
-    public String getPlaneId();
+    String getPlaneId();
 
-    public Map<String, EntityMementoManifest> getEntityIdToManifest();
+    Map<String, EntityMementoManifest> getEntityIdToManifest();
 
-    public Map<String, String> getLocationIdToType();
+    Map<String, String> getLocationIdToType();
 
-    public Map<String, String> getPolicyIdToType();
+    Map<String, String> getPolicyIdToType();
 
-    public Map<String, String> getEnricherIdToType();
+    Map<String, String> getEnricherIdToType();
 
-    public Map<String, String> getFeedIdToType();
+    Map<String, String> getFeedIdToType();
     
-    public CatalogItemMemento getCatalogItemMemento(String id);
+    CatalogItemMemento getCatalogItemMemento(String id);
 
-    public Collection<String> getCatalogItemIds();
+    Collection<String> getCatalogItemIds();
 
-    public Map<String, CatalogItemMemento> getCatalogItemMementos();
+    Map<String, CatalogItemMemento> getCatalogItemMementos();
 
-    public boolean isEmpty();
+    boolean isEmpty();
     
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/Memento.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/Memento.java b/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/Memento.java
index 5911f28..d45df06 100644
--- a/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/Memento.java
+++ b/api/src/main/java/org/apache/brooklyn/api/mgmt/rebind/mementos/Memento.java
@@ -20,6 +20,7 @@ package org.apache.brooklyn.api.mgmt.rebind.mementos;
 
 import java.io.Serializable;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -43,11 +44,19 @@ public interface Memento extends Serializable {
     
     String getId();
     
-    public String getType();
-    
-    public String getCatalogItemId();
-    
-    public String getDisplayName();
+    String getType();
+
+    /**
+     * The principal catalog item id.
+     */
+    String getCatalogItemId();
+
+    /**
+     * Catalog Item Ids of all defining catalog items.
+     */
+    List<String> getCatalogItemSuperIds();
+
+    String getDisplayName();
     
     /**
      * A (weakly-typed) property set for this memento.
@@ -62,11 +71,11 @@ public interface Memento extends Serializable {
      * @deprecated since 0.7.0; use config/attributes so generic persistence will work, rather than requiring "custom fields"
      */
     @Deprecated
-    public Map<String, ? extends Object> getCustomFields();
+    Map<String, ? extends Object> getCustomFields();
     
-    public String toVerboseString();
+    String toVerboseString();
     
-    public void injectTypeClass(Class<?> clazz);
+    void injectTypeClass(Class<?> clazz);
     
     /**
      * Returns the injected type class, or null if not injected.
@@ -74,12 +83,12 @@ public interface Memento extends Serializable {
      * This is useful for ensuring the correct classloader is used (e.g. for {@link EntityMemento} 
      * previously calling {@code EntityTypes.getDefinedSensors(getType())}. 
      */
-    public Class<?> getTypeClass();
+    Class<?> getTypeClass();
 
-    public Collection<Object> getTags();
+    Collection<Object> getTags();
     
-    public Map<String,Set<String>> getRelations();
+    Map<String,Set<String>> getRelations();
     
     /** Null for {@link Entity}, but important for adjuncts; see {@link EntityAdjunct#getUniqueTag()} */
-    public String getUniqueTag();
+    String getUniqueTag();
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/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
index 37c15e4..b50e38a 100644
--- 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
@@ -46,10 +46,12 @@ import org.apache.brooklyn.core.test.entity.TestEntity;
 import org.apache.brooklyn.test.support.TestResourceUnavailableException;
 import org.apache.brooklyn.util.core.ResourceUtils;
 import org.apache.brooklyn.util.core.osgi.Osgis;
+import org.apache.brooklyn.util.core.xstream.XmlUtil;
 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.apache.brooklyn.util.text.Strings;
 import org.jclouds.compute.domain.OsFamily;
 import org.osgi.framework.Bundle;
 import org.osgi.framework.launch.Framework;
@@ -65,6 +67,8 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
+import javax.xml.xpath.XPathConstants;
+
 public class RebindOsgiTest extends AbstractYamlRebindTest {
 
     @SuppressWarnings("unused")
@@ -164,7 +168,7 @@ public class RebindOsgiTest extends AbstractYamlRebindTest {
     }
 
 
-    @Test(groups = "Broken")
+    @Test
     public void testReboundDeepCatalogItemCanLoadResources() throws Exception {
         TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
         TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_MORE_ENTITIES_0_1_0_PATH);

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/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 42e8766..8524455 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
@@ -20,11 +20,10 @@ package org.apache.brooklyn.core.catalog.internal;
 
 import java.util.Collection;
 import java.util.Iterator;
-import java.util.ListIterator;
+import java.util.List;
 
 import javax.annotation.Nullable;
 
-import com.google.common.collect.Lists;
 import org.apache.brooklyn.api.catalog.BrooklynCatalog;
 import org.apache.brooklyn.api.catalog.CatalogItem;
 import org.apache.brooklyn.api.catalog.CatalogItem.CatalogBundle;
@@ -334,6 +333,33 @@ public class CatalogUtils {
         mgmt.getCatalog().persist(item);
     }
 
+    public static BrooklynClassLoadingContextSequential newClassLoadingContextForCatalogItems(
+        ManagementContext managementContext, List<String> catalogItemIds) {
+
+        BrooklynClassLoadingContextSequential seqLoader =
+            new BrooklynClassLoadingContextSequential(managementContext);
+        for (String catalogItemId : catalogItemIds) {
+            addCatalogItemContext(managementContext, seqLoader, catalogItemId);
+        }
+        // TODO what if not all items were found? need to consider what the right behaviour is.
+        // TODO for now take the course of using whatever items we *did* find
+        if (seqLoader.getPrimaries().size() != catalogItemIds.size()) {
+            log.warn("Couldn't find all catalog items  used for instantiating entity " + managementContext);
+        }
+        return seqLoader;
+    }
+
+    private static void addCatalogItemContext(ManagementContext managementContext, BrooklynClassLoadingContextSequential loader, String catalogItemId) {
+        RegisteredType item = managementContext.getTypeRegistry().get(catalogItemId);
+
+        if (item != null) {
+            BrooklynClassLoadingContext itemLoader = newClassLoadingContext(managementContext, item);
+            loader.add(itemLoader);
+        } else {
+            // TODO review what to do here
+            log.debug("Can't find catalog item " + catalogItemId);
+        }
+    }
 
     public static String[] bundleIds(Bundle bundle) {
         return new String[] {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/AbstractManagementContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/AbstractManagementContext.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/AbstractManagementContext.java
index a92a07b..36ba8d1 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/AbstractManagementContext.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/internal/AbstractManagementContext.java
@@ -20,6 +20,7 @@ package org.apache.brooklyn.core.mgmt.internal;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 import static java.lang.String.format;
+import static org.apache.brooklyn.core.catalog.internal.CatalogUtils.newClassLoadingContextForCatalogItems;
 
 import java.net.URI;
 import java.net.URL;
@@ -50,7 +51,6 @@ import org.apache.brooklyn.api.mgmt.ha.HighAvailabilityManager;
 import org.apache.brooklyn.api.mgmt.rebind.RebindManager;
 import org.apache.brooklyn.api.objs.BrooklynObject;
 import org.apache.brooklyn.api.typereg.BrooklynTypeRegistry;
-import org.apache.brooklyn.api.typereg.RegisteredType;
 import org.apache.brooklyn.config.StringConfigMap;
 import org.apache.brooklyn.core.catalog.internal.BasicBrooklynCatalog;
 import org.apache.brooklyn.core.catalog.internal.CatalogInitialization;
@@ -134,15 +134,7 @@ public abstract class AbstractManagementContext implements ManagementContextInte
                     final List<String> catalogItemSuperIds = internal.getCatalogItemSuperIds();
                     if (catalogItemSuperIds.size() > 0) {
                         BrooklynClassLoadingContextSequential seqLoader =
-                            new BrooklynClassLoadingContextSequential(internal.getManagementContext());
-                        for (String catalogItemId : catalogItemSuperIds) {
-                            addCatalogItemContext(internal, seqLoader, catalogItemId);
-                        }
-                        // TODO what if not all items were found? need to consider what the right behaviour is.
-                        // TODO for now take the course of using whatever items we *did* find
-                        if (seqLoader.getPrimaries().size() != catalogItemSuperIds.size()) {
-                            log.error("Couldn't find all catalog items  used for instantiating entity " + internal);
-                        }
+                            newClassLoadingContextForCatalogItems(internal.getManagementContext(), catalogItemSuperIds);
                         JavaBrooklynClassLoadingContext entityLoader =
                             JavaBrooklynClassLoadingContext.create(input.getClass().getClassLoader());
                         seqLoader.add(entityLoader);
@@ -160,19 +152,6 @@ public abstract class AbstractManagementContext implements ManagementContextInte
         });
     }
 
-    private static void addCatalogItemContext(EntityInternal entity, BrooklynClassLoadingContextSequential loader, String catalogItemId) {
-        RegisteredType item = entity.getManagementContext().getTypeRegistry().get(catalogItemId);
-
-        if (item != null) {
-            BrooklynClassLoadingContext itemLoader = CatalogUtils.newClassLoadingContext(entity.getManagementContext(), item);
-            loader.add(itemLoader);
-        } else {
-            log.error("Can't find catalog item " + catalogItemId +
-                " used for instantiating entity " + entity +
-                ". Falling back to application classpath.");
-        }
-    }
-
     private final AtomicLong totalEffectorInvocationCount = new AtomicLong();
 
     protected DeferredBrooklynProperties configMap;

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/BrooklynMementoPersisterToObjectStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/BrooklynMementoPersisterToObjectStore.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/BrooklynMementoPersisterToObjectStore.java
index a89a528..019ac86 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/BrooklynMementoPersisterToObjectStore.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/persist/BrooklynMementoPersisterToObjectStore.java
@@ -35,7 +35,9 @@ import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import javax.annotation.Nullable;
+import javax.xml.xpath.XPathConstants;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.brooklyn.api.mgmt.rebind.PersistenceExceptionHandler;
 import org.apache.brooklyn.api.mgmt.rebind.RebindExceptionHandler;
 import org.apache.brooklyn.api.mgmt.rebind.mementos.BrooklynMemento;
@@ -57,6 +59,7 @@ import org.apache.brooklyn.core.mgmt.persist.PersistenceObjectStore.StoreObjectA
 import org.apache.brooklyn.core.mgmt.rebind.PeriodicDeltaChangeListener;
 import org.apache.brooklyn.core.mgmt.rebind.dto.BrooklynMementoImpl;
 import org.apache.brooklyn.core.mgmt.rebind.dto.BrooklynMementoManifestImpl;
+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.xstream.XmlUtil;
@@ -77,6 +80,7 @@ import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
+import org.w3c.dom.NodeList;
 
 /** Implementation of the {@link BrooklynMementoPersister} backed by a pluggable
  * {@link PersistenceObjectStore} such as a file system or a jclouds object store */
@@ -324,8 +328,47 @@ public class BrooklynMementoPersisterToObjectStore implements BrooklynMementoPer
         return result;
     }
 
+    private static class XPathHelper {
+        private String contents;
+        private String prefix;
+
+        public XPathHelper(String contents, String prefix) {
+            this.contents = contents;
+            this.prefix = prefix;
+        }
+
+        private String get(String innerPath) {
+            return (String) XmlUtil.xpathHandlingIllegalChars(contents, prefix+innerPath);
+        }
+        private List<String> getStringList(String innerPath) {
+            List<String> result = MutableList.of();
+            final NodeList nodeList =
+                (NodeList) XmlUtil.xpathHandlingIllegalChars(contents, prefix + innerPath + "/string", XPathConstants.NODESET);
+            for(int c = 0 ; c < nodeList.getLength() ; c++) {
+                result.add(nodeList.item(c).getFirstChild().getNodeValue());
+            }
+            return result;
+        }
+    }
+
+    // We must be able to cope with XML serialized with either a single "catalogItemId"
+    // or a list "catalogItemSuperIds" of catalog item ids. Only one should be encountered
+    // but in any case prefer the list of ids.
+    private ImmutableList<String> getCatalogItemIds(XPathHelper x) {
+        final MutableList<String> list = MutableList.of();
+        final List<String> catalogItemSuperIds = x.getStringList("catalogItemSuperIds");
+        final String catalogItemId = Strings.emptyToNull(x.get("catalogItemId"));
+        if (!catalogItemSuperIds.isEmpty()) {
+            list.addAll(catalogItemSuperIds);
+        } else if (catalogItemId != null) {
+            list.add(catalogItemId);
+        }
+        return ImmutableList.copyOf(list);
+    }
+
     @Override
-    public BrooklynMementoManifest loadMementoManifest(BrooklynMementoRawData mementoData, final RebindExceptionHandler exceptionHandler) throws IOException {
+    public BrooklynMementoManifest loadMementoManifest(BrooklynMementoRawData mementoData,
+                                                       final RebindExceptionHandler exceptionHandler) throws IOException {
         if (mementoData==null)
             mementoData = loadMementoRawData(exceptionHandler);
         
@@ -336,19 +379,10 @@ public class BrooklynMementoPersisterToObjectStore implements BrooklynMementoPer
         Visitor visitor = new Visitor() {
             @Override
             public void visit(BrooklynObjectType type, String objectId, final String contents) throws Exception {
-                final String prefix = "/"+type.toCamelCase()+"/";
-
-                class XPathHelper {
-                    private String get(String innerPath) {
-                        return (String) XmlUtil.xpathHandlingIllegalChars(contents, prefix+innerPath);
-                    }
-                }
-                XPathHelper x = new XPathHelper();
-                
+                XPathHelper x = new XPathHelper(contents, "/"+type.toCamelCase()+"/");
                 switch (type) {
                     case ENTITY:
-                        builder.entity(x.get("id"), x.get("type"), 
-                            Strings.emptyToNull(x.get("parent")), Strings.emptyToNull(x.get("catalogItemId")));
+                        builder.entity(x.get("id"), x.get("type"), Strings.emptyToNull(x.get("parent")), getCatalogItemIds(x));
                         break;
                     case LOCATION:
                     case POLICY:

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/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 0585f9b..0054a19 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
@@ -30,6 +30,7 @@ import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.brooklyn.api.catalog.BrooklynCatalog;
 import org.apache.brooklyn.api.catalog.CatalogItem;
 import org.apache.brooklyn.api.entity.Application;
@@ -794,9 +795,9 @@ public abstract class RebindIteration {
         }
     }
     
-    protected String findCatalogItemId(ClassLoader cl, Map<String, EntityMementoManifest> entityIdToManifest, EntityMementoManifest entityManifest) {
-        if (entityManifest.getCatalogItemId() != null) {
-            return entityManifest.getCatalogItemId();
+    protected List<String> findCatalogItemIds(ClassLoader cl, Map<String, EntityMementoManifest> entityIdToManifest, EntityMementoManifest entityManifest) {
+        if (!entityManifest.getCatalogItemSuperIds().isEmpty()) {
+            return entityManifest.getCatalogItemSuperIds();
         }
 
         if (BrooklynFeatureEnablement.isEnabled(BrooklynFeatureEnablement.FEATURE_BACKWARDS_COMPATIBILITY_INFER_CATALOG_ITEM_ON_REBIND)) {
@@ -806,12 +807,12 @@ public abstract class RebindIteration {
                 if (ptr.getCatalogItemId() != null) {
                     RegisteredType type = managementContext.getTypeRegistry().get(ptr.getCatalogItemId());
                     if (type != null) {
-                        return type.getId();
+                        return ImmutableList.of(type.getId());
                     } else {
                         //Couldn't find a catalog item with this id, but return it anyway and
                         //let the caller deal with the error.
                         //TODO under what circumstances is this permitted?
-                        return ptr.getCatalogItemId();
+                        return ImmutableList.of(ptr.getCatalogItemId());
                     }
                 }
                 if (ptr.getParent() != null) {
@@ -831,7 +832,7 @@ public abstract class RebindIteration {
                 RegisteredType t = types.get(ptr.getType(), BrooklynCatalog.DEFAULT_VERSION);
                 if (t != null) {
                     LOG.debug("Inferred catalog item ID "+t.getId()+" for "+entityManifest+" from ancestor "+ptr);
-                    return t.getId();
+                    return ImmutableList.of(t.getId());
                 }
                 if (ptr.getParent() != null) {
                     ptr = entityIdToManifest.get(ptr.getParent());
@@ -851,20 +852,20 @@ public abstract class RebindIteration {
                 boolean canLoadClass = loader.tryLoadClass(entityManifest.getType()).isPresent();
                 if (canLoadClass) {
                     LOG.warn("Missing catalog item for "+entityManifest.getId()+" ("+entityManifest.getType()+"), inferring as "+item.getId()+" because that is able to load the item");
-                    return item.getId();
+                    return ImmutableList.of(item.getId());
                 }
             }
         }
-        return null;
+        return ImmutableList.of();
     }
 
     protected static class LoadedClass<T extends BrooklynObject> {
         protected final Class<? extends T> clazz;
-        protected final String catalogItemId;
+        protected final List<String> catalogItemIds;
         
-        protected LoadedClass(Class<? extends T> clazz, String catalogItemId) {
+        protected LoadedClass(Class<? extends T> clazz, List<String> catalogItemIds) {
             this.clazz = clazz;
-            this.catalogItemId = catalogItemId;
+            this.catalogItemIds = catalogItemIds;
         }
     }
 
@@ -882,13 +883,12 @@ public abstract class RebindIteration {
 
         protected Entity newEntity(EntityMementoManifest entityManifest) {
             String entityId = entityManifest.getId();
-            String catalogItemId = findCatalogItemId(classLoader, mementoManifest.getEntityIdToManifest(), entityManifest);
+            List<String> catalogItemIds = findCatalogItemIds(classLoader, mementoManifest.getEntityIdToManifest(), entityManifest);
             String entityType = entityManifest.getType();
-            
-            LoadedClass<? extends Entity> loaded = load(Entity.class, entityType, catalogItemId, entityId);
+
+            LoadedClass<? extends Entity> loaded = load(Entity.class, entityType, catalogItemIds, entityId);
             Class<? extends Entity> entityClazz = loaded.clazz;
-            String transformedCatalogItemId = loaded.catalogItemId;
-            
+
             Entity entity;
             
             if (InternalFactory.isNewStyle(entityClazz)) {
@@ -924,63 +924,46 @@ public abstract class RebindIteration {
                 ((AbstractEntity)entity).setManagementContext(managementContext);
                 managementContext.prePreManage(entity);
             }
-            
-            setCatalogItemId(entity, transformedCatalogItemId);
+
+            setCatalogItemIds(entity, loaded.catalogItemIds);
+
             return entity;
         }
 
-        protected void setCatalogItemId(BrooklynObject item, String catalogItemId) {
-            if (catalogItemId!=null) {
-                // TODO add support for nested catalog superids here.
-                ((BrooklynObjectInternal)item).setCatalogItemId(catalogItemId);
-            }
+        protected void setCatalogItemIds(BrooklynObject object, List<String> superIds) {
+            ((BrooklynObjectInternal)object).setCatalogItemIds(superIds);
         }
 
+
         protected <T extends BrooklynObject> LoadedClass<? extends T> load(Class<T> bType, Memento memento) {
-            return load(bType, memento.getType(), memento.getCatalogItemId(), memento.getId());
+            return load(bType, memento.getType(), memento.getCatalogItemSuperIds(), memento.getId());
         }
         
         @SuppressWarnings("unchecked")
-        protected <T extends BrooklynObject> LoadedClass<? extends T> load(Class<T> bType, String jType, String catalogItemId, String contextSuchAsId) {
+        protected <T extends BrooklynObject> LoadedClass<? extends T> load(Class<T> bType, String jType, List<String> catalogItemIds, String contextSuchAsId) {
             checkNotNull(jType, "Type of %s (%s) must not be null", contextSuchAsId, bType.getSimpleName());
-            
-            if (catalogItemId != null) {
-                CatalogItem<?, ?> catalogItem = rebindContext.lookup().lookupCatalogItem(catalogItemId);
-                if (catalogItem == null) {
-                    if (BrooklynFeatureEnablement.isEnabled(BrooklynFeatureEnablement.FEATURE_AUTO_FIX_CATALOG_REF_ON_REBIND)) {
-                        // See https://issues.apache.org/jira/browse/BROOKLYN-149
-                        // This is a dangling reference to the catalog item (which will have been logged by lookupCatalogItem).
-                        // Try loading as any version.
-                        if (CatalogUtils.looksLikeVersionedId(catalogItemId)) {
-                            String symbolicName = CatalogUtils.getSymbolicNameFromVersionedId(catalogItemId);
-                            catalogItem = rebindContext.lookup().lookupCatalogItem(symbolicName);
-                            
-                            if (catalogItem != null) {
-                                LOG.warn("Unable to load catalog item "+catalogItemId+" for "+contextSuchAsId
-                                        +" ("+bType.getSimpleName()+"); will auto-upgrade to "+catalogItem.getCatalogItemId());
-                                catalogItemId = catalogItem.getCatalogItemId();
-                            }
-                        }
-                    }
-                }
-                if (catalogItem != null) {
-                    BrooklynClassLoadingContext loader = CatalogUtils.newClassLoadingContext(managementContext, catalogItem);
-                    return new LoadedClass<T>(loader.loadClass(jType, bType), catalogItemId);
+
+            List<String> idsFromReboundCatalog = MutableList.of();
+            if (catalogItemIds != null && !catalogItemIds.isEmpty()) {
+                findCatalogIdsInReboundCatalog(bType, catalogItemIds, contextSuchAsId, idsFromReboundCatalog);
+                if (!idsFromReboundCatalog.isEmpty()) {
+                    BrooklynClassLoadingContext loader = CatalogUtils.newClassLoadingContextForCatalogItems(managementContext, idsFromReboundCatalog);
+                    return new LoadedClass<T>(loader.loadClass(jType, bType), catalogItemIds);
                 } else {
-                    LOG.warn("Unable to load catalog item "+catalogItemId+" for "+contextSuchAsId
+                    LOG.warn("Unable to load catalog items "+ catalogItemIds +" for "+contextSuchAsId
                             +" ("+bType.getSimpleName()+"); will try default class loader");
                 }
             }
             
             try {
-                return new LoadedClass<T>((Class<T>)loadClass(jType), catalogItemId);
+                return new LoadedClass<T>((Class<T>)loadClass(jType), idsFromReboundCatalog);
             } catch (Exception e) {
                 Exceptions.propagateIfFatal(e);
                 LOG.warn("Unable to load "+jType+" using reflections; will try standard context");
             }
 
-            if (catalogItemId != null) {
-                throw new IllegalStateException("Unable to load catalog item "+catalogItemId+" for "+contextSuchAsId+", or load class from classpath");
+            if (catalogItemIds != null && !catalogItemIds.isEmpty()) {
+                throw new IllegalStateException("Unable to load catalog item "+ catalogItemIds +" for "+contextSuchAsId+", or load class from classpath");
             } else if (BrooklynFeatureEnablement.isEnabled(BrooklynFeatureEnablement.FEATURE_BACKWARDS_COMPATIBILITY_INFER_CATALOG_ITEM_ON_REBIND)) {
                 //Try loading from whichever catalog bundle succeeds.
                 BrooklynCatalog catalog = managementContext.getCatalog();
@@ -988,7 +971,7 @@ public abstract class RebindIteration {
                     BrooklynClassLoadingContext catalogLoader = CatalogUtils.newClassLoadingContext(managementContext, item);
                     Maybe<Class<?>> catalogClass = catalogLoader.tryLoadClass(jType);
                     if (catalogClass.isPresent()) {
-                        return new LoadedClass<T>((Class<? extends T>) catalogClass.get(), catalogItemId);
+                        return new LoadedClass<T>((Class<? extends T>) catalogClass.get(), catalogItemIds);
                     }
                 }
                 throw new IllegalStateException("No catalogItemId specified for "+contextSuchAsId+" and can't load class (" + jType + ") from either classpath or catalog items");
@@ -997,6 +980,31 @@ public abstract class RebindIteration {
             }
         }
 
+        private <T extends BrooklynObject> void findCatalogIdsInReboundCatalog(Class<T> bType, List<String> catalogItemIds, String contextSuchAsId, List<String> idsToUse) {
+            for (String catalogItemId : catalogItemIds) {
+                CatalogItem<?, ?> catalogItem = rebindContext.lookup().lookupCatalogItem(catalogItemId);
+                if (catalogItem == null) {
+                    if (BrooklynFeatureEnablement.isEnabled(BrooklynFeatureEnablement.FEATURE_AUTO_FIX_CATALOG_REF_ON_REBIND)) {
+                        // See https://issues.apache.org/jira/browse/BROOKLYN-149
+                        // This is a dangling reference to the catalog item (which will have been logged by lookupCatalogItem).
+                        // Try loading as any version.
+                        if (CatalogUtils.looksLikeVersionedId(catalogItemId)) {
+                            String symbolicName = CatalogUtils.getSymbolicNameFromVersionedId(catalogItemId);
+                            catalogItem = rebindContext.lookup().lookupCatalogItem(symbolicName);
+
+                            if (catalogItem != null) {
+                                LOG.warn("Unable to load catalog item "+ catalogItemIds +" for "+contextSuchAsId
+                                        +" ("+bType.getSimpleName()+"); will auto-upgrade to "+catalogItem.getCatalogItemId());
+                                idsToUse.add(catalogItem.getCatalogItemId());
+                            }
+                        }
+                    }
+                } else {
+                    idsToUse.add(catalogItemId);
+                }
+            }
+        }
+
         protected Class<?> loadClass(String jType) throws ClassNotFoundException {
             try {
             return reflections.loadClass(jType);
@@ -1051,9 +1059,8 @@ public abstract class RebindIteration {
          */
         protected Policy newPolicy(PolicyMemento memento) {
             String id = memento.getId();
-            LoadedClass<? extends Policy> loaded = load(Policy.class, memento.getType(), memento.getCatalogItemId(), id);
+            LoadedClass<? extends Policy> loaded = load(Policy.class, memento.getType(), memento.getCatalogItemSuperIds(), id);
             Class<? extends Policy> policyClazz = loaded.clazz;
-            String transformedCatalogItemId = loaded.catalogItemId;
 
             Policy policy;
             if (InternalFactory.isNewStyle(policyClazz)) {
@@ -1077,7 +1084,7 @@ public abstract class RebindIteration {
                 policy = invokeConstructor(null, policyClazz, new Object[] {flags});
             }
             
-            setCatalogItemId(policy, transformedCatalogItemId);
+            setCatalogItemIds(policy, memento.getCatalogItemSuperIds());
             return policy;
         }
 
@@ -1088,7 +1095,6 @@ public abstract class RebindIteration {
             String id = memento.getId();
             LoadedClass<? extends Enricher> loaded = load(Enricher.class, memento);
             Class<? extends Enricher> enricherClazz = loaded.clazz;
-            String transformedCatalogItemId = loaded.catalogItemId;
 
             Enricher enricher;
             if (InternalFactory.isNewStyle(enricherClazz)) {
@@ -1112,7 +1118,7 @@ public abstract class RebindIteration {
                 enricher = invokeConstructor(reflections, enricherClazz, new Object[] {flags});
             }
             
-            setCatalogItemId(enricher, transformedCatalogItemId);
+            setCatalogItemIds(enricher, memento.getCatalogItemSuperIds());
             return enricher;
         }
 
@@ -1123,7 +1129,6 @@ public abstract class RebindIteration {
             String id = memento.getId();
             LoadedClass<? extends Feed> loaded = load(Feed.class, memento);
             Class<? extends Feed> feedClazz = loaded.clazz;
-            String transformedCatalogItemId = loaded.catalogItemId;
 
             Feed feed;
             if (InternalFactory.isNewStyle(feedClazz)) {
@@ -1136,7 +1141,7 @@ public abstract class RebindIteration {
                 throw new IllegalStateException("rebind of feed without no-arg constructor unsupported: id="+id+"; type="+feedClazz);
             }
             
-            setCatalogItemId(feed, transformedCatalogItemId);
+            setCatalogItemIds(feed, memento.getCatalogItemSuperIds());
             return feed;
         }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/AbstractMemento.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/AbstractMemento.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/AbstractMemento.java
index 6fae7f5..5feb0af 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/AbstractMemento.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/AbstractMemento.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.Iterables;
 import org.apache.brooklyn.api.mgmt.rebind.mementos.Memento;
 import org.apache.brooklyn.core.BrooklynVersion;
 import org.apache.brooklyn.core.config.Sanitizer;
@@ -45,7 +46,7 @@ public abstract class AbstractMemento implements Memento, Serializable {
         protected String type;
         protected Class<?> typeClass;
         protected String displayName;
-        protected String catalogItemId;
+        protected List<String> catalogItemSuperIds;
         protected Map<String, Object> customFields = Maps.newLinkedHashMap();
         protected List<Object> tags = Lists.newArrayList();
         protected Map<String,Set<String>> relations = Maps.newLinkedHashMap();
@@ -64,7 +65,7 @@ public abstract class AbstractMemento implements Memento, Serializable {
             type = other.getType();
             typeClass = other.getTypeClass();
             displayName = other.getDisplayName();
-            catalogItemId = other.getCatalogItemId();
+            catalogItemSuperIds = other.getCatalogItemSuperIds();
             customFields.putAll(other.getCustomFields());
             tags.addAll(other.getTags());
             relations.putAll(other.getRelations());
@@ -85,7 +86,7 @@ public abstract class AbstractMemento implements Memento, Serializable {
     private String type;
     private String id;
     private String displayName;
-    private String catalogItemId;
+    private List<String> catalogItemSuperIds;
     private List<Object> tags;
     private Map<String,Set<String>> relations;
     
@@ -105,7 +106,7 @@ public abstract class AbstractMemento implements Memento, Serializable {
         type = builder.type;
         typeClass = builder.typeClass;
         displayName = builder.displayName;
-        catalogItemId = builder.catalogItemId;
+        catalogItemSuperIds = builder.catalogItemSuperIds;
         setCustomFields(builder.customFields);
         tags = toPersistedList(builder.tags);
         relations = toPersistedMap(builder.relations);
@@ -148,7 +149,12 @@ public abstract class AbstractMemento implements Memento, Serializable {
 
     @Override
     public String getCatalogItemId() {
-        return catalogItemId;
+        return Iterables.getFirst(getCatalogItemSuperIds(), null);
+    }
+
+    @Override
+    public List<String> getCatalogItemSuperIds() {
+        return catalogItemSuperIds;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/BrooklynMementoManifestImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/BrooklynMementoManifestImpl.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/BrooklynMementoManifestImpl.java
index 664ddfe..a4a7ecc 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/BrooklynMementoManifestImpl.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/BrooklynMementoManifestImpl.java
@@ -21,6 +21,7 @@ package org.apache.brooklyn.core.mgmt.rebind.dto;
 import java.io.Serializable;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.brooklyn.api.mgmt.rebind.mementos.BrooklynMementoManifest;
@@ -55,8 +56,8 @@ public class BrooklynMementoManifestImpl implements BrooklynMementoManifest, Ser
         public Builder brooklynVersion(String val) {
             brooklynVersion = val; return this;
         }
-        public Builder entity(String id, String type, String parent, String catalogItemId) {
-            entityIdToManifest.put(id, new EntityMementoManifestImpl(id, type, parent, catalogItemId));
+        public Builder entity(String id, String type, String parent, List<String> catalogItemIds) {
+            entityIdToManifest.put(id, new EntityMementoManifestImpl(id, type, parent, catalogItemIds));
             return this;
         }
         public Builder location(String id, String type) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/EntityMementoManifestImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/EntityMementoManifestImpl.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/EntityMementoManifestImpl.java
index accf741..0f20526 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/EntityMementoManifestImpl.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/EntityMementoManifestImpl.java
@@ -18,19 +18,23 @@
  */
 package org.apache.brooklyn.core.mgmt.rebind.dto;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
 import org.apache.brooklyn.api.mgmt.rebind.mementos.BrooklynMementoManifest.EntityMementoManifest;
 
+import java.util.List;
+
 public class EntityMementoManifestImpl implements EntityMementoManifest {
     private String id;
     private String type;
     private String parentId;
-    private String catalogItemId;
+    private List<String> catalogItemIds;
 
-    public EntityMementoManifestImpl(String id, String type, String parentId, String catalogItemId) {
+    public EntityMementoManifestImpl(String id, String type, String parentId, List<String> catalogItemIds) {
         this.id = id;
         this.type = type;
         this.parentId = parentId;
-        this.catalogItemId = catalogItemId;
+        this.catalogItemIds = ImmutableList.copyOf(catalogItemIds);
     }
 
     @Override
@@ -50,7 +54,12 @@ public class EntityMementoManifestImpl implements EntityMementoManifest {
 
     @Override
     public String getCatalogItemId() {
-        return catalogItemId;
+        return Iterables.getFirst(catalogItemIds, null);
+    }
+
+    @Override
+    public List<String> getCatalogItemSuperIds() {
+        return catalogItemIds;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/MementosGenerators.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/MementosGenerators.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/MementosGenerators.java
index da54080..305467f 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/MementosGenerators.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/rebind/dto/MementosGenerators.java
@@ -182,7 +182,7 @@ public class MementosGenerators {
         builder.isTopLevelApp = (entity instanceof Application && entity.getParent() == null);
 
         builder.configKeys.addAll(entity.getEntityType().getConfigKeys());
-        
+
         Map<ConfigKey<?>, ?> localConfig = entity.config().getAllLocalRaw();
         for (Map.Entry<ConfigKey<?>, ?> entry : localConfig.entrySet()) {
             ConfigKey<?> key = checkNotNull(entry.getKey(), localConfig);
@@ -448,11 +448,12 @@ public class MementosGenerators {
         }
         OsgiClassPrefixer prefixer = new OsgiClassPrefixer();
         Optional<String> typePrefix = prefixer.getPrefix(instance.getClass());
-        
+
         builder.id = instance.getId();
         builder.displayName = instance.getDisplayName();
-        builder.catalogItemId = instance.getCatalogItemId();
+        builder.catalogItemSuperIds = instance.getCatalogItemSuperIds();
         builder.type = (typePrefix.isPresent() ? typePrefix.get() : "") + instance.getClass().getName();
+        builder.type = instance.getClass().getName();
         builder.typeClass = instance.getClass();
         if (instance instanceof EntityAdjunct) {
             builder.uniqueTag = ((EntityAdjunct)instance).getUniqueTag();

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/4c7ca74f/core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlUtil.java b/core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlUtil.java
index 82b4867..8c4cc3d 100644
--- a/core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlUtil.java
+++ b/core/src/main/java/org/apache/brooklyn/util/core/xstream/XmlUtil.java
@@ -81,18 +81,22 @@ public class XmlUtil {
             throw Exceptions.propagate(e);
         }
     }
-    
-    /**
-     * Executes the given xpath on the given xml. If this fails becaues the xml is invalid
-     * (e.g. contains "&#x1b;"), then it will attempt to escape such illegal characters
-     * and try again. Note that the *escaped* values may be contained in the returned result!
-     * The escaping used is the prefix "BR_UNICODE_"; if that string is already in the xml,
-     * then it will replace that with "NOT_BR_UNICODE_".
-     */
-    @Beta
+
     public static Object xpathHandlingIllegalChars(String xml, String xpath) {
+        return xpathHandlingIllegalChars(xml, xpath, XPathConstants.STRING);
+    }
+
+        /**
+         * Executes the given xpath on the given xml. If this fails becaues the xml is invalid
+         * (e.g. contains "&#x1b;"), then it will attempt to escape such illegal characters
+         * and try again. Note that the *escaped* values may be contained in the returned result!
+         * The escaping used is the prefix "BR_UNICODE_"; if that string is already in the xml,
+         * then it will replace that with "NOT_BR_UNICODE_".
+         */
+    @Beta
+    public static Object xpathHandlingIllegalChars(String xml, String xpath, QName returnType) {
         try {
-            return xpath(xml, xpath);
+            return xpath(xml, xpath, returnType);
         } catch (Exception e) {
             SAXException saxe = Exceptions.getFirstThrowableOfType(e, SAXException.class);
             if (saxe != null && saxe.toString().contains("&#")) {
@@ -101,7 +105,7 @@ public class XmlUtil {
                 Escaper escaper = new Escaper();
                 String xmlCleaned = escaper.escape(xml);
                 try {
-                    Object result = xpath(xmlCleaned, xpath);
+                    Object result = xpath(xmlCleaned, xpath, returnType);
                     if (result instanceof String) {
                         return escaper.unescape((String)result);
                     } else {