You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by sj...@apache.org on 2015/10/15 16:02:23 UTC

[02/16] incubator-brooklyn git commit: CAMP parsing improvements

CAMP parsing improvements

Code cleanup, including:
  * fix recursive resolving check, avoiding stack overflow error
  * go through the PlanToSpecTransformer list for each nested catalog item, allowing nesting of different types of catalog items (already implemented, but now integrating it tighter with the parse workflow

Resolve catalog items tiop level, encountered types, cleanup


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

Branch: refs/heads/master
Commit: 3d415c41c3b3b0eeef3df1f4ad71cec7ddb8de40
Parents: 00615b7
Author: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Authored: Thu Oct 8 13:51:49 2015 +0300
Committer: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Committed: Wed Oct 14 17:08:53 2015 +0300

----------------------------------------------------------------------
 .../core/mgmt/EntityManagementUtils.java        |  10 +-
 .../core/plan/PlanToSpecTransformer.java        |   3 +-
 .../core/plan/XmlPlanToSpecTransformer.java     |   3 +-
 .../api/AssemblyTemplateSpecInstantiator.java   |  13 +-
 .../BrooklynAssemblyTemplateInstantiator.java   | 169 ++--------------
 .../BrooklynComponentTemplateResolver.java      | 194 +++++++++---------
 .../spi/creation/BrooklynEntityMatcher.java     |  15 +-
 .../brooklyn/spi/creation/CampCatalogUtils.java | 197 ++-----------------
 .../spi/creation/CampToSpecTransformer.java     |   8 +-
 .../camp/brooklyn/spi/creation/CampUtils.java   | 187 ++++++++++++++++++
 .../service/DefaultServiceTypeResolver.java     |  23 +++
 .../brooklyn/catalog/CatalogYamlEntityTest.java |  43 ++++
 .../test/lite/TestAppAssemblyInstantiator.java  |  12 +-
 13 files changed, 426 insertions(+), 451 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/core/src/main/java/org/apache/brooklyn/core/mgmt/EntityManagementUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/mgmt/EntityManagementUtils.java b/core/src/main/java/org/apache/brooklyn/core/mgmt/EntityManagementUtils.java
index 3a46416..0dca83c 100644
--- a/core/src/main/java/org/apache/brooklyn/core/mgmt/EntityManagementUtils.java
+++ b/core/src/main/java/org/apache/brooklyn/core/mgmt/EntityManagementUtils.java
@@ -20,6 +20,7 @@ package org.apache.brooklyn.core.mgmt;
 
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.Callable;
 
 import javax.annotation.Nullable;
@@ -55,6 +56,7 @@ import com.google.common.base.Function;
 import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 
@@ -96,11 +98,15 @@ public class EntityManagementUtils {
         }).get();
     }
 
-    public static <T,SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> SpecT createCatalogSpec(ManagementContext mgmt, final CatalogItem<T, SpecT> item) {
+    public static <T,SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> SpecT createCatalogSpec(ManagementContext mgmt, CatalogItem<T, SpecT> item) {
+        return createCatalogSpec(mgmt, item, ImmutableSet.<String>of());
+    }
+
+    public static <T,SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> SpecT createCatalogSpec(ManagementContext mgmt, final CatalogItem<T, SpecT> item, final Set<String> encounteredTypes) {
         return PlanToSpecFactory.attemptWithLoaders(mgmt, new Function<PlanToSpecTransformer, SpecT>() {
             @Override
             public SpecT apply(PlanToSpecTransformer input) {
-                return input.createCatalogSpec(item);
+                return input.createCatalogSpec(item, encounteredTypes);
             }
         }).get();
     }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/core/src/main/java/org/apache/brooklyn/core/plan/PlanToSpecTransformer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/brooklyn/core/plan/PlanToSpecTransformer.java b/core/src/main/java/org/apache/brooklyn/core/plan/PlanToSpecTransformer.java
index c6e3a4a..b9ca8ca 100644
--- a/core/src/main/java/org/apache/brooklyn/core/plan/PlanToSpecTransformer.java
+++ b/core/src/main/java/org/apache/brooklyn/core/plan/PlanToSpecTransformer.java
@@ -19,6 +19,7 @@
 package org.apache.brooklyn.core.plan;
 
 import java.util.ServiceLoader;
+import java.util.Set;
 
 import org.apache.brooklyn.api.catalog.CatalogItem;
 import org.apache.brooklyn.api.entity.Application;
@@ -56,6 +57,6 @@ public interface PlanToSpecTransformer extends ManagementContextInjectable {
      * implementations will typically look at the {@link CatalogItem#getCatalogItemType()} first.
      * <p>
      * should throw {@link PlanNotRecognizedException} if this transformer does not know what to do with the plan. */
-    <T,SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> SpecT createCatalogSpec(CatalogItem<T, SpecT> item) throws PlanNotRecognizedException;
+    <T,SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> SpecT createCatalogSpec(CatalogItem<T, SpecT> item, Set<String> encounteredTypes) throws PlanNotRecognizedException;
     
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/core/src/test/java/org/apache/brooklyn/core/plan/XmlPlanToSpecTransformer.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/brooklyn/core/plan/XmlPlanToSpecTransformer.java b/core/src/test/java/org/apache/brooklyn/core/plan/XmlPlanToSpecTransformer.java
index a535932..a4c3766 100644
--- a/core/src/test/java/org/apache/brooklyn/core/plan/XmlPlanToSpecTransformer.java
+++ b/core/src/test/java/org/apache/brooklyn/core/plan/XmlPlanToSpecTransformer.java
@@ -19,6 +19,7 @@
 package org.apache.brooklyn.core.plan;
 
 import java.io.StringReader;
+import java.util.Set;
 
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -74,7 +75,7 @@ public class XmlPlanToSpecTransformer implements PlanToSpecTransformer {
 
     @SuppressWarnings({ "unchecked" })
     @Override
-    public <T, SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> SpecT createCatalogSpec(CatalogItem<T, SpecT> item) {
+    public <T, SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> SpecT createCatalogSpec(CatalogItem<T, SpecT> item, Set<String> encounteredTypes) {
         if (item.getPlanYaml()==null) throw new PlanNotRecognizedException("Plan is null");
         if (item.getCatalogItemType()==CatalogItemType.ENTITY) {
             return (SpecT)toEntitySpec(parseXml(item.getPlanYaml()), 1);

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/api/AssemblyTemplateSpecInstantiator.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/api/AssemblyTemplateSpecInstantiator.java b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/api/AssemblyTemplateSpecInstantiator.java
index df7227b..1dfc351 100644
--- a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/api/AssemblyTemplateSpecInstantiator.java
+++ b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/api/AssemblyTemplateSpecInstantiator.java
@@ -18,6 +18,7 @@
  */
 package org.apache.brooklyn.camp.brooklyn.api;
 
+import java.util.List;
 import java.util.Set;
 
 import org.apache.brooklyn.api.entity.Application;
@@ -29,7 +30,13 @@ import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
 
 public interface AssemblyTemplateSpecInstantiator extends AssemblyTemplateInstantiator {
 
-    EntitySpec<? extends Application> createSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader, boolean autoUnwrapIfAppropriate);
-    EntitySpec<?> createNestedSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext itemLoader, Set<String> encounteredCatalogTypes);
-    
+    /**
+     * Gets the single item returned by {@link #createServiceSpecs}
+     * and wraps it in an Application if needed, applying top-level
+     * attributes and locations to the root entity.
+     */
+    EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader);
+    /** Returns specs for each item in the services list */
+    List<EntitySpec<?>> createServiceSpecs(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext itemLoader, Set<String> encounteredCatalogTypes);
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynAssemblyTemplateInstantiator.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynAssemblyTemplateInstantiator.java b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynAssemblyTemplateInstantiator.java
index 63f111a..e54e2ef 100644
--- a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynAssemblyTemplateInstantiator.java
+++ b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynAssemblyTemplateInstantiator.java
@@ -18,30 +18,20 @@
  */
 package org.apache.brooklyn.camp.brooklyn.spi.creation;
 
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.io.StringReader;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Set;
 
-import org.apache.brooklyn.api.catalog.CatalogItem;
 import org.apache.brooklyn.api.entity.Application;
-import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.entity.EntitySpec;
 import org.apache.brooklyn.api.mgmt.ManagementContext;
 import org.apache.brooklyn.camp.CampPlatform;
-import org.apache.brooklyn.camp.brooklyn.BrooklynCampConstants;
 import org.apache.brooklyn.camp.brooklyn.api.AssemblyTemplateSpecInstantiator;
 import org.apache.brooklyn.camp.spi.Assembly;
 import org.apache.brooklyn.camp.spi.AssemblyTemplate;
 import org.apache.brooklyn.camp.spi.AssemblyTemplate.Builder;
 import org.apache.brooklyn.camp.spi.PlatformComponentTemplate;
 import org.apache.brooklyn.camp.spi.collection.ResolvableLink;
-import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
-import org.apache.brooklyn.core.catalog.internal.BasicBrooklynCatalog.BrooklynLoaderTracker;
-import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
 import org.apache.brooklyn.core.entity.Entities;
 import org.apache.brooklyn.core.mgmt.EntityManagementUtils;
 import org.apache.brooklyn.core.mgmt.EntityManagementUtils.CreationResult;
@@ -49,10 +39,7 @@ import org.apache.brooklyn.core.mgmt.HasBrooklynManagementContext;
 import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
 import org.apache.brooklyn.core.mgmt.classloading.JavaBrooklynClassLoadingContext;
 import org.apache.brooklyn.entity.stock.BasicApplicationImpl;
-import org.apache.brooklyn.util.core.ResourceUtils;
 import org.apache.brooklyn.util.core.flags.TypeCoercions;
-import org.apache.brooklyn.util.exceptions.Exceptions;
-import org.apache.brooklyn.util.net.Urls;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -73,21 +60,28 @@ public class BrooklynAssemblyTemplateInstantiator implements AssemblyTemplateSpe
         return platform.assemblies().get(app.getApplicationId());
     }
 
-    public Application create(AssemblyTemplate template, CampPlatform platform) {
-        ManagementContext mgmt = getBrooklynManagementContext(platform);
+    private Application create(AssemblyTemplate template, CampPlatform platform) {
+        ManagementContext mgmt = getManagementContext(platform);
         BrooklynClassLoadingContext loader = JavaBrooklynClassLoadingContext.create(mgmt);
-        EntitySpec<? extends Application> spec = createSpec(template, platform, loader, true);
+        EntitySpec<? extends Application> spec = createApplicationSpec(template, platform, loader);
         Application instance = mgmt.getEntityManager().createEntity(spec);
         log.info("CAMP placing '{}' under management", instance);
         Entities.startManagement(instance, mgmt);
         return instance;
     }
-    
-    private ManagementContext getBrooklynManagementContext(CampPlatform platform) {
-        return ((HasBrooklynManagementContext)platform).getBrooklynManagementContext();
+
+    @Override
+    public List<EntitySpec<?>> createServiceSpecs(AssemblyTemplate template,
+            CampPlatform platform, BrooklynClassLoadingContext itemLoader,
+            Set<String> encounteredCatalogTypes) {
+        return buildTemplateServicesAsSpecs(itemLoader, template, platform, encounteredCatalogTypes, true);
     }
 
-    public EntitySpec<? extends Application> createSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader, boolean autoUnwrapIfPossible) {
+    @Override
+    public EntitySpec<? extends Application> createApplicationSpec(
+            AssemblyTemplate template,
+            CampPlatform platform,
+            BrooklynClassLoadingContext loader) {
         log.debug("CAMP creating application instance for {} ({})", template.getId(), template);
 
         // AssemblyTemplates created via PDP, _specifying_ then entities to put in
@@ -98,12 +92,12 @@ public class BrooklynAssemblyTemplateInstantiator implements AssemblyTemplateSpe
         app.configure(EntityManagementUtils.WRAPPER_APP_MARKER, Boolean.TRUE);
 
         // first build the children into an empty shell app
-        List<EntitySpec<?>> childSpecs = buildTemplateServicesAsSpecs(loader, template, platform, true);
+        List<EntitySpec<?>> childSpecs = createServiceSpecs(template, platform, loader, Sets.<String>newLinkedHashSet());
         for (EntitySpec<?> childSpec : childSpecs) {
             app.child(childSpec);
         }
 
-        if (autoUnwrapIfPossible && shouldUnwrap(template, app)) {
+        if (shouldUnwrap(template, app)) {
             app = EntityManagementUtils.unwrapApplication(app);
         }
 
@@ -124,149 +118,24 @@ public class BrooklynAssemblyTemplateInstantiator implements AssemblyTemplateSpe
         return wrapTemplate;
     }
 
-    protected boolean shouldUnwrap(AssemblyTemplate template, EntitySpec<? extends Application> app) {
+    private boolean shouldUnwrap(AssemblyTemplate template, EntitySpec<? extends Application> app) {
         if (Boolean.TRUE.equals(TypeCoercions.coerce(template.getCustomAttributes().get(NEVER_UNWRAP_APPS_PROPERTY), Boolean.class)))
             return false;
         return EntityManagementUtils.canPromoteWrappedApplication(app);
     }
 
-    private List<EntitySpec<?>> buildTemplateServicesAsSpecs(BrooklynClassLoadingContext loader, AssemblyTemplate template, CampPlatform platform, boolean canUseOtherTransformers) {
-        return buildTemplateServicesAsSpecsImpl(loader, template, platform, Sets.<String>newLinkedHashSet(), canUseOtherTransformers);
-    }
-
-    private List<EntitySpec<?>> buildTemplateServicesAsSpecsImpl(BrooklynClassLoadingContext loader, AssemblyTemplate template, CampPlatform platform, Set<String> encounteredCatalogTypes, boolean canUseOtherTransformers) {
+    private List<EntitySpec<?>> buildTemplateServicesAsSpecs(BrooklynClassLoadingContext loader, AssemblyTemplate template, CampPlatform platform, Set<String> encounteredCatalogTypes, boolean canUseOtherTransformers) {
         List<EntitySpec<?>> result = Lists.newArrayList();
 
         for (ResolvableLink<PlatformComponentTemplate> ctl: template.getPlatformComponentTemplates().links()) {
             PlatformComponentTemplate appChildComponentTemplate = ctl.resolve();
             BrooklynComponentTemplateResolver entityResolver = BrooklynComponentTemplateResolver.Factory.newInstance(loader, appChildComponentTemplate);
-            EntitySpec<?> spec;
-            spec = resolveCampSpec(platform, ResourceUtils.create(this), entityResolver, encounteredCatalogTypes, canUseOtherTransformers);
+            EntitySpec<?> spec = entityResolver.resolveSpec(encounteredCatalogTypes, canUseOtherTransformers);
             result.add(spec);
         }
         return result;
     }
 
-    static EntitySpec<?> resolveCampSpec(
-            CampPlatform platform,
-            ResourceUtils ru,
-            BrooklynComponentTemplateResolver entityResolver,
-            Set<String> encounteredCatalogTypes,
-            boolean canUseOtherTransformers) {
-        
-        String brooklynType = entityResolver.getServiceTypeResolver().getBrooklynType(entityResolver.getDeclaredType());
-        CatalogItem<Entity, EntitySpec<?>> item = entityResolver.getServiceTypeResolver().getCatalogItem(entityResolver, entityResolver.getDeclaredType());
-
-        if (log.isTraceEnabled()) log.trace("Building CAMP template services: type="+brooklynType+"; item="+item+"; loader="+entityResolver.getLoader()+"; encounteredCatalogTypes="+encounteredCatalogTypes);
-
-        EntitySpec<?> spec = null;
-        String protocol = Urls.getProtocol(brooklynType);
-        if (protocol != null) {
-            if (BrooklynCampConstants.YAML_URL_PROTOCOL_WHITELIST.contains(protocol)) {
-                spec = tryResolveYamlUrlReferenceSpec(platform, ru, brooklynType, entityResolver.getLoader(), encounteredCatalogTypes);
-                if (spec != null) {
-                    entityResolver.populateSpec(spec);
-                }
-            } else {
-                // TODO support https above
-                // TODO this will probably be logged if we refer to  chef:cookbook  or other service types which BCTR accepts;
-                // better would be to have BCTR supporting the calls above
-                log.debug("The reference " + brooklynType + " looks like a URL (running the CAMP Brooklyn assembly-template instantiator) but the protocol " +
-                        protocol + " isn't white listed (" + BrooklynCampConstants.YAML_URL_PROTOCOL_WHITELIST + "). " +
-                        "Will try to load it as catalog item or java type.");
-            }
-        }
-
-        if (spec == null) {
-            // load from java or yaml
-            spec = entityResolver.resolveSpec(encounteredCatalogTypes, canUseOtherTransformers);
-        }
-
-        return spec;
-    }
-
-    private static EntitySpec<?> tryResolveYamlUrlReferenceSpec(
-            CampPlatform platform,
-            ResourceUtils ru,
-            String brooklynType, BrooklynClassLoadingContext itemLoader,
-            Set<String> encounteredCatalogTypes) {
-        Reader yaml;
-        try {
-            yaml = new InputStreamReader(ru.getResourceFromUrl(brooklynType), "UTF-8");
-        } catch (Exception e) {
-            log.warn("AssemblyTemplate type " + brooklynType + " which looks like a URL can't be fetched.", e);
-            return null;
-        }
-        try {
-            return createNestedSpec(platform, encounteredCatalogTypes, yaml, itemLoader);
-        } finally {
-            try {
-                yaml.close();
-            } catch (IOException e) {
-                throw Exceptions.propagate(e);
-            }
-        }
-    }
-
-    static EntitySpec<?> resolveCatalogYamlReferenceCampSpec(
-            CampPlatform platform,
-            CatalogItem<Entity, EntitySpec<?>> item,
-            Set<String> encounteredCatalogTypes) {
-        ManagementContext mgmt = getManagementContext(platform);
-
-        String yaml = item.getPlanYaml();
-        Reader input = new StringReader(yaml);
-        BrooklynClassLoadingContext itemLoader = CatalogUtils.newClassLoadingContext(mgmt, item);
-
-        return createNestedSpec(platform, encounteredCatalogTypes, input, itemLoader);
-    }
-
-    private static EntitySpec<?> createNestedSpec(CampPlatform platform,
-            Set<String> encounteredCatalogTypes, Reader input,
-            BrooklynClassLoadingContext itemLoader) {
-
-        AssemblyTemplate at;
-        BrooklynLoaderTracker.setLoader(itemLoader);
-        try {
-            at = platform.pdp().registerDeploymentPlan(input);
-        } finally {
-            BrooklynLoaderTracker.unsetLoader(itemLoader);
-        }
-        return createNestedSpecStatic(at, platform, itemLoader, encounteredCatalogTypes);
-    }
-
-    @Override
-    public EntitySpec<?> createNestedSpec(
-            AssemblyTemplate template,
-            CampPlatform platform,
-            BrooklynClassLoadingContext itemLoader,
-            Set<String> encounteredCatalogTypes) {
-        return createNestedSpecStatic(template, platform, itemLoader, encounteredCatalogTypes);
-    }
-    
-    private static EntitySpec<?> createNestedSpecStatic(
-        AssemblyTemplate template,
-        CampPlatform platform,
-        BrooklynClassLoadingContext itemLoader,
-        Set<String> encounteredCatalogTypes) {
-        // In case we want to allow multiple top-level entities in a catalog we need to think
-        // about what it would mean to subsequently call buildChildrenEntitySpecs on the list of top-level entities!
-        try {
-            AssemblyTemplateInstantiator ati = template.getInstantiator().newInstance();
-            if (ati instanceof BrooklynAssemblyTemplateInstantiator) {
-                List<EntitySpec<?>> specs = ((BrooklynAssemblyTemplateInstantiator)ati).buildTemplateServicesAsSpecsImpl(itemLoader, template, platform, encounteredCatalogTypes, false);
-                if (specs.size() > 1) {
-                    throw new UnsupportedOperationException("Only supporting single service in catalog item currently: got "+specs);
-                }
-                return specs.get(0);
-            } else {
-                throw new IllegalStateException("Cannot create application with instantiator: " + ati);
-            }
-        } catch (Exception e) {
-            throw Exceptions.propagate(e);
-        }
-    }
-
     private static ManagementContext getManagementContext(CampPlatform platform) {
         return ((HasBrooklynManagementContext)platform).getBrooklynManagementContext();
     }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynComponentTemplateResolver.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynComponentTemplateResolver.java b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynComponentTemplateResolver.java
index 5857d61..914b5e6 100644
--- a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynComponentTemplateResolver.java
+++ b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynComponentTemplateResolver.java
@@ -32,12 +32,12 @@ import org.apache.brooklyn.api.catalog.CatalogItem;
 import org.apache.brooklyn.api.entity.Application;
 import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.entity.EntitySpec;
+import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
 import org.apache.brooklyn.api.location.Location;
 import org.apache.brooklyn.api.mgmt.ManagementContext;
-import org.apache.brooklyn.camp.CampPlatform;
 import org.apache.brooklyn.camp.brooklyn.BrooklynCampConstants;
 import org.apache.brooklyn.camp.brooklyn.BrooklynCampReservedKeys;
-import org.apache.brooklyn.camp.brooklyn.spi.creation.service.BrooklynServiceTypeResolver;
+import org.apache.brooklyn.camp.brooklyn.spi.creation.service.DefaultServiceTypeResolver;
 import org.apache.brooklyn.camp.brooklyn.spi.creation.service.ServiceTypeResolver;
 import org.apache.brooklyn.camp.spi.AbstractResource;
 import org.apache.brooklyn.camp.spi.ApplicationComponentTemplate;
@@ -46,15 +46,12 @@ import org.apache.brooklyn.camp.spi.PlatformComponentTemplate;
 import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
 import org.apache.brooklyn.core.config.ConfigKeys;
-import org.apache.brooklyn.core.entity.AbstractEntity;
 import org.apache.brooklyn.core.mgmt.BrooklynTags;
 import org.apache.brooklyn.core.mgmt.BrooklynTaskTags;
 import org.apache.brooklyn.core.mgmt.EntityManagementUtils;
 import org.apache.brooklyn.core.mgmt.ManagementContextInjectable;
 import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
 import org.apache.brooklyn.core.mgmt.classloading.JavaBrooklynClassLoadingContext;
-import org.apache.brooklyn.core.mgmt.internal.ManagementContextInternal;
-import org.apache.brooklyn.core.objs.proxy.InternalEntityFactory;
 import org.apache.brooklyn.util.collections.MutableList;
 import org.apache.brooklyn.util.collections.MutableSet;
 import org.apache.brooklyn.util.core.ResourceUtils;
@@ -62,9 +59,9 @@ import org.apache.brooklyn.util.core.config.ConfigBag;
 import org.apache.brooklyn.util.core.flags.FlagUtils;
 import org.apache.brooklyn.util.core.flags.FlagUtils.FlagConfigKeyAndValueRecord;
 import org.apache.brooklyn.util.core.task.Tasks;
-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.net.Urls;
 import org.apache.brooklyn.util.text.Strings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -91,7 +88,7 @@ public class BrooklynComponentTemplateResolver {
     private final ServiceTypeResolver typeResolver;
     private final AtomicBoolean alreadyBuilt = new AtomicBoolean(false);
 
-    public BrooklynComponentTemplateResolver(BrooklynClassLoadingContext loader, ConfigBag attrs, AbstractResource optionalTemplate, String type, ServiceTypeResolver typeResolver) {
+    private BrooklynComponentTemplateResolver(BrooklynClassLoadingContext loader, ConfigBag attrs, AbstractResource optionalTemplate, String type, ServiceTypeResolver typeResolver) {
         this.loader = loader;
         this.mgmt = loader.getManagementContext();
         this.attrs = ConfigBag.newInstanceCopying(attrs);
@@ -101,14 +98,10 @@ public class BrooklynComponentTemplateResolver {
         this.typeResolver = typeResolver;
     }
 
-    public BrooklynClassLoadingContext getLoader() { return loader; }
     public ManagementContext getManagementContext() { return mgmt; }
     public ConfigBag getAttrs() { return attrs; }
-    public Maybe<AbstractResource> getTemplate() { return template; }
     public BrooklynYamlTypeInstantiator.Factory getYamlLoader() { return yamlLoader; }
-    public ServiceTypeResolver getServiceTypeResolver() { return typeResolver; }
     public String getDeclaredType() { return type; }
-    public Boolean isAlreadyBuilt() { return alreadyBuilt.get(); }
 
     public static class Factory {
 
@@ -119,7 +112,7 @@ public class BrooklynComponentTemplateResolver {
         }
 
         // TODO This could be extended to support multiple prefixes per resolver and a 'best-match' algorithm
-        protected static ServiceTypeResolver findService(BrooklynClassLoadingContext context, String type) {
+        private static ServiceTypeResolver findService(BrooklynClassLoadingContext context, String type) {
             if (type.indexOf(':') != -1) {
                 String prefix = Splitter.on(":").splitToList(type).get(0);
                 ServiceLoader<ServiceTypeResolver> loader = ServiceLoader.load(ServiceTypeResolver.class,
@@ -149,11 +142,11 @@ public class BrooklynComponentTemplateResolver {
             ServiceTypeResolver typeResolver = computeResolverType(context, null, optionalTemplate, attrs);
             String type = getDeclaredType(null, optionalTemplate, attrs);
             if (typeResolver == null) // use default
-                typeResolver = new BrooklynServiceTypeResolver();
+                typeResolver = new DefaultServiceTypeResolver();
             return new BrooklynComponentTemplateResolver(context, attrs, optionalTemplate, type, typeResolver);
         }
 
-        public static String getDeclaredType(String knownServiceType, AbstractResource optionalTemplate, @Nullable ConfigBag attrs) {
+        private static String getDeclaredType(String knownServiceType, AbstractResource optionalTemplate, @Nullable ConfigBag attrs) {
             String type = knownServiceType;
             if (type==null && optionalTemplate!=null) {
                 type = optionalTemplate.getType();
@@ -168,31 +161,44 @@ public class BrooklynComponentTemplateResolver {
         private static String extractServiceTypeAttribute(@Nullable ConfigBag attrs) {
             return BrooklynYamlTypeInstantiator.InstantiatorFromKey.extractTypeName("service", attrs).orNull();
         }
+    }
 
-        public static boolean supportsType(BrooklynClassLoadingContext context, String serviceType) {
-            ServiceTypeResolver typeResolver = computeResolverType(context, serviceType, null, null);
-            if (typeResolver != null) return true;
-            return newInstance(context, serviceType).canResolve();
+    public boolean canResolve() {
+        if (!(typeResolver instanceof DefaultServiceTypeResolver)) {
+            return true;
         }
-    }
 
-    protected boolean canResolve() {
         CatalogItem<Entity, EntitySpec<?>> item = typeResolver.getCatalogItem(this, type);
-        if (item == null) {
-            return loader.tryLoadClass(getJavaType(), Entity.class).isPresent();
+        if (item != null) {
+            if (item.isDisabled()) {
+                log.warn("Disallowed attempt to use disabled catalog item "+item.getId());
+                return false;
+            } else if (item.isDeprecated()) {
+                log.warn("Use of deprecated catalog item "+item.getId());
+            }
+            return true;
         }
-        
-        if (item.isDisabled()) {
-            log.warn("Disallowed attempt to use disabled catalog item "+item.getId());
-            return false;
-        } else if (item.isDeprecated()) {
-            log.warn("Use of deprecated catalog item "+item.getId());
+
+        if (tryLoadEntityClass().isPresent()) {
+            return true;
         }
+
+        String protocol = Urls.getProtocol(type);
+        if (protocol != null) {
+            if (BrooklynCampConstants.YAML_URL_PROTOCOL_WHITELIST.contains(protocol)) {
+                return true;
+            } else {
+                log.debug("The reference '" + type + "' looks like a URL (running the CAMP Brooklyn entity-matcher) but the protocol '" + 
+                        protocol + "' isn't white listed " + BrooklynCampConstants.YAML_URL_PROTOCOL_WHITELIST + ". " +
+                        "Not recognized as catalog item or java item as well!");
+            }
+        }
+
         return true;
     }
 
     /** returns the entity class, if needed in contexts which scan its statics for example */
-    protected Class<? extends Entity> loadEntityClass() {
+    private Class<? extends Entity> loadEntityClass() {
         Maybe<Class<? extends Entity>> result = tryLoadEntityClass();
         if (result.isAbsent())
             throw new IllegalStateException("Could not find "+typeResolver.getBrooklynType(type), ((Maybe.Absent<?>)result).getException());
@@ -200,16 +206,16 @@ public class BrooklynComponentTemplateResolver {
     }
 
     /** tries to load the Java entity class */
-    protected Maybe<Class<? extends Entity>> tryLoadEntityClass() {
+    private Maybe<Class<? extends Entity>> tryLoadEntityClass() {
         return loader.tryLoadClass(getJavaType(), Entity.class);
     }
 
     // TODO Generalise to have other prefixes (e.g. explicit "catalog:" etc)?
-    protected boolean isJavaTypePrefix() {
+    private boolean isJavaTypePrefix() {
         return type != null && (type.toLowerCase().startsWith("java:") || type.toLowerCase().startsWith("brooklyn:java:"));
     }
 
-    protected String getJavaType() {
+    private String getJavaType() {
         CatalogItem<Entity, EntitySpec<?>> item = typeResolver.getCatalogItem(this, type);
         if (!isJavaTypePrefix() && item != null && item.getJavaType() != null) {
             return item.getJavaType();
@@ -218,33 +224,67 @@ public class BrooklynComponentTemplateResolver {
         }
     }
 
-    /** resolves the spec, updating the loader if a catalog item is loaded */
-    @SuppressWarnings("unchecked")
-    protected <T extends Entity> EntitySpec<T> resolveSpec(Set<String> encounteredCatalogTypes, boolean canUseOtherTransformers) {
+    public <T extends Entity> EntitySpec<T> resolveSpec(Set<String> encounteredCatalogTypes, boolean canUseOtherTransformers) {
         if (alreadyBuilt.getAndSet(true))
             throw new IllegalStateException("Spec can only be used once: "+this);
 
-        CatalogItem<Entity, EntitySpec<?>> item = getServiceTypeResolver().getCatalogItem(this, getDeclaredType());
-        try {
-            EntitySpec<T> spec = createSpec(item, encounteredCatalogTypes);
-            populateSpec(spec);
-            return spec;
-        } catch (Exception e) {
-            Exceptions.propagateIfFatal(e);
-            
-            // maybe it requires a different transformer? TODO better would be to know this from a transformer type on the catalog item type
-            if (item!=null && canUseOtherTransformers) {
-                log.debug("Could not resolve child "+item+" as CAMP; trying other transformers: "+e);
-                // should only be called when resolving children. and encountered types should probably be empty. 
-                return (EntitySpec<T>) EntityManagementUtils.createCatalogSpec(mgmt, (CatalogItem) item);
+        String brooklynType = typeResolver.getBrooklynType(type);
+        CatalogItem<Entity, EntitySpec<?>> item = typeResolver.getCatalogItem(this, type);
+
+        if (log.isTraceEnabled()) log.trace("Building CAMP template services: type="+brooklynType+"; item="+item+"; loader="+loader+"; encounteredCatalogTypes="+encounteredCatalogTypes);
+
+        // TODO implement as service type
+        EntitySpec<T> spec = null;
+        String protocol = Urls.getProtocol(brooklynType);
+        if (protocol != null) {
+            if (BrooklynCampConstants.YAML_URL_PROTOCOL_WHITELIST.contains(protocol)) {
+                spec = tryResolveYamlUrlReferenceSpec(brooklynType, encounteredCatalogTypes);
+                if (spec != null) {
+                    populateSpec(spec);
+                }
             } else {
-                throw Exceptions.propagate(e);
+                // TODO this will probably be logged if we refer to  chef:cookbook  or other service types which BCTR accepts;
+                // better would be to have BCTR supporting the calls above
+                log.debug("The reference " + brooklynType + " looks like a URL (running the CAMP Brooklyn assembly-template instantiator) but the protocol " +
+                        protocol + " isn't white listed (" + BrooklynCampConstants.YAML_URL_PROTOCOL_WHITELIST + "). " +
+                        "Will try to load it as catalog item or java type.");
             }
         }
+
+        if (spec == null) {
+            // load from java or yaml
+            spec = resolveLocalSpec(encounteredCatalogTypes, canUseOtherTransformers);
+        }
+
+        return spec;
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T extends Entity> EntitySpec<T> tryResolveYamlUrlReferenceSpec(String brooklynType, Set<String> encounteredCatalogTypes) {
+        String yaml;
+        try {
+            yaml = ResourceUtils.create(this).getResourceAsString(brooklynType);
+        } catch (Exception e) {
+            log.warn("AssemblyTemplate type " + brooklynType + " which looks like a URL can't be fetched.", e);
+            return null;
+        }
+        // Referenced specs are expected to be CAMP format as well.
+        List<EntitySpec<?>> serviceSpecs = CampUtils.createServiceSpecs(yaml, loader, encounteredCatalogTypes);
+        if (serviceSpecs.size() > 1) {
+            throw new UnsupportedOperationException("Only supporting single service in remotely referenced plans: got "+serviceSpecs);
+        }
+        return (EntitySpec<T>) serviceSpecs.get(0);
+    }
+
+    private <T extends Entity> EntitySpec<T> resolveLocalSpec(Set<String> encounteredCatalogTypes, boolean canUseOtherTransformers) {
+        CatalogItem<Entity, EntitySpec<?>> item = typeResolver.getCatalogItem(this, type);
+        EntitySpec<T> spec = createSpec(item, encounteredCatalogTypes);
+        populateSpec(spec);
+        return spec;
     }
 
     @SuppressWarnings({ "unchecked" })
-    protected <T extends Entity> EntitySpec<T> createSpec(CatalogItem<Entity, EntitySpec<?>> item, Set<String> encounteredCatalogTypes) {
+    private <T extends Entity,SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> EntitySpec<T> createSpec(CatalogItem<Entity, EntitySpec<?>> item, Set<String> encounteredCatalogTypes) {
         if (item == null) {
             // ignore; presumably a java type or some such?
         } else if (item.isDisabled()) {
@@ -280,17 +320,12 @@ public class BrooklynComponentTemplateResolver {
         // Only case that's left is a catalog item with CAMP YAML content - try to parse it recursively
         // including its OSGi bundles in the loader classpath.
         } else {
-            // TODO if it isn't camp it will throw. better would be to know the transformers type for the item!
-
-            EntitySpec<?> spec = BrooklynAssemblyTemplateInstantiator.resolveCatalogYamlReferenceCampSpec(getCampPlatform(), item, encounteredCatalogTypes);
-            spec.catalogItemId(item.getId());
-            
-            return (EntitySpec<T>)spec;
+            return (EntitySpec<T>) EntityManagementUtils.createCatalogSpec(mgmt, (CatalogItem<T,SpecT>)item, encounteredCatalogTypes);
         }
     }
     
     @SuppressWarnings("unchecked")
-    protected <T extends Entity> EntitySpec<T> createSpecFromJavaType() {
+    private <T extends Entity> EntitySpec<T> createSpecFromJavaType() {
         Class<T> type = (Class<T>) loadEntityClass();
         
         EntitySpec<T> spec;
@@ -311,9 +346,8 @@ public class BrooklynComponentTemplateResolver {
         return spec;
     }
 
-    //called from BrooklynAssemblyTemplateInstantiator as well
     @SuppressWarnings("unchecked")
-    protected <T extends Entity> void populateSpec(EntitySpec<T> spec) {
+    private <T extends Entity> void populateSpec(EntitySpec<T> spec) {
         String name, templateId=null, planId=null;
         if (template.isPresent()) {
             name = template.get().getName();
@@ -335,7 +369,7 @@ public class BrooklynComponentTemplateResolver {
             Iterable<Map<String,?>> children = (Iterable<Map<String,?>>)childrenObj;
             for (Map<String,?> childAttrs : children) {
                 BrooklynComponentTemplateResolver entityResolver = BrooklynComponentTemplateResolver.Factory.newInstance(loader, childAttrs);
-                EntitySpec<? extends Entity> childSpec = BrooklynAssemblyTemplateInstantiator.resolveCampSpec(getCampPlatform(), ResourceUtils.create(this), entityResolver, encounteredCatalogTypes, true);
+                EntitySpec<? extends Entity> childSpec = entityResolver.resolveSpec(encounteredCatalogTypes, true);
                 spec.child(childSpec);
             }
         }
@@ -354,31 +388,8 @@ public class BrooklynComponentTemplateResolver {
         configureEntityConfig(spec);
     }
 
-    /** returns new *uninitialised* entity, with just a few of the pieces from the spec;
-     * initialisation occurs soon after, in {@link #initEntity(ManagementContext, Entity, EntitySpec)},
-     * inside an execution context and after entity ID's are recognised
-     */
-    protected <T extends Entity> T newEntity(EntitySpec<T> spec) {
-        Class<? extends T> entityImpl = (spec.getImplementation() != null) ? spec.getImplementation() : mgmt.getEntityManager().getEntityTypeRegistry().getImplementedBy(spec.getType());
-        InternalEntityFactory entityFactory = ((ManagementContextInternal)mgmt).getEntityFactory();
-        T entity = entityFactory.constructEntity(entityImpl, spec);
-
-        String planId = (String)spec.getConfig().get(BrooklynCampConstants.PLAN_ID);
-        if (planId != null) {
-            entity.config().set(BrooklynCampConstants.PLAN_ID, planId);
-        }
-
-        if (spec.getLocations().size() > 0) {
-            ((AbstractEntity)entity).addLocations(spec.getLocations());
-        }
-
-        if (spec.getParent() != null) entity.setParent(spec.getParent());
-
-        return entity;
-    }
-
     @SuppressWarnings({ "unchecked", "rawtypes" })
-    protected void configureEntityConfig(EntitySpec<?> spec) {
+    private void configureEntityConfig(EntitySpec<?> spec) {
         // first take *recognised* flags and config keys from the top-level, and put them in the bag (of brooklyn.config)
         // attrs will contain only brooklyn.xxx properties when coming from BrooklynEntityMatcher.
         // Any top-level flags will go into "brooklyn.flags". When resolving a spec from $brooklyn:entitySpec
@@ -442,7 +453,7 @@ public class BrooklynComponentTemplateResolver {
         return allKeys;
     }
 
-    protected static class SpecialFlagsTransformer implements Function<Object, Object> {
+    private static class SpecialFlagsTransformer implements Function<Object, Object> {
         protected final ManagementContext mgmt;
         /* TODO find a way to make do without loader here?
          * it is not very nice having to serialize it; but serialization of BLCL is now relatively clean.
@@ -458,6 +469,7 @@ public class BrooklynComponentTemplateResolver {
             this.loader = loader;
             mgmt = loader.getManagementContext();
         }
+        @Override
         public Object apply(Object input) {
             if (input instanceof Map)
                 return transformSpecialFlags((Map<?, ?>)input);
@@ -501,7 +513,7 @@ public class BrooklynComponentTemplateResolver {
                 @SuppressWarnings("unchecked")
                 Map<String, Object> resolvedConfig = (Map<String, Object>)transformSpecialFlags(specConfig.getSpecConfiguration());
                 specConfig.setSpecConfiguration(resolvedConfig);
-                return Factory.newInstance(getLoader(), specConfig.getSpecConfiguration()).resolveSpec(null, false);
+                return Factory.newInstance(getLoader(), specConfig.getSpecConfiguration()).resolveLocalSpec(null, false);
             }
             if (flag instanceof ManagementContextInjectable) {
                 log.debug("Injecting Brooklyn management context info object: {}", flag);
@@ -511,14 +523,4 @@ public class BrooklynComponentTemplateResolver {
             return flag;
         }
     }
-
-    @SuppressWarnings("unchecked")
-    protected List<Map<String, Object>> getChildren(Map<String, Object> attrs) {
-        if (attrs==null) return null;
-        return (List<Map<String, Object>>) attrs.get(BrooklynCampReservedKeys.BROOKLYN_CHILDREN);
-    }
-
-    private CampPlatform getCampPlatform() {
-        return CampCatalogUtils.getCampPlatform(mgmt);
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynEntityMatcher.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynEntityMatcher.java b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynEntityMatcher.java
index c0a2e19..96e8c4e 100644
--- a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynEntityMatcher.java
+++ b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/BrooklynEntityMatcher.java
@@ -22,7 +22,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.brooklyn.api.mgmt.ManagementContext;
-import org.apache.brooklyn.camp.brooklyn.BrooklynCampConstants;
 import org.apache.brooklyn.camp.brooklyn.BrooklynCampReservedKeys;
 import org.apache.brooklyn.camp.spi.PlatformComponentTemplate;
 import org.apache.brooklyn.camp.spi.PlatformComponentTemplate.Builder;
@@ -33,7 +32,6 @@ import org.apache.brooklyn.core.catalog.internal.BasicBrooklynCatalog;
 import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
 import org.apache.brooklyn.core.mgmt.classloading.JavaBrooklynClassLoadingContext;
 import org.apache.brooklyn.util.collections.MutableMap;
-import org.apache.brooklyn.util.net.Urls;
 import org.apache.brooklyn.util.text.Strings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -67,19 +65,8 @@ public class BrooklynEntityMatcher implements PdpMatcher {
             if (serviceType==null) throw new NullPointerException("Service must declare a type ("+service+")");
             BrooklynClassLoadingContext loader = BasicBrooklynCatalog.BrooklynLoaderTracker.getLoader();
             if (loader == null) loader = JavaBrooklynClassLoadingContext.create(mgmt);
-            if (BrooklynComponentTemplateResolver.Factory.supportsType(loader, serviceType))
+            if (BrooklynComponentTemplateResolver.Factory.newInstance(loader, serviceType).canResolve())
                 return serviceType;
-
-            String protocol = Urls.getProtocol(serviceType);
-            if (protocol != null) {
-                if (BrooklynCampConstants.YAML_URL_PROTOCOL_WHITELIST.contains(protocol)) {
-                    return serviceType;
-                } else {
-                    log.debug("The reference '" + serviceType + "' looks like a URL (running the CAMP Brooklyn entity-matcher) but the protocol '" + 
-                            protocol + "' isn't white listed " + BrooklynCampConstants.YAML_URL_PROTOCOL_WHITELIST + ". " +
-                            "Not recognized as catalog item or java item as well!");
-                }
-            }
         }
         return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampCatalogUtils.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampCatalogUtils.java b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampCatalogUtils.java
index 3d64897..4865241 100644
--- a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampCatalogUtils.java
+++ b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampCatalogUtils.java
@@ -18,67 +18,50 @@
  */
 package org.apache.brooklyn.camp.brooklyn.spi.creation;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Map;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.brooklyn.api.catalog.CatalogItem;
-import org.apache.brooklyn.api.catalog.CatalogItem.CatalogItemType;
 import org.apache.brooklyn.api.entity.EntitySpec;
 import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
-import org.apache.brooklyn.api.location.Location;
-import org.apache.brooklyn.api.location.LocationSpec;
 import org.apache.brooklyn.api.mgmt.ManagementContext;
-import org.apache.brooklyn.api.policy.Policy;
-import org.apache.brooklyn.api.policy.PolicySpec;
 import org.apache.brooklyn.camp.CampPlatform;
-import org.apache.brooklyn.camp.brooklyn.BrooklynCampConstants;
-import org.apache.brooklyn.camp.brooklyn.api.AssemblyTemplateSpecInstantiator;
-import org.apache.brooklyn.camp.spi.AssemblyTemplate;
-import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
-import org.apache.brooklyn.camp.spi.pdp.DeploymentPlan;
-import org.apache.brooklyn.core.catalog.internal.BasicBrooklynCatalog;
-import org.apache.brooklyn.core.catalog.internal.BasicBrooklynCatalog.BrooklynLoaderTracker;
 import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
 import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
-import org.apache.brooklyn.core.objs.BrooklynObjectInternal.ConfigurationSupportInternal;
-import org.apache.brooklyn.util.collections.MutableSet;
-import org.apache.brooklyn.util.exceptions.Exceptions;
-import org.apache.brooklyn.util.guava.Maybe;
-import org.apache.brooklyn.util.stream.Streams;
 import org.apache.brooklyn.util.text.Strings;
-import org.apache.brooklyn.util.yaml.Yamls;
 
 import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
 
 public class CampCatalogUtils {
 
-    public static AbstractBrooklynObjectSpec<?, ?> createSpec(ManagementContext mgmt, CatalogItem<?, ?> item) {
+    public static AbstractBrooklynObjectSpec<?, ?> createSpec(ManagementContext mgmt, CatalogItem<?, ?> item, Set<String> encounteredTypes) {
         // preferred way is to parse the yaml, to resolve references late;
         // the parsing on load is to populate some fields, but it is optional.
         // TODO messy for location and policy that we need brooklyn.{locations,policies} root of the yaml, but it works;
         // see related comment when the yaml is set, in addAbstractCatalogItems
         // (not sure if anywhere else relies on that syntax; if not, it should be easy to fix!)
         BrooklynClassLoadingContext loader = CatalogUtils.newClassLoadingContext(mgmt, item);
-        DeploymentPlan plan = makePlanFromYaml(loader.getManagementContext(), item.getPlanYaml());
-        Preconditions.checkNotNull(item.getCatalogItemType(), "catalog item type for "+plan);
+        Preconditions.checkNotNull(item.getCatalogItemType(), "catalog item type for "+item.getPlanYaml());
+
+        // symbolicName could be null if coming from the catalog parser where it tries to load before knowing the id
+        if (item.getSymbolicName() != null) {
+            encounteredTypes.add(item.getSymbolicName());
+        }
+
         AbstractBrooklynObjectSpec<?, ?> spec;
         switch (item.getCatalogItemType()) {
             case TEMPLATE:
             case ENTITY:
-                spec = createEntitySpec(item.getSymbolicName(), plan, loader);
+                spec = createEntitySpec(item.getPlanYaml(), loader, encounteredTypes);
                 break;
             case LOCATION: 
-                spec = createLocationSpec(plan, loader);
+                spec = CampUtils.createLocationSpec(item.getPlanYaml(), loader, encounteredTypes);
                 break;
             case POLICY: 
-                spec = createPolicySpec(item.getSymbolicName(), plan, loader);
+                spec = CampUtils.createPolicySpec(item.getPlanYaml(), loader, encounteredTypes);
                 break;
             default:
-                throw new IllegalStateException("Unknown CI Type "+item.getCatalogItemType()+" for "+plan);
+                throw new IllegalStateException("Unknown CI Type "+item.getCatalogItemType()+" for "+item.getPlanYaml());
         }
 
         ((AbstractBrooklynObjectSpec<?, ?>)spec).catalogItemId(item.getId());
@@ -88,157 +71,17 @@ public class CampCatalogUtils {
 
         return spec;
     }
-
-    private static EntitySpec<?> createEntitySpec(String symbolicName, DeploymentPlan plan, BrooklynClassLoadingContext loader) {
-        CampPlatform camp = getCampPlatform(loader.getManagementContext());
-
-        // TODO should not register new AT each time we instantiate from the same plan; use some kind of cache
-        AssemblyTemplate at;
-        BrooklynLoaderTracker.setLoader(loader);
-        try {
-            at = camp.pdp().registerDeploymentPlan(plan);
-        } finally {
-            BrooklynLoaderTracker.unsetLoader(loader);
-        }
-
-        try {
-            AssemblyTemplateInstantiator instantiator = at.getInstantiator().newInstance();
-            if (instantiator instanceof AssemblyTemplateSpecInstantiator) {
-                return ((AssemblyTemplateSpecInstantiator)instantiator).createNestedSpec(at, camp, loader, 
-                    getInitialEncounteredSymbol(symbolicName));
-            }
-            throw new IllegalStateException("Unable to instantiate YAML; incompatible instantiator "+instantiator+" for "+at);
-        } catch (Exception e) {
-            throw Exceptions.propagate(e);
+    
+    private static EntitySpec<?> createEntitySpec(String plan, BrooklynClassLoadingContext loader, Set<String> encounteredTypes) {
+        List<EntitySpec<?>> serviceEntitySpecs = CampUtils.createServiceSpecs(plan, loader, encounteredTypes);
+        if (serviceEntitySpecs.size() > 1) {
+            throw new UnsupportedOperationException("Only supporting single service in catalog item currently: got "+serviceEntitySpecs);
         }
+        return serviceEntitySpecs.get(0);
     }
 
-    private static MutableSet<String> getInitialEncounteredSymbol(String symbolicName) {
-        return symbolicName==null ? MutableSet.<String>of() : MutableSet.of(symbolicName);
-    }
-
-    private static PolicySpec<?> createPolicySpec(String symbolicName, DeploymentPlan plan, BrooklynClassLoadingContext loader) {
-        return createPolicySpec(plan, loader, getInitialEncounteredSymbol(symbolicName));
-    }
-
-    private static PolicySpec<?> createPolicySpec(DeploymentPlan plan, BrooklynClassLoadingContext loader, Set<String> encounteredCatalogTypes) {
-        //Would ideally re-use the PolicySpecResolver
-        //but it is CAMP specific and there is no easy way to get hold of it.
-        Object policies = checkNotNull(plan.getCustomAttributes().get(BasicBrooklynCatalog.POLICIES_KEY), "policy config");
-        if (!(policies instanceof Iterable<?>)) {
-            throw new IllegalStateException("The value of " + BasicBrooklynCatalog.POLICIES_KEY + " must be an Iterable.");
-        }
-
-        Object policy = Iterables.getOnlyElement((Iterable<?>)policies);
-
-        return createPolicySpec(loader, policy, encounteredCatalogTypes);
-    }
-
-    @SuppressWarnings("unchecked")
-    private static PolicySpec<?> createPolicySpec(BrooklynClassLoadingContext loader, Object policy, Set<String> encounteredCatalogTypes) {
-        Map<String, Object> itemMap;
-        if (policy instanceof String) {
-            itemMap = ImmutableMap.<String, Object>of("type", policy);
-        } else if (policy instanceof Map) {
-            itemMap = (Map<String, Object>) policy;
-        } else {
-            throw new IllegalStateException("Policy expected to be string or map. Unsupported object type " + policy.getClass().getName() + " (" + policy.toString() + ")");
-        }
-
-        String versionedId = (String) checkNotNull(Yamls.getMultinameAttribute(itemMap, "policy_type", "policyType", "type"), "policy type");
-        PolicySpec<? extends Policy> spec;
-        CatalogItem<?, ?> policyItem = CatalogUtils.getCatalogItemOptionalVersion(loader.getManagementContext(), versionedId);
-        if (policyItem != null && !encounteredCatalogTypes.contains(policyItem.getSymbolicName())) {
-            if (policyItem.getCatalogItemType() != CatalogItemType.POLICY) {
-                throw new IllegalStateException("Non-policy catalog item in policy context: " + policyItem);
-            }
-            //TODO re-use createSpec
-            BrooklynClassLoadingContext itemLoader = CatalogUtils.newClassLoadingContext(loader.getManagementContext(), policyItem);
-            if (policyItem.getPlanYaml() != null) {
-                DeploymentPlan plan = makePlanFromYaml(loader.getManagementContext(), policyItem.getPlanYaml());
-                encounteredCatalogTypes.add(policyItem.getSymbolicName());
-                return createPolicySpec(plan, itemLoader, encounteredCatalogTypes);
-            } else if (policyItem.getJavaType() != null) {
-                spec = PolicySpec.create((Class<Policy>)itemLoader.loadClass(policyItem.getJavaType()));
-            } else {
-                throw new IllegalStateException("Invalid policy item - neither yaml nor javaType: " + policyItem);
-            }
-        } else {
-            spec = PolicySpec.create(loader.loadClass(versionedId, Policy.class));
-        }
-        Map<String, Object> brooklynConfig = (Map<String, Object>) itemMap.get("brooklyn.config");
-        if (brooklynConfig != null) {
-            spec.configure(brooklynConfig);
-        }
-        return spec;
-    }
-
-    private static LocationSpec<?> createLocationSpec(DeploymentPlan plan, BrooklynClassLoadingContext loader) {
-        // See #createPolicySpec; this impl is modeled on that.
-        // spec.catalogItemId is set by caller
-        Object locations = checkNotNull(plan.getCustomAttributes().get(BasicBrooklynCatalog.LOCATIONS_KEY), "location config");
-        if (!(locations instanceof Iterable<?>)) {
-            throw new IllegalStateException("The value of " + BasicBrooklynCatalog.LOCATIONS_KEY + " must be an Iterable.");
-        }
-
-        Object location = Iterables.getOnlyElement((Iterable<?>)locations);
-
-        return createLocationSpec(loader, location); 
-    }
-
-    @SuppressWarnings("unchecked")
-    private static LocationSpec<?> createLocationSpec(BrooklynClassLoadingContext loader, Object location) {
-        Map<String, Object> itemMap;
-        if (location instanceof String) {
-            itemMap = ImmutableMap.<String, Object>of("type", location);
-        } else if (location instanceof Map) {
-            itemMap = (Map<String, Object>) location;
-        } else {
-            throw new IllegalStateException("Location expected to be string or map. Unsupported object type " + location.getClass().getName() + " (" + location.toString() + ")");
-        }
-
-        String type = (String) checkNotNull(Yamls.getMultinameAttribute(itemMap, "location_type", "locationType", "type"), "location type");
-        Map<String, Object> brooklynConfig = (Map<String, Object>) itemMap.get("brooklyn.config");
-        Maybe<Class<? extends Location>> javaClass = loader.tryLoadClass(type, Location.class);
-        if (javaClass.isPresent()) {
-            LocationSpec<?> spec = LocationSpec.create(javaClass.get());
-            if (brooklynConfig != null) {
-                spec.configure(brooklynConfig);
-            }
-            return spec;
-        } else {
-            Maybe<Location> loc = loader.getManagementContext().getLocationRegistry().resolve(type, false, brooklynConfig);
-            if (loc.isPresent()) {
-                // TODO extensions?
-                Map<String, Object> locConfig = ((ConfigurationSupportInternal)loc.get().config()).getBag().getAllConfig();
-                Class<? extends Location> locType = loc.get().getClass();
-                Set<Object> locTags = loc.get().tags().getTags();
-                String locDisplayName = loc.get().getDisplayName();
-                return LocationSpec.create(locType)
-                        .configure(locConfig)
-                        .displayName(locDisplayName)
-                        .tags(locTags);
-            } else {
-                throw new IllegalStateException("No class or resolver found for location type "+type);
-            }
-        }
-    }
-
-    private static DeploymentPlan makePlanFromYaml(ManagementContext mgmt, String yaml) {
-        CampPlatform camp = getCampPlatform(mgmt);
-        return camp.pdp().parseDeploymentPlan(Streams.newReaderWithContents(yaml));
-    }
-
-    /**
-     * @return the CAMP platform associated with a management context, if there is one.
-     */
     public static CampPlatform getCampPlatform(ManagementContext mgmt) {
-        CampPlatform result = mgmt.getConfig().getConfig(BrooklynCampConstants.CAMP_PLATFORM);
-        if (result!=null) {
-            return result;
-        } else {
-            throw new IllegalStateException("No CAMP Platform is registered with this Brooklyn management context.");
-        }
+        return CampUtils.getCampPlatform(mgmt);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampToSpecTransformer.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampToSpecTransformer.java b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampToSpecTransformer.java
index 9b9f1ec..dd0a6f4 100644
--- a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampToSpecTransformer.java
+++ b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampToSpecTransformer.java
@@ -19,6 +19,7 @@
 package org.apache.brooklyn.camp.brooklyn.spi.creation;
 
 import java.io.StringReader;
+import java.util.Set;
 
 import org.apache.brooklyn.api.catalog.CatalogItem;
 import org.apache.brooklyn.api.entity.Application;
@@ -68,7 +69,7 @@ public class CampToSpecTransformer implements PlanToSpecTransformer {
             }
             if (instantiator instanceof AssemblyTemplateSpecInstantiator) {
                 BrooklynClassLoadingContext loader = JavaBrooklynClassLoadingContext.create(mgmt);
-                return ((AssemblyTemplateSpecInstantiator) instantiator).createSpec(at, camp, loader, true);
+                return ((AssemblyTemplateSpecInstantiator) instantiator).createApplicationSpec(at, camp, loader);
             } else {
                 // The unknown instantiator can create the app (Assembly), but not a spec.
                 // Currently, all brooklyn plans should produce the above.
@@ -94,8 +95,9 @@ public class CampToSpecTransformer implements PlanToSpecTransformer {
 
     @SuppressWarnings({ "unchecked", "rawtypes" })
     @Override
-    public <T, SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> SpecT createCatalogSpec(CatalogItem<T, SpecT> item) {
-        return (SpecT) CampCatalogUtils.createSpec(mgmt, (CatalogItem)item);
+    public <T, SpecT extends AbstractBrooklynObjectSpec<? extends T, SpecT>> SpecT createCatalogSpec(CatalogItem<T, SpecT> item, Set<String> encounteredTypes) {
+        // Not really clear what should happen to the top-level attributes, ignored until a good use case appears.
+        return (SpecT) CampCatalogUtils.createSpec(mgmt, (CatalogItem)item, encounteredTypes);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampUtils.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampUtils.java b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampUtils.java
new file mode 100644
index 0000000..4fca75c
--- /dev/null
+++ b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/CampUtils.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.brooklyn.camp.brooklyn.spi.creation;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.StringReader;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.brooklyn.api.catalog.CatalogItem;
+import org.apache.brooklyn.api.catalog.CatalogItem.CatalogItemType;
+import org.apache.brooklyn.api.entity.EntitySpec;
+import org.apache.brooklyn.api.location.Location;
+import org.apache.brooklyn.api.location.LocationSpec;
+import org.apache.brooklyn.api.mgmt.ManagementContext;
+import org.apache.brooklyn.api.policy.Policy;
+import org.apache.brooklyn.api.policy.PolicySpec;
+import org.apache.brooklyn.camp.CampPlatform;
+import org.apache.brooklyn.camp.brooklyn.BrooklynCampConstants;
+import org.apache.brooklyn.camp.brooklyn.api.AssemblyTemplateSpecInstantiator;
+import org.apache.brooklyn.camp.spi.AssemblyTemplate;
+import org.apache.brooklyn.camp.spi.instantiate.AssemblyTemplateInstantiator;
+import org.apache.brooklyn.camp.spi.pdp.DeploymentPlan;
+import org.apache.brooklyn.core.catalog.internal.BasicBrooklynCatalog;
+import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
+import org.apache.brooklyn.core.catalog.internal.BasicBrooklynCatalog.BrooklynLoaderTracker;
+import org.apache.brooklyn.core.mgmt.classloading.BrooklynClassLoadingContext;
+import org.apache.brooklyn.core.objs.BrooklynObjectInternal.ConfigurationSupportInternal;
+import org.apache.brooklyn.util.exceptions.Exceptions;
+import org.apache.brooklyn.util.guava.Maybe;
+import org.apache.brooklyn.util.stream.Streams;
+import org.apache.brooklyn.util.yaml.Yamls;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+
+public class CampUtils {
+
+    public static List<EntitySpec<?>> createServiceSpecs(String plan, BrooklynClassLoadingContext loader, Set<String> encounteredTypes) {
+        CampPlatform camp = getCampPlatform(loader.getManagementContext());
+
+        AssemblyTemplate at;
+        BrooklynLoaderTracker.setLoader(loader);
+        try {
+            at = camp.pdp().registerDeploymentPlan(new StringReader(plan));
+        } finally {
+            BrooklynLoaderTracker.unsetLoader(loader);
+        }
+
+        try {
+            AssemblyTemplateInstantiator instantiator = at.getInstantiator().newInstance();
+            if (instantiator instanceof AssemblyTemplateSpecInstantiator) {
+                return ((AssemblyTemplateSpecInstantiator)instantiator).createServiceSpecs(at, camp, loader, encounteredTypes);
+            }
+            throw new IllegalStateException("Unable to instantiate YAML; incompatible instantiator "+instantiator+" for "+at);
+        } catch (Exception e) {
+            throw Exceptions.propagate(e);
+        }
+    }
+
+    public static PolicySpec<?> createPolicySpec(String yamlPlan, BrooklynClassLoadingContext loader, Set<String> encounteredCatalogTypes) {
+        DeploymentPlan plan = makePlanFromYaml(loader.getManagementContext(), yamlPlan);
+
+        //Would ideally re-use the PolicySpecResolver
+        //but it is CAMP specific and there is no easy way to get hold of it.
+        Object policies = checkNotNull(plan.getCustomAttributes().get(BasicBrooklynCatalog.POLICIES_KEY), "policy config");
+        if (!(policies instanceof Iterable<?>)) {
+            throw new IllegalStateException("The value of " + BasicBrooklynCatalog.POLICIES_KEY + " must be an Iterable.");
+        }
+
+        Object policy = Iterables.getOnlyElement((Iterable<?>)policies);
+
+        return createPolicySpec(loader, policy, encounteredCatalogTypes);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static PolicySpec<?> createPolicySpec(BrooklynClassLoadingContext loader, Object policy, Set<String> encounteredCatalogTypes) {
+        Map<String, Object> itemMap;
+        if (policy instanceof String) {
+            itemMap = ImmutableMap.<String, Object>of("type", policy);
+        } else if (policy instanceof Map) {
+            itemMap = (Map<String, Object>) policy;
+        } else {
+            throw new IllegalStateException("Policy expected to be string or map. Unsupported object type " + policy.getClass().getName() + " (" + policy.toString() + ")");
+        }
+
+        String versionedId = (String) checkNotNull(Yamls.getMultinameAttribute(itemMap, "policy_type", "policyType", "type"), "policy type");
+        PolicySpec<? extends Policy> spec;
+        CatalogItem<?, ?> policyItem = CatalogUtils.getCatalogItemOptionalVersion(loader.getManagementContext(), versionedId);
+        if (policyItem != null && !encounteredCatalogTypes.contains(policyItem.getSymbolicName())) {
+            if (policyItem.getCatalogItemType() != CatalogItemType.POLICY) {
+                throw new IllegalStateException("Non-policy catalog item in policy context: " + policyItem);
+            }
+            spec = (PolicySpec<? extends Policy>) CampCatalogUtils.createSpec(loader.getManagementContext(), policyItem, encounteredCatalogTypes);
+        } else {
+            spec = PolicySpec.create(loader.loadClass(versionedId, Policy.class));
+        }
+        Map<String, Object> brooklynConfig = (Map<String, Object>) itemMap.get("brooklyn.config");
+        if (brooklynConfig != null) {
+            spec.configure(brooklynConfig);
+        }
+        return spec;
+    }
+
+    public static LocationSpec<?> createLocationSpec(String yamlPlan, BrooklynClassLoadingContext loader, Set<String> encounteredTypes) {
+        DeploymentPlan plan = makePlanFromYaml(loader.getManagementContext(), yamlPlan);
+        Object locations = checkNotNull(plan.getCustomAttributes().get(BasicBrooklynCatalog.LOCATIONS_KEY), "location config");
+        if (!(locations instanceof Iterable<?>)) {
+            throw new IllegalStateException("The value of " + BasicBrooklynCatalog.LOCATIONS_KEY + " must be an Iterable.");
+        }
+
+        Object location = Iterables.getOnlyElement((Iterable<?>)locations);
+
+        return createLocationSpec(loader, location); 
+    }
+
+    @SuppressWarnings("unchecked")
+    private static LocationSpec<?> createLocationSpec(BrooklynClassLoadingContext loader, Object location) {
+        Map<String, Object> itemMap;
+        if (location instanceof String) {
+            itemMap = ImmutableMap.<String, Object>of("type", location);
+        } else if (location instanceof Map) {
+            itemMap = (Map<String, Object>) location;
+        } else {
+            throw new IllegalStateException("Location expected to be string or map. Unsupported object type " + location.getClass().getName() + " (" + location.toString() + ")");
+        }
+
+        String type = (String) checkNotNull(Yamls.getMultinameAttribute(itemMap, "location_type", "locationType", "type"), "location type");
+        Map<String, Object> brooklynConfig = (Map<String, Object>) itemMap.get("brooklyn.config");
+        Maybe<Class<? extends Location>> javaClass = loader.tryLoadClass(type, Location.class);
+        if (javaClass.isPresent()) {
+            LocationSpec<?> spec = LocationSpec.create(javaClass.get());
+            if (brooklynConfig != null) {
+                spec.configure(brooklynConfig);
+            }
+            return spec;
+        } else {
+            Maybe<Location> loc = loader.getManagementContext().getLocationRegistry().resolve(type, false, brooklynConfig);
+            if (loc.isPresent()) {
+                // TODO extensions?
+                Map<String, Object> locConfig = ((ConfigurationSupportInternal)loc.get().config()).getBag().getAllConfig();
+                Class<? extends Location> locType = loc.get().getClass();
+                Set<Object> locTags = loc.get().tags().getTags();
+                String locDisplayName = loc.get().getDisplayName();
+                return LocationSpec.create(locType)
+                        .configure(locConfig)
+                        .displayName(locDisplayName)
+                        .tags(locTags);
+            } else {
+                throw new IllegalStateException("No class or resolver found for location type "+type);
+            }
+        }
+    }
+
+    public static DeploymentPlan makePlanFromYaml(ManagementContext mgmt, String yaml) {
+        CampPlatform camp = getCampPlatform(mgmt);
+        return camp.pdp().parseDeploymentPlan(Streams.newReaderWithContents(yaml));
+    }
+
+    public static CampPlatform getCampPlatform(ManagementContext mgmt) {
+        CampPlatform result = mgmt.getConfig().getConfig(BrooklynCampConstants.CAMP_PLATFORM);
+        if (result!=null) {
+            return result;
+        } else {
+            throw new IllegalStateException("No CAMP Platform is registered with this Brooklyn management context.");
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/service/DefaultServiceTypeResolver.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/service/DefaultServiceTypeResolver.java b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/service/DefaultServiceTypeResolver.java
new file mode 100644
index 0000000..fdd57d5
--- /dev/null
+++ b/usage/camp/src/main/java/org/apache/brooklyn/camp/brooklyn/spi/creation/service/DefaultServiceTypeResolver.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.brooklyn.camp.brooklyn.spi.creation.service;
+
+public class DefaultServiceTypeResolver extends BrooklynServiceTypeResolver {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java b/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java
index 7389190..f7c98c0 100644
--- a/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java
+++ b/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java
@@ -596,6 +596,49 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
     }
 
     @Test
+    public void testDeeplyNestedTypesDoesNotRecurse() throws Exception {
+        String symbolicName = "my.catalog.app.id.basic";
+        // Need to have a stand alone caller first so we can create an item to depend on it.
+        // After that replace it/insert a new version which completes the cycle
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + symbolicName + ".caller",
+                "  version: " + TEST_VERSION + "pre",
+                "",
+                "services:",
+                "- type: org.apache.brooklyn.entity.stock.BasicEntity");
+
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + symbolicName + ".callee",
+                "  version: " + TEST_VERSION,
+                "",
+                "services:",
+                "- type: " + symbolicName + ".caller");
+
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + symbolicName + ".caller",
+                "  version: " + TEST_VERSION,
+                "",
+                "services:",
+                "- type: org.apache.brooklyn.entity.stock.BasicEntity",
+                // Being a child is important, triggers the case where
+                // we allow retrying with other transformers
+                // and thus breaking the recursive check.
+                "  brooklyn.children:",
+                "  - type: " + symbolicName + ".callee");
+
+        try {
+            createAndStartApplication(
+                    "services:",
+                    "- type: " + symbolicName + ".caller");
+        } catch (IllegalStateException e) {
+            assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
+        }
+    }
+
+    @Test
     public void testOsgiNotLeakingToParent() {
         addCatalogOSGiEntity(SIMPLE_ENTITY_TYPE);
         try {

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3d415c41/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/TestAppAssemblyInstantiator.java
----------------------------------------------------------------------
diff --git a/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/TestAppAssemblyInstantiator.java b/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/TestAppAssemblyInstantiator.java
index 831805f..422998b 100644
--- a/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/TestAppAssemblyInstantiator.java
+++ b/usage/camp/src/test/java/org/apache/brooklyn/camp/brooklyn/test/lite/TestAppAssemblyInstantiator.java
@@ -18,6 +18,7 @@
  */
 package org.apache.brooklyn.camp.brooklyn.test.lite;
 
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -39,6 +40,8 @@ import org.apache.brooklyn.core.test.entity.TestEntity;
 import org.apache.brooklyn.util.collections.MutableMap;
 import org.apache.brooklyn.util.core.config.ConfigBag;
 
+import com.google.common.collect.ImmutableList;
+
 /** simple illustrative instantiator which always makes a {@link TestApplication}, populated with {@link TestEntity} children,
  * all setting {@link TestEntity#CONF_NAME} for the name in the plan and in the service specs
  * <p>
@@ -52,14 +55,14 @@ public class TestAppAssemblyInstantiator extends BasicAssemblyTemplateInstantiat
         }
         ManagementContext mgmt = ((HasBrooklynManagementContext)platform).getBrooklynManagementContext();
         
-        TestApplication app = (TestApplication) mgmt.getEntityManager().createEntity( createSpec(template, platform, null, false) );
+        TestApplication app = (TestApplication) mgmt.getEntityManager().createEntity( createApplicationSpec(template, platform, null) );
         mgmt.getEntityManager().manage(app);
 
         return new TestAppAssembly(app);
     }
 
     @Override
-    public EntitySpec<? extends Application> createSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader, boolean autoUnwrap) {
+    public EntitySpec<? extends Application> createApplicationSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext loader) {
         EntitySpec<TestApplication> app = EntitySpec.create(TestApplication.class)
             .configure(TestEntity.CONF_NAME, template.getName())
             .configure(TestEntity.CONF_MAP_THING, MutableMap.of("type", template.getType(), "desc", template.getDescription()));
@@ -84,8 +87,9 @@ public class TestAppAssemblyInstantiator extends BasicAssemblyTemplateInstantiat
     }
 
     @Override
-    public EntitySpec<?> createNestedSpec(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext itemLoader, Set<String> encounteredCatalogTypes) {
-        return createSpec(template, platform, itemLoader, true);
+    public List<EntitySpec<?>> createServiceSpecs(AssemblyTemplate template, CampPlatform platform, BrooklynClassLoadingContext itemLoader, Set<String> encounteredCatalogTypes) {
+        EntitySpec<?> createApplicationSpec = createApplicationSpec(template, platform, itemLoader);
+        return ImmutableList.<EntitySpec<?>>of(createApplicationSpec);
     }
 
 }