You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by sv...@apache.org on 2017/02/13 06:26:41 UTC

[1/9] brooklyn-server git commit: Split CatalogYamlLocationTest for non-osgi and osgi

Repository: brooklyn-server
Updated Branches:
  refs/heads/master 6571bab4a -> 0f649fe17


Split CatalogYamlLocationTest for non-osgi and osgi


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

Branch: refs/heads/master
Commit: 7ff81869bbc6dbef9f7baf0c9211c75f2b0ea7e5
Parents: 6571bab
Author: Aled Sage <al...@gmail.com>
Authored: Fri Feb 10 14:21:11 2017 +0000
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Feb 10 14:51:43 2017 +0000

----------------------------------------------------------------------
 .../camp/brooklyn/AbstractYamlTest.java         |  16 ++
 .../catalog/CatalogOsgiYamlLocationTest.java    | 198 +++++++++++++++++++
 .../catalog/CatalogYamlLocationTest.java        |  81 +-------
 3 files changed, 220 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/7ff81869/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java
index a919aba..376fb5c 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java
@@ -28,6 +28,7 @@ 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.api.mgmt.Task;
+import org.apache.brooklyn.api.typereg.RegisteredType;
 import org.apache.brooklyn.camp.brooklyn.spi.creation.CampTypePlanTransformer;
 import org.apache.brooklyn.core.catalog.internal.CatalogUtils;
 import org.apache.brooklyn.core.entity.Entities;
@@ -38,6 +39,7 @@ import org.apache.brooklyn.core.mgmt.internal.LocalManagementContext;
 import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests;
 import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests.Builder;
 import org.apache.brooklyn.core.typereg.RegisteredTypeLoadingContexts;
+import org.apache.brooklyn.core.typereg.RegisteredTypePredicates;
 import org.apache.brooklyn.util.collections.MutableMap;
 import org.apache.brooklyn.util.core.ResourceUtils;
 import org.apache.brooklyn.util.net.Urls;
@@ -48,6 +50,8 @@ import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 
 import com.google.common.base.Joiner;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
 
 public abstract class AbstractYamlTest {
 
@@ -228,6 +232,18 @@ public abstract class AbstractYamlTest {
         return CatalogUtils.getVersionedId(id, TEST_VERSION);
     }
 
+    protected String ver(String id, String version) {
+        return CatalogUtils.getVersionedId(id, version);
+    }
+    
+    protected int countCatalogLocations() {
+        return countCatalogItemsMatching(RegisteredTypePredicates.IS_LOCATION);
+    }
+
+    protected int countCatalogItemsMatching(Predicate<? super RegisteredType> filter) {
+        return Iterables.size(mgmt().getTypeRegistry().getMatching(filter));
+    }
+    
     public void forceCatalogUpdate() {
         forceUpdate = true;
     }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/7ff81869/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlLocationTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlLocationTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlLocationTest.java
new file mode 100644
index 0000000..94b2c50
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlLocationTest.java
@@ -0,0 +1,198 @@
+/*
+ * 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.catalog;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.location.Location;
+import org.apache.brooklyn.api.location.LocationDefinition;
+import org.apache.brooklyn.api.location.LocationSpec;
+import org.apache.brooklyn.api.typereg.OsgiBundleWithUrl;
+import org.apache.brooklyn.api.typereg.RegisteredType;
+import org.apache.brooklyn.camp.brooklyn.AbstractYamlTest;
+import org.apache.brooklyn.core.config.BasicConfigKey;
+import org.apache.brooklyn.core.entity.Entities;
+import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
+import org.apache.brooklyn.core.typereg.RegisteredTypePredicates;
+import org.apache.brooklyn.core.typereg.RegisteredTypes;
+import org.apache.brooklyn.test.support.TestResourceUnavailableException;
+import org.apache.brooklyn.util.text.StringFunctions;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+public class CatalogOsgiYamlLocationTest extends AbstractYamlTest {
+    private static final String SIMPLE_LOCATION_TYPE = "org.apache.brooklyn.test.osgi.entities.SimpleLocation";
+
+    @Override
+    protected boolean disableOsgi() {
+        return false;
+    }
+
+    @AfterMethod(alwaysRun=true)
+    @Override
+    public void tearDown() throws Exception {
+        try {
+            for (RegisteredType ci : mgmt().getTypeRegistry().getMatching(RegisteredTypePredicates.IS_LOCATION)) {
+                mgmt().getCatalog().deleteCatalogItem(ci.getSymbolicName(), ci.getVersion());
+            }
+        } finally {
+            super.tearDown();
+        }
+    }
+    
+    @Test
+    public void testAddCatalogItemOsgi() throws Exception {
+        assertEquals(countCatalogLocations(), 0);
+
+        String symbolicName = "my.catalog.location.id.load";
+        addCatalogLocation(symbolicName, SIMPLE_LOCATION_TYPE, getOsgiLibraries());
+        assertAdded(symbolicName, SIMPLE_LOCATION_TYPE);
+        assertOsgi(symbolicName);
+        removeAndAssert(symbolicName);
+    }
+
+    @Test
+    public void testAddCatalogItemOsgiLegacySyntax() throws Exception {
+        assertEquals(countCatalogLocations(), 0);
+
+        String symbolicName = "my.catalog.location.id.load";
+        addCatalogLocationLegacySyntax(symbolicName, SIMPLE_LOCATION_TYPE, getOsgiLibraries());
+        assertAdded(symbolicName, SIMPLE_LOCATION_TYPE);
+        assertOsgi(symbolicName);
+        removeAndAssert(symbolicName);
+    }
+
+    private void assertOsgi(String symbolicName) {
+        RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
+        Collection<OsgiBundleWithUrl> libs = item.getLibraries();
+        assertEquals(libs.size(), 1);
+        assertEquals(Iterables.getOnlyElement(libs).getUrl(), Iterables.getOnlyElement(getOsgiLibraries()));
+    }
+
+    private void assertAdded(String symbolicName, String expectedJavaType) {
+        RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
+        assertEquals(item.getSymbolicName(), symbolicName);
+        Assert.assertTrue(RegisteredTypes.isSubtypeOf(item, Location.class), "Expected Location, not "+item.getSuperTypes());
+        assertEquals(countCatalogLocations(), 1);
+
+        // Item added to catalog should automatically be available in location registry
+        LocationDefinition def = mgmt().getLocationRegistry().getDefinedLocationByName(symbolicName);
+        assertEquals(def.getId(), symbolicName);
+        assertEquals(def.getName(), symbolicName);
+        
+        LocationSpec<?> spec = mgmt().getTypeRegistry().createSpec(item, null, LocationSpec.class);
+        assertEquals(spec.getType().getName(), expectedJavaType);
+    }
+    
+    private void removeAndAssert(String symbolicName) {
+        // Deleting item: should be gone from catalog, and from location registry
+        deleteCatalogEntity(symbolicName);
+
+        assertEquals(countCatalogLocations(), 0);
+        assertNull(mgmt().getLocationRegistry().getDefinedLocationByName(symbolicName));
+    }
+
+    @Test
+    public void testLaunchApplicationReferencingOsgiLocation() throws Exception {
+        String symbolicName = "my.catalog.location.id.launch";
+        addCatalogLocation(symbolicName, SIMPLE_LOCATION_TYPE, getOsgiLibraries());
+        runLaunchApplicationReferencingLocation(symbolicName, SIMPLE_LOCATION_TYPE);
+        
+        deleteCatalogEntity(symbolicName);
+    }
+    
+    protected void runLaunchApplicationReferencingLocation(String locTypeInYaml, String locType) throws Exception {
+        Entity app = createAndStartApplication(
+            "name: simple-app-yaml",
+            "location: ",
+            "  "+locTypeInYaml+":",
+            "    config2: config2 override",
+            "    config3: config3",
+            "services: ",
+            "  - type: org.apache.brooklyn.entity.stock.BasicStartable");
+
+        Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
+        Location location = Iterables.getOnlyElement(Entities.getAllInheritedLocations(simpleEntity));
+        assertEquals(location.getClass().getName(), locType);
+        assertEquals(location.getConfig(new BasicConfigKey<String>(String.class, "config1")), "config1");
+        assertEquals(location.getConfig(new BasicConfigKey<String>(String.class, "config2")), "config2 override");
+        assertEquals(location.getConfig(new BasicConfigKey<String>(String.class, "config3")), "config3");
+    }
+
+    private List<String> getOsgiLibraries() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+        return ImmutableList.of(OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL);
+    }
+    
+    private void addCatalogLocation(String symbolicName, String locationType, List<String> libraries) {
+        ImmutableList.Builder<String> yaml = ImmutableList.<String>builder().add(
+                "brooklyn.catalog:",
+                "  id: " + symbolicName,
+                "  version: " + TEST_VERSION,
+                "  itemType: location",
+                "  name: My Catalog Location",
+                "  description: My description");
+        if (libraries!=null && libraries.size() > 0) {
+            yaml.add("  libraries:")
+                .addAll(Lists.transform(libraries, StringFunctions.prepend("  - url: ")));
+        }
+        yaml.add(
+                "  item:",
+                "    type: " + locationType,
+                "    brooklyn.config:",
+                "      config1: config1",
+                "      config2: config2");
+        
+        
+        addCatalogItems(yaml.build());
+    }
+
+    private void addCatalogLocationLegacySyntax(String symbolicName, String locationType, List<String> libraries) {
+        ImmutableList.Builder<String> yaml = ImmutableList.<String>builder().add(
+                "brooklyn.catalog:",
+                "  id: " + symbolicName,
+                "  name: My Catalog Location",
+                "  description: My description",
+                "  version: " + TEST_VERSION);
+        if (libraries!=null && libraries.size() > 0) {
+            yaml.add("  libraries:")
+                .addAll(Lists.transform(libraries, StringFunctions.prepend("  - url: ")));
+        }
+        yaml.add(
+                "",
+                "brooklyn.locations:",
+                "- type: " + locationType,
+                "  brooklyn.config:",
+                "    config1: config1",
+                "    config2: config2");
+        
+        
+        addCatalogItems(yaml.build());
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/7ff81869/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlLocationTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlLocationTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlLocationTest.java
index df00c6c..01fb484 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlLocationTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlLocationTest.java
@@ -23,7 +23,6 @@ import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNull;
 import static org.testng.Assert.assertTrue;
 
-import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
@@ -31,20 +30,16 @@ import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.location.Location;
 import org.apache.brooklyn.api.location.LocationDefinition;
 import org.apache.brooklyn.api.location.LocationSpec;
-import org.apache.brooklyn.api.typereg.OsgiBundleWithUrl;
 import org.apache.brooklyn.api.typereg.RegisteredType;
 import org.apache.brooklyn.camp.brooklyn.AbstractYamlTest;
 import org.apache.brooklyn.core.config.BasicConfigKey;
 import org.apache.brooklyn.core.entity.Entities;
-import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
 import org.apache.brooklyn.core.typereg.RegisteredTypePredicates;
 import org.apache.brooklyn.core.typereg.RegisteredTypes;
 import org.apache.brooklyn.entity.stock.BasicEntity;
 import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
 import org.apache.brooklyn.test.Asserts;
-import org.apache.brooklyn.test.support.TestResourceUnavailableException;
 import org.apache.brooklyn.util.collections.CollectionFunctionals;
-import org.apache.brooklyn.util.text.StringFunctions;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.Test;
@@ -53,17 +48,10 @@ import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 
 public class CatalogYamlLocationTest extends AbstractYamlTest {
     private static final String LOCALHOST_LOCATION_SPEC = "localhost";
     private static final String LOCALHOST_LOCATION_TYPE = LocalhostMachineProvisioningLocation.class.getName();
-    private static final String SIMPLE_LOCATION_TYPE = "org.apache.brooklyn.test.osgi.entities.SimpleLocation";
-
-    @Override
-    protected boolean disableOsgi() {
-        return false;
-    }
 
     @AfterMethod(alwaysRun=true)
     @Override
@@ -88,44 +76,15 @@ public class CatalogYamlLocationTest extends AbstractYamlTest {
     }
 
     @Test
-    public void testAddCatalogItemOsgi() throws Exception {
-        assertEquals(countCatalogLocations(), 0);
-
-        String symbolicName = "my.catalog.location.id.load";
-        addCatalogLocation(symbolicName, SIMPLE_LOCATION_TYPE, getOsgiLibraries());
-        assertAdded(symbolicName, SIMPLE_LOCATION_TYPE);
-        assertOsgi(symbolicName);
-        removeAndAssert(symbolicName);
-    }
-
-    @Test
     public void testAddCatalogItemLegacySyntax() throws Exception {
         assertEquals(countCatalogLocations(), 0);
 
         String symbolicName = "my.catalog.location.id.load";
-        addCatalogLocationLegacySyntax(symbolicName, LOCALHOST_LOCATION_TYPE, null);
+        addCatalogLocationLegacySyntax(symbolicName, LOCALHOST_LOCATION_TYPE);
         assertAdded(symbolicName, LOCALHOST_LOCATION_TYPE);
         removeAndAssert(symbolicName);
     }
 
-    @Test
-    public void testAddCatalogItemOsgiLegacySyntax() throws Exception {
-        assertEquals(countCatalogLocations(), 0);
-
-        String symbolicName = "my.catalog.location.id.load";
-        addCatalogLocationLegacySyntax(symbolicName, SIMPLE_LOCATION_TYPE, getOsgiLibraries());
-        assertAdded(symbolicName, SIMPLE_LOCATION_TYPE);
-        assertOsgi(symbolicName);
-        removeAndAssert(symbolicName);
-    }
-
-    private void assertOsgi(String symbolicName) {
-        RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
-        Collection<OsgiBundleWithUrl> libs = item.getLibraries();
-        assertEquals(libs.size(), 1);
-        assertEquals(Iterables.getOnlyElement(libs).getUrl(), Iterables.getOnlyElement(getOsgiLibraries()));
-    }
-
     @SuppressWarnings({ "rawtypes" })
     private void assertAdded(String symbolicName, String expectedJavaType) {
         RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
@@ -171,7 +130,7 @@ public class CatalogYamlLocationTest extends AbstractYamlTest {
     @Test
     public void testLaunchApplicationReferencingLocationClassLegacySyntax() throws Exception {
         String symbolicName = "my.catalog.location.id.launch";
-        addCatalogLocationLegacySyntax(symbolicName, LOCALHOST_LOCATION_TYPE, null);
+        addCatalogLocationLegacySyntax(symbolicName, LOCALHOST_LOCATION_TYPE);
         runLaunchApplicationReferencingLocation(symbolicName, LOCALHOST_LOCATION_TYPE);
 
         deleteCatalogEntity(symbolicName);
@@ -180,21 +139,12 @@ public class CatalogYamlLocationTest extends AbstractYamlTest {
     @Test
     public void testLaunchApplicationReferencingLocationSpecLegacySyntax() throws Exception {
         String symbolicName = "my.catalog.location.id.launch";
-        addCatalogLocationLegacySyntax(symbolicName, LOCALHOST_LOCATION_SPEC, null);
+        addCatalogLocationLegacySyntax(symbolicName, LOCALHOST_LOCATION_SPEC);
         runLaunchApplicationReferencingLocation(symbolicName, LOCALHOST_LOCATION_TYPE);
 
         deleteCatalogEntity(symbolicName);
     }
 
-    @Test
-    public void testLaunchApplicationReferencingOsgiLocation() throws Exception {
-        String symbolicName = "my.catalog.location.id.launch";
-        addCatalogLocation(symbolicName, SIMPLE_LOCATION_TYPE, getOsgiLibraries());
-        runLaunchApplicationReferencingLocation(symbolicName, SIMPLE_LOCATION_TYPE);
-        
-        deleteCatalogEntity(symbolicName);
-    }
-    
     // See https://issues.apache.org/jira/browse/BROOKLYN-248
     @Test
     public void testTypeInheritance() throws Exception {
@@ -380,11 +330,6 @@ public class CatalogYamlLocationTest extends AbstractYamlTest {
         assertEquals(location.getConfig(new BasicConfigKey<String>(String.class, "config3")), "config3");
     }
 
-    private List<String> getOsgiLibraries() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-        return ImmutableList.of(OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL);
-    }
-    
     private void addCatalogLocation(String symbolicName, String locationType, List<String> libraries) {
         ImmutableList.Builder<String> yaml = ImmutableList.<String>builder().add(
                 "brooklyn.catalog:",
@@ -392,12 +337,7 @@ public class CatalogYamlLocationTest extends AbstractYamlTest {
                 "  version: " + TEST_VERSION,
                 "  itemType: location",
                 "  name: My Catalog Location",
-                "  description: My description");
-        if (libraries!=null && libraries.size() > 0) {
-            yaml.add("  libraries:")
-                .addAll(Lists.transform(libraries, StringFunctions.prepend("  - url: ")));
-        }
-        yaml.add(
+                "  description: My description",
                 "  item:",
                 "    type: " + locationType,
                 "    brooklyn.config:",
@@ -408,18 +348,13 @@ public class CatalogYamlLocationTest extends AbstractYamlTest {
         addCatalogItems(yaml.build());
     }
 
-    private void addCatalogLocationLegacySyntax(String symbolicName, String locationType, List<String> libraries) {
+    private void addCatalogLocationLegacySyntax(String symbolicName, String locationType) {
         ImmutableList.Builder<String> yaml = ImmutableList.<String>builder().add(
                 "brooklyn.catalog:",
                 "  id: " + symbolicName,
                 "  name: My Catalog Location",
                 "  description: My description",
-                "  version: " + TEST_VERSION);
-        if (libraries!=null && libraries.size() > 0) {
-            yaml.add("  libraries:")
-                .addAll(Lists.transform(libraries, StringFunctions.prepend("  - url: ")));
-        }
-        yaml.add(
+                "  version: " + TEST_VERSION,
                 "",
                 "brooklyn.locations:",
                 "- type: " + locationType,
@@ -431,10 +366,6 @@ public class CatalogYamlLocationTest extends AbstractYamlTest {
         addCatalogItems(yaml.build());
     }
 
-    private int countCatalogLocations() {
-        return Iterables.size(mgmt().getTypeRegistry().getMatching(RegisteredTypePredicates.IS_LOCATION));
-    }
-
     @Test
     public void testManagedLocationsCreateAndCleanup() {
         assertLocationRegistryCount(0);


[5/9] brooklyn-server git commit: Split CatalogYamlEntityTest for non-osgi and osgi

Posted by sv...@apache.org.
Split CatalogYamlEntityTest for non-osgi and osgi


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

Branch: refs/heads/master
Commit: c5ae189c544dc1dc48a37d5537b86488a33e75ec
Parents: 27ad37c
Author: Aled Sage <al...@gmail.com>
Authored: Fri Feb 10 14:06:34 2017 +0000
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Feb 10 22:16:06 2017 +0000

----------------------------------------------------------------------
 .../catalog/CatalogOsgiYamlEntityTest.java      | 861 +++++++++++++++++++
 .../brooklyn/catalog/CatalogYamlEntityTest.java | 796 +++++------------
 2 files changed, 1064 insertions(+), 593 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/c5ae189c/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlEntityTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlEntityTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlEntityTest.java
new file mode 100644
index 0000000..f492f12
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlEntityTest.java
@@ -0,0 +1,861 @@
+/*
+ * 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.catalog;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+
+import java.io.InputStream;
+
+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.typereg.BrooklynTypeRegistry;
+import org.apache.brooklyn.api.typereg.RegisteredType;
+import org.apache.brooklyn.camp.brooklyn.AbstractYamlTest;
+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.mgmt.osgi.OsgiStandaloneTest;
+import org.apache.brooklyn.core.test.entity.TestEntity;
+import org.apache.brooklyn.core.test.entity.TestEntityImpl;
+import org.apache.brooklyn.entity.stock.BasicApplication;
+import org.apache.brooklyn.entity.stock.BasicEntity;
+import org.apache.brooklyn.test.support.TestResourceUnavailableException;
+import org.apache.brooklyn.util.core.ResourceUtils;
+import org.apache.brooklyn.util.osgi.OsgiTestResources;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+
+public class CatalogOsgiYamlEntityTest extends AbstractYamlTest {
+    
+    // Some of these testes duplicate several of the non-osgi test. However, that is important 
+    // because there are subtleties of which OSGi bundles a catalog item will use for loading,
+    // particularly when nesting and/or sub-typing entities.
+    //
+    // The non-osgi tests are much faster to run!
+
+    private static final String SIMPLE_ENTITY_TYPE = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY;
+
+    @Override
+    protected boolean disableOsgi() {
+        return false;
+    }
+
+    @Test
+    public void testAddOsgiCatalogItem() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String symbolicName = "my.catalog.app.id.load";
+        addCatalogOSGiEntity(symbolicName);
+        RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
+        assertEquals(item.getSymbolicName(), symbolicName);
+
+        deleteCatalogEntity(symbolicName);
+    }
+
+    @Test
+    public void testLaunchApplicationReferencingCatalog() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String symbolicName = "my.catalog.app.id.launch";
+        registerAndLaunchAndAssertSimpleEntity(symbolicName, SIMPLE_ENTITY_TYPE);
+    }
+
+    @Test
+    public void testLaunchApplicationWithCatalogReferencingOtherCatalog() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String referencedSymbolicName = "my.catalog.app.id.referenced";
+        String referrerSymbolicName = "my.catalog.app.id.referring";
+        addCatalogOSGiEntity(referencedSymbolicName, SIMPLE_ENTITY_TYPE);
+        addCatalogEntity(referrerSymbolicName, ver(referencedSymbolicName));
+
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: " + ver(referrerSymbolicName));
+
+        Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(simpleEntity.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+
+        deleteCatalogEntity(referencedSymbolicName);
+        deleteCatalogEntity(referrerSymbolicName);
+    }
+
+    @Test
+    public void testLaunchApplicationChildWithCatalogReferencingOtherCatalog() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String referencedSymbolicName = "my.catalog.app.id.child.referenced";
+        String referrerSymbolicName = "my.catalog.app.id.child.referring";
+        addCatalogOSGiEntity(referencedSymbolicName, SIMPLE_ENTITY_TYPE);
+        addCatalogChildEntity(referrerSymbolicName, ver(referencedSymbolicName));
+
+        Entity app = createAndStartApplication(
+            "name: simple-app-yaml",
+            "location: localhost",
+            "services:",
+            "- type: "+BasicEntity.class.getName(),
+            "  brooklyn.children:",
+            "  - type: " + ver(referrerSymbolicName));
+
+        Entity child = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(child.getEntityType().getName(), BasicEntity.class.getName());
+        Entity grandChild = Iterables.getOnlyElement(child.getChildren());
+        assertEquals(grandChild.getEntityType().getName(), BasicEntity.class.getName());
+        Entity grandGrandChild = Iterables.getOnlyElement(grandChild.getChildren());
+        assertEquals(grandGrandChild.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+
+        deleteCatalogEntity(referencedSymbolicName);
+        deleteCatalogEntity(referrerSymbolicName);
+    }
+
+    @Test
+    public void testLaunchApplicationChildWithCatalogReferencingOtherCatalogServicesBlock() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String referencedSymbolicName = "my.catalog.app.id.child.referenced";
+        String referrerSymbolicName = "my.catalog.app.id.child.referring";
+        addCatalogOSGiEntity(referencedSymbolicName, SIMPLE_ENTITY_TYPE);
+        addCatalogChildOSGiEntityWithServicesBlock(referrerSymbolicName, ver(referencedSymbolicName));
+
+        Entity app = createAndStartApplication(
+            "services:",
+            "- type: "+BasicEntity.class.getName(),
+            "  brooklyn.children:",
+            "  - type: " + ver(referrerSymbolicName));
+
+        Entity child = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(child.getEntityType().getName(), BasicEntity.class.getName());
+        Entity grandChild = Iterables.getOnlyElement(child.getChildren());
+        assertEquals(grandChild.getEntityType().getName(), BasicEntity.class.getName());
+        Entity grandGrandChild = Iterables.getOnlyElement(grandChild.getChildren());
+        assertEquals(grandGrandChild.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+
+        deleteCatalogEntity(referencedSymbolicName);
+        deleteCatalogEntity(referrerSymbolicName);
+    }
+    
+    @Test
+    public void testLaunchApplicationWithTypeUsingJavaColonPrefix() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String symbolicName = SIMPLE_ENTITY_TYPE;
+        String serviceName = "java:"+SIMPLE_ENTITY_TYPE;
+        registerAndLaunchAndAssertSimpleEntity(symbolicName, serviceName);
+    }
+
+    @Test
+    public void testLaunchApplicationLoopWithJavaTypeName() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String symbolicName = SIMPLE_ENTITY_TYPE;
+        String serviceName = SIMPLE_ENTITY_TYPE;
+        registerAndLaunchAndAssertSimpleEntity(symbolicName, serviceName);
+    }
+
+    @Test
+    public void testReferenceInstalledBundleByName() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String firstItemId = "my.catalog.app.id.register_bundle";
+        String secondItemId = "my.catalog.app.id.reference_bundle";
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + firstItemId,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  libraries:",
+            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  item:",
+            "    type: " + SIMPLE_ENTITY_TYPE);
+        deleteCatalogEntity(firstItemId);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + secondItemId,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  libraries:",
+            "  - name: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_NAME,
+            "    version: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_VERSION,
+            "  item:",
+            "    type: " + SIMPLE_ENTITY_TYPE);
+
+        deleteCatalogEntity(secondItemId);
+    }
+
+    @Test
+    public void testReferenceNonInstalledBundledByNameFails() {
+        String nonExistentId = "none-existent-id";
+        String nonExistentVersion = "9.9.9";
+        try {
+            addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: my.catalog.app.id.non_existing.ref",
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  libraries:",
+                "  - name: " + nonExistentId,
+                "    version: " + nonExistentVersion,
+                "  item:",
+                "    type: " + SIMPLE_ENTITY_TYPE);
+            fail();
+        } catch (IllegalStateException e) {
+            Assert.assertEquals(e.getMessage(), "Bundle from null failed to install: Bundle CatalogBundleDto{symbolicName=" + nonExistentId + ", version=" + nonExistentVersion + ", url=null} not previously registered, but URL is empty.");
+        }
+    }
+
+    @Test
+    public void testPartialBundleReferenceFails() {
+        try {
+            addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: my.catalog.app.id.non_existing.ref",
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  libraries:",
+                "  - name: io.brooklyn.brooklyn-test-osgi-entities",
+                "  item:",
+                "    type: " + SIMPLE_ENTITY_TYPE);
+            fail();
+        } catch (NullPointerException e) {
+            Assert.assertEquals(e.getMessage(), "both name and version are required");
+        }
+        try {
+            addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: my.catalog.app.id.non_existing.ref",
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  libraries:",
+                "  - version: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_VERSION,
+                "  item:",
+                "    type: " + SIMPLE_ENTITY_TYPE);
+            fail();
+        } catch (NullPointerException e) {
+            Assert.assertEquals(e.getMessage(), "both name and version are required");
+        }
+    }
+
+    @Test
+    public void testFullBundleReference() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String itemId = "my.catalog.app.id.full_ref";
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + itemId,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  libraries:",
+            "  - name: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_NAME,
+            "    version: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_VERSION,
+            "    url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  item:",
+            "    type: " + SIMPLE_ENTITY_TYPE);
+        deleteCatalogEntity(itemId);
+    }
+
+    /**
+     * Test that the name:version contained in the OSGi bundle will
+     * override the values supplied in the YAML.
+     */
+    @Test
+    public void testFullBundleReferenceUrlMetaOverridesLocalNameVersion() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String firstItemId = "my.catalog.app.id.register_bundle";
+        String nonExistentId = "non_existent_id";
+        String nonExistentVersion = "9.9.9";
+        try {
+            addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + firstItemId,
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  libraries:",
+                "  - name: " + nonExistentId,
+                "    version: " + nonExistentVersion,
+                "    url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+                "",
+                "  item:",
+                "    type: " + SIMPLE_ENTITY_TYPE);
+            fail();
+        } catch (IllegalStateException e) {
+            assertEquals(e.getMessage(), "Bundle from " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL + " failed to install: " +
+                    "Bundle already installed as " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_NAME + ":" +
+                    OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_VERSION + " but user explicitly requested " +
+                    "CatalogBundleDto{symbolicName=" + nonExistentId + ", version=" + nonExistentVersion + ", url=" +
+                    OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL + "}");
+        }
+    }
+
+    @Test
+    public void testUpdatingItemAllowedIfSame() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String id = "my.catalog.app.id.duplicate";
+        addCatalogOSGiEntity(id);
+        addCatalogOSGiEntity(id);
+    }
+    
+    @Test(expectedExceptions = IllegalStateException.class)
+    public void testUpdatingItemFailsIfDifferent() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String id = "my.catalog.app.id.duplicate";
+        addCatalogOSGiEntity(id);
+        addCatalogOSGiEntity(id, SIMPLE_ENTITY_TYPE, true);
+    }
+
+    @Test
+    public void testForcedUpdatingItem() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String id = "my.catalog.app.id.duplicate";
+        addCatalogOSGiEntity(id);
+        forceCatalogUpdate();
+        addCatalogOSGiEntity(id);
+        deleteCatalogEntity(id);
+    }
+
+    @Test
+    public void testCreateSpecFromCatalogItem() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String id = "my.catalog.app.id.create_spec";
+        addCatalogOSGiEntity(id);
+        BrooklynTypeRegistry catalog = mgmt().getTypeRegistry();
+        RegisteredType item = catalog.get(id, TEST_VERSION);
+        EntitySpec<?> spec = catalog.createSpec(item, null, EntitySpec.class);
+        Assert.assertNotNull(spec);
+        AbstractBrooklynObjectSpec<?,?> spec2 = catalog.createSpec(item, null, null);
+        Assert.assertNotNull(spec2);
+    }
+    
+    @Test
+    public void testLoadResourceFromBundle() throws Exception {
+        String id = "resource.test";
+        addCatalogOSGiEntity(id, SIMPLE_ENTITY_TYPE);
+        String yaml =
+                "services: \n" +
+                "  - serviceType: "+ver(id);
+        Entity app = createAndStartApplication(yaml);
+        Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
+        InputStream icon = new ResourceUtils(simpleEntity).getResourceFromUrl("classpath:/org/apache/brooklyn/test/osgi/entities/icon.gif");
+        assertTrue(icon != null);
+        icon.close();
+    }
+    
+    @Test
+    public void testMissingTypeDoesNotRecurse() {
+        String symbolicName = "my.catalog.app.id.basic";
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  item:",
+            "    type: org.apache.brooklyn.entity.stock.BasicEntity");
+
+        try {
+            addCatalogItems(
+                    "brooklyn.catalog:",
+                    "  id: " + symbolicName,
+                    "  version: " + TEST_VERSION + "-update",
+                    "  itemType: entity",
+                    "  item:",
+                    "    type: " + symbolicName);
+            fail("Catalog addition expected to fail due to non-existent java type " + symbolicName);
+        } catch (IllegalStateException e) {
+            assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
+        }
+    }
+    
+    @Test
+    public void testVersionedTypeDoesNotRecurse() {
+        String symbolicName = "my.catalog.app.id.basic";
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  item:",
+            "    type: org.apache.brooklyn.entity.stock.BasicEntity");
+
+        String versionedId = CatalogUtils.getVersionedId(symbolicName, TEST_VERSION);
+        try {
+            addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + symbolicName,
+                "  version: " + TEST_VERSION + "-update",
+                "  itemType: entity",
+                "  item:",
+                "    type: " + versionedId);
+            fail("Catalog addition expected to fail due to non-existent java type " + versionedId);
+        } catch (IllegalStateException e) {
+            assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
+        }
+    }
+
+    @Test
+    public void testIndirectRecursionFails() 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",
+                "  itemType: entity",
+                "  item:",
+                "    type: "+BasicEntity.class.getName());
+
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + symbolicName + ".callee",
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  item:",
+                "    type: " + symbolicName + ".caller");
+
+        try {
+            addCatalogItems(
+                    "brooklyn.catalog:",
+                    "  id: " + symbolicName + ".caller",
+                    "  version: " + TEST_VERSION,
+                    "  itemType: entity",
+                    "  item:",
+                    "    type: " + symbolicName + ".callee");
+            fail();
+        } catch (IllegalStateException e) {
+            assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
+        }
+    }
+
+    @Test
+    public void testChildItemsDoNotRecurse() 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",
+                "  itemType: entity",
+                "  item:",
+                "    type: org.apache.brooklyn.entity.stock.BasicEntity");
+
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + symbolicName + ".callee",
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  item:",
+                "    type: " + symbolicName + ".caller");
+
+        try {
+            // TODO Only passes if include "services:" and if itemType=entity, rather than "template"!
+            // Being a child is important, triggers the case where: we allow retrying with other transformers.
+            addCatalogItems(
+                    "brooklyn.catalog:",
+                    "  id: " + symbolicName + ".caller",
+                    "  version: " + TEST_VERSION,
+                    "  itemType: entity",
+                    "  item:",
+                    "    services:",
+                    "    - type: org.apache.brooklyn.entity.stock.BasicEntity",
+                    "      brooklyn.children:",
+                    "      - type: " + symbolicName + ".callee");
+            fail();
+        } catch (IllegalStateException e) {
+            assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
+        }
+    }
+
+    @Test
+    public void testRecursiveCheckForDepenentsOnly() throws Exception {
+        String symbolicName = "my.catalog.app.id.basic";
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + symbolicName,
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  item:",
+                "    type: org.apache.brooklyn.entity.stock.BasicEntity");
+
+        createAndStartApplication(
+                "services:",
+                "- type: " + ver(symbolicName),
+                "  brooklyn.children:",
+                "  - type: " + ver(symbolicName),
+                "- type: " + ver(symbolicName),
+                "  brooklyn.children:",
+                "  - type: " + ver(symbolicName));
+    }
+
+    @Test
+    public void testOsgiNotLeakingToParent() {
+        addCatalogOSGiEntity(SIMPLE_ENTITY_TYPE);
+        try {
+            addCatalogItems(
+                    "brooklyn.catalog:",
+                    "  id: " + SIMPLE_ENTITY_TYPE,
+                    "  version: " + TEST_VERSION + "-update",
+                    "  itemType: entity",
+                    "  item:",
+                    "    type: " + SIMPLE_ENTITY_TYPE);
+            fail("Catalog addition expected to fail due to non-existent java type " + SIMPLE_ENTITY_TYPE);
+        } catch (IllegalStateException e) {
+            assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
+        }
+    }
+
+    @Test
+    public void testConfigAppliedToCatalogItem() throws Exception {
+        addCatalogOSGiEntity("test", TestEntity.class.getName());
+        String testName = "test-applies-config-on-catalog-item";
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: " + ver("test"),
+                "  brooklyn.config:",
+                "    test.confName: " + testName);
+        Entity testEntity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+    }
+
+    @Test
+    public void testFlagsAppliesToCatalogItem() throws Exception {
+        addCatalogOSGiEntity("test", TestEntity.class.getName());
+        String testName = "test-applies-config-on-catalog-item";
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: " + ver("test"),
+                "  confName: " + testName);
+        Entity testEntity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+    }
+
+    @Test
+    public void testExplicitFlagsAppliesToCatalogItem() throws Exception {
+        addCatalogOSGiEntity("test", TestEntity.class.getName());
+        String testName = "test-applies-config-on-catalog-item";
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: " + ver("test"),
+                "  brooklyn.flags:",
+                "    confName: " + testName);
+        Entity testEntity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+    }
+    
+
+    @Test
+    public void testConfigAppliedToCatalogItemImpl() throws Exception {
+        addCatalogOSGiEntity("test", TestEntityImpl.class.getName());
+        String testName = "test-applies-config-on-catalog-item";
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: " + ver("test"),
+                "  brooklyn.config:",
+                "    test.confName: " + testName);
+        Entity testEntity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+    }
+
+    @Test
+    public void testFlagsAppliesToCatalogItemImpl() throws Exception {
+        addCatalogOSGiEntity("test", TestEntityImpl.class.getName());
+        String testName = "test-applies-config-on-catalog-item";
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: " + ver("test"),
+                "  confName: " + testName);
+        Entity testEntity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+    }
+
+    @Test
+    public void testExplicitFlagsAppliesToCatalogItemImpl() throws Exception {
+        addCatalogOSGiEntity("test", TestEntityImpl.class.getName());
+        String testName = "test-applies-config-on-catalog-item";
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: " + ver("test"),
+                "  brooklyn.flags:",
+                "    confName: " + testName);
+        Entity testEntity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+    }
+
+    @Test
+    public void testHardcodedCatalog() throws Exception {
+        createAppEntitySpec(
+                "services:",
+                "- type: cluster",
+                "- type: vanilla");
+    }
+    
+    @Test(groups = "Broken")
+    public void testSameCatalogReferences() {
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  items:",
+            "  - id: referenced-entity",
+            "    item:",
+            "      services:",
+            "      - type: " + BasicEntity.class.getName(),
+            "  - id: referrer-entity",
+            "    item:",
+            "      services:",
+            "      - type: " + BasicApplication.class.getName(),
+            "        brooklyn.children:",
+            "        - type: referenced-entity",
+            "        brooklyn.config:",
+            "          spec: ",
+            "            $brooklyn:entitySpec:",
+            "              type: referenced-entity");
+
+    }
+
+    @Test
+    public void testItemWithBrooklynParameters() throws Exception {
+        String id = "inline_version.app";
+        String version = TEST_VERSION;
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + id,
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  item:",
+                "    type: " + BasicApplication.class.getName(),
+                "    brooklyn.parameters:",
+                "    - name: test.myconf",
+                "      type:  string",
+                "      default: myval",
+                "    brooklyn.config:",
+                "      myconf2: $brooklyn:config(\"test.myconf\")",
+                "      myconf2.from.root: $brooklyn:root().config(\"test.myconf\")",
+                "    brooklyn.children:",
+                "    - type: "+BasicEntity.class.getName(),
+                "      brooklyn.config:",
+                "        myconf3: $brooklyn:config(\"test.myconf\")",
+                "        myconf3.from.root: $brooklyn:root().config(\"test.myconf\")");
+
+        RegisteredType catalogItem = mgmt().getTypeRegistry().get(id, version);
+        assertEquals(catalogItem.getVersion(), version);
+        
+        String yaml = Joiner.on("\n").join(
+                "name: simple-app-yaml",
+                "location: localhost",
+                "services:",
+                "  - type: "+id+":"+version);
+        Entity app = createAndStartApplication(yaml);
+        Entity child = Iterables.getOnlyElement(app.getChildren());
+        ConfigKey<?> configKey = app.getEntityType().getConfigKey("test.myconf");
+        assertNotNull(configKey);
+        assertEquals(app.config().get(configKey), "myval");
+        assertEquals(app.config().get(ConfigKeys.newStringConfigKey("myconf2.from.root")), "myval");
+        assertEquals(child.config().get(ConfigKeys.newStringConfigKey("myconf3.from.root")), "myval");
+        assertEquals(app.config().get(ConfigKeys.newStringConfigKey("myconf2")), "myval");
+        
+        // TODO Because of https://issues.apache.org/jira/browse/BROOKLYN-267, the assertion below fails: 
+        // assertEquals(child.config().get(ConfigKeys.newStringConfigKey("myconf3")), "myval");
+        
+        mgmt().getCatalog().deleteCatalogItem(id, version);
+    }
+
+    @Test
+    public void testCreateOsgiSpecFromRegistry() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String symbolicName = "my.catalog.app.id.registry.spec";
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  name: My Catalog App",
+            "  description: My description",
+            "  icon_url: classpath://path/to/myicon.jpg",
+            "  version: " + TEST_VERSION,
+            "  libraries:",
+            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  item: " + SIMPLE_ENTITY_TYPE);
+
+        BrooklynTypeRegistry registry = mgmt().getTypeRegistry();
+        RegisteredType item = registry.get(symbolicName, TEST_VERSION);
+        AbstractBrooklynObjectSpec<?, ?> spec = registry.createSpec(item, null, null);
+        assertEquals(spec.getCatalogItemId(), ver(symbolicName));
+
+        deleteCatalogEntity(symbolicName);
+    }
+
+    @Test
+    public void testIndirectCatalogItemCanLoadResources() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String symbolicNameInner = "my.catalog.app.id.inner";
+        String symbolicNameOuter = "my.catalog.app.id.outer";
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  version: " + TEST_VERSION,
+            "  items:",
+            "  - id: " + symbolicNameInner,
+            "    name: My Catalog App",
+            "    description: My description",
+            "    icon_url: classpath://path/to/myicon.jpg",
+            "    libraries:",
+            "    - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "    item: " + SIMPLE_ENTITY_TYPE,
+            "  - id: " + symbolicNameOuter,
+            "    item: " + symbolicNameInner);
+
+        String yaml = "name: " + symbolicNameOuter + "\n" +
+                "services: \n" +
+                "  - serviceType: "+ver(symbolicNameOuter);
+        Entity app = createAndStartApplication(yaml);
+        Entity entity = app.getChildren().iterator().next();
+
+        ResourceUtils.create(entity).getResourceAsString("classpath://yaml-ref-osgi-entity.yaml");
+
+        deleteCatalogEntity(symbolicNameInner);
+        deleteCatalogEntity(symbolicNameOuter);
+    }
+
+    // The test is disabled as it fails. The entity will get assigned the outer-most catalog
+    // item which doesn't have the necessary libraries with visibility to the entity's classpath
+    // When loading resources from inside the entity then we will use the wrong BCLCS. A workaround
+    // has been implemented which explicitly adds the entity's class loader to the fallbacks.
+    @Test(groups="WIP")
+    public void testCatalogItemIdInReferencedItems() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String symbolicNameInner = "my.catalog.app.id.inner";
+        String symbolicNameOuter = "my.catalog.app.id.outer";
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  version: " + TEST_VERSION,
+            "  items:",
+            "  - id: " + symbolicNameInner,
+            "    name: My Catalog App",
+            "    description: My description",
+            "    icon_url: classpath://path/to/myicon.jpg",
+            "    libraries:",
+            "    - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "    item: " + SIMPLE_ENTITY_TYPE,
+            "  - id: " + symbolicNameOuter,
+            "    item: " + symbolicNameInner);
+
+        String yaml = "name: " + symbolicNameOuter + "\n" +
+                "services: \n" +
+                "  - serviceType: "+ver(symbolicNameOuter);
+
+        Entity app = createAndStartApplication(yaml);
+        Entity entity = app.getChildren().iterator().next();
+
+        // Fails
+        assertEquals(entity.getCatalogItemId(), ver(symbolicNameInner));
+
+        deleteCatalogEntity(symbolicNameInner);
+        deleteCatalogEntity(symbolicNameOuter);
+    }
+
+    private void registerAndLaunchAndAssertSimpleEntity(String symbolicName, String serviceType) throws Exception {
+        addCatalogOSGiEntity(symbolicName, serviceType);
+        String yaml = "name: simple-app-yaml\n" +
+                      "location: localhost\n" +
+                      "services: \n" +
+                      "  - serviceType: "+ver(symbolicName);
+        Entity app = createAndStartApplication(yaml);
+
+        Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(simpleEntity.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+
+        deleteCatalogEntity(symbolicName);
+    }
+
+    private void addCatalogEntity(String symbolicName, String serviceType) {
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  item:",
+            "    type: " + serviceType);
+    }
+
+    private void addCatalogChildEntity(String symbolicName, String serviceType) {
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  item:",
+            "    type: " + BasicEntity.class.getName(),
+            "    brooklyn.children:",
+            "    - type: " + serviceType);
+    }
+
+    private void addCatalogOSGiEntity(String symbolicName) {
+        addCatalogOSGiEntity(symbolicName, SIMPLE_ENTITY_TYPE);
+    }
+
+    private void addCatalogOSGiEntity(String symbolicName, String serviceType) {
+        addCatalogOSGiEntity(symbolicName, serviceType, false);
+    }
+    
+    private void addCatalogOSGiEntity(String symbolicName, String serviceType, boolean extraLib) {
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  name: My Catalog App",
+            "  description: My description",
+            "  icon_url: classpath://path/to/myicon.jpg",
+            "  libraries:",
+            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL +
+            (extraLib ? "\n"+"  - url: "+OsgiStandaloneTest.BROOKLYN_OSGI_TEST_A_0_1_0_URL : ""),
+            "  item:",
+            "    type: " + serviceType);
+    }
+
+    private void addCatalogChildOSGiEntityWithServicesBlock(String symbolicName, String serviceType) {
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  name: My Catalog App",
+            "  description: My description",
+            "  icon_url: classpath://path/to/myicon.jpg",
+            "  libraries:",
+            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  item:",
+            "    services:",
+            "    - type: " + BasicEntity.class.getName(),
+            "      brooklyn.children:",
+            "      - type: " + serviceType);
+    }
+}


[2/9] brooklyn-server git commit: Split CatalogYamlPolicyTest for non-osgi and osgi

Posted by sv...@apache.org.
Split CatalogYamlPolicyTest for non-osgi and osgi


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

Branch: refs/heads/master
Commit: f2ea02d5ac653a5b0d3db80c8e582cf0b096cf20
Parents: 7ff8186
Author: Aled Sage <al...@gmail.com>
Authored: Fri Feb 10 14:34:56 2017 +0000
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Feb 10 22:15:49 2017 +0000

----------------------------------------------------------------------
 .../camp/brooklyn/AbstractYamlTest.java         |   4 +
 .../catalog/CatalogOsgiYamlPolicyTest.java      | 165 +++++++++++++++++++
 .../brooklyn/catalog/CatalogYamlPolicyTest.java |  73 +++-----
 3 files changed, 191 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/f2ea02d5/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java
index 376fb5c..5581aba 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/AbstractYamlTest.java
@@ -240,6 +240,10 @@ public abstract class AbstractYamlTest {
         return countCatalogItemsMatching(RegisteredTypePredicates.IS_LOCATION);
     }
 
+    protected int countCatalogPolicies() {
+        return countCatalogItemsMatching(RegisteredTypePredicates.IS_POLICY);
+    }
+
     protected int countCatalogItemsMatching(Predicate<? super RegisteredType> filter) {
         return Iterables.size(mgmt().getTypeRegistry().getMatching(filter));
     }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/f2ea02d5/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlPolicyTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlPolicyTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlPolicyTest.java
new file mode 100644
index 0000000..e26aca6
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlPolicyTest.java
@@ -0,0 +1,165 @@
+/*
+ * 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.catalog;
+
+import static org.testng.Assert.assertEquals;
+
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.policy.Policy;
+import org.apache.brooklyn.api.typereg.RegisteredType;
+import org.apache.brooklyn.camp.brooklyn.AbstractYamlTest;
+import org.apache.brooklyn.core.config.BasicConfigKey;
+import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
+import org.apache.brooklyn.entity.stock.BasicEntity;
+import org.apache.brooklyn.test.support.TestResourceUnavailableException;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Iterables;
+
+public class CatalogOsgiYamlPolicyTest extends AbstractYamlTest {
+    private static final String SIMPLE_POLICY_TYPE = "org.apache.brooklyn.test.osgi.entities.SimplePolicy";
+
+    @Override
+    protected boolean disableOsgi() {
+        return false;
+    }
+
+    @Test
+    public void testAddCatalogItemOsgi() throws Exception {
+        assertEquals(countCatalogPolicies(), 0);
+
+        String symbolicName = "my.catalog.policy.id.load";
+        addCatalogOsgiPolicy(symbolicName, SIMPLE_POLICY_TYPE);
+
+        RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
+        assertEquals(item.getSymbolicName(), symbolicName);
+        assertEquals(countCatalogPolicies(), 1);
+
+        deleteCatalogEntity(symbolicName);
+    }
+
+    @Test
+    public void testAddCatalogItemTopLevelLegacySyntaxOsgi() throws Exception {
+        assertEquals(countCatalogPolicies(), 0);
+
+        String symbolicName = "my.catalog.policy.id.load";
+        addCatalogOsgiPolicyLegacySyntax(symbolicName, SIMPLE_POLICY_TYPE);
+
+        RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
+        assertEquals(item.getSymbolicName(), symbolicName);
+        assertEquals(countCatalogPolicies(), 1);
+
+        deleteCatalogEntity(symbolicName);
+    }
+
+    @Test
+    public void testLaunchApplicationReferencingPolicy() throws Exception {
+        String symbolicName = "my.catalog.policy.id.launch";
+        addCatalogOsgiPolicy(symbolicName, SIMPLE_POLICY_TYPE);
+        Entity app = createAndStartApplication(
+            "name: simple-app-yaml",
+            "location: localhost",
+            "services: ",
+            "  - type: " + BasicEntity.class.getName(), 
+            "    brooklyn.policies:\n" +
+            "    - type: " + ver(symbolicName),
+            "      brooklyn.config:",
+            "        config2: config2 override",
+            "        config3: config3");
+
+        Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
+        Policy policy = Iterables.getOnlyElement(simpleEntity.policies());
+        assertEquals(policy.getPolicyType().getName(), SIMPLE_POLICY_TYPE);
+        assertEquals(policy.getConfig(new BasicConfigKey<String>(String.class, "config1")), "config1");
+        assertEquals(policy.getConfig(new BasicConfigKey<String>(String.class, "config2")), "config2 override");
+        assertEquals(policy.getConfig(new BasicConfigKey<String>(String.class, "config3")), "config3");
+
+        deleteCatalogEntity(symbolicName);
+    }
+
+    @Test
+    public void testLaunchApplicationWithCatalogReferencingOtherCatalog() throws Exception {
+        String referencedSymbolicName = "my.catalog.policy.id.referenced";
+        String referrerSymbolicName = "my.catalog.policy.id.referring";
+        addCatalogOsgiPolicy(referencedSymbolicName, SIMPLE_POLICY_TYPE);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + referrerSymbolicName,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  name: My Catalog App",
+            "  description: My description",
+            "  icon_url: classpath://path/to/myicon.jpg",
+            "  item:",
+            "    type: " + BasicEntity.class.getName(),
+            "    brooklyn.policies:",
+            "    - type: " + ver(referencedSymbolicName));
+
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: "+ ver(referrerSymbolicName));
+
+        Entity entity = Iterables.getOnlyElement(app.getChildren());
+        Policy policy = Iterables.getOnlyElement(entity.policies());
+        assertEquals(policy.getPolicyType().getName(), SIMPLE_POLICY_TYPE);
+
+        deleteCatalogEntity(referencedSymbolicName);
+    }
+
+    private void addCatalogOsgiPolicy(String symbolicName, String policyType) {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  version: " + TEST_VERSION,
+            "  itemType: policy",
+            "  name: My Catalog Policy",
+            "  description: My description",
+            "  icon_url: classpath://path/to/myicon.jpg",
+            "  libraries:",
+            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  item:",
+            "    type: " + policyType,
+            "    brooklyn.config:",
+            "      config1: config1",
+            "      config2: config2");
+    }
+
+    private void addCatalogOsgiPolicyLegacySyntax(String symbolicName, String policyType) {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  name: My Catalog Policy",
+            "  description: My description",
+            "  icon_url: classpath://path/to/myicon.jpg",
+            "  version: " + TEST_VERSION,
+            "  libraries:",
+            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "",
+            "brooklyn.policies:",
+            "- type: " + policyType,
+            "  brooklyn.config:",
+            "    config1: config1",
+            "    config2: config2");
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/f2ea02d5/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlPolicyTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlPolicyTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlPolicyTest.java
index 573bb7c..e1cbbb7 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlPolicyTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlPolicyTest.java
@@ -24,29 +24,22 @@ import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.policy.Policy;
 import org.apache.brooklyn.api.typereg.RegisteredType;
 import org.apache.brooklyn.camp.brooklyn.AbstractYamlTest;
-import org.apache.brooklyn.core.catalog.CatalogPredicates;
 import org.apache.brooklyn.core.config.BasicConfigKey;
-import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
-import org.apache.brooklyn.test.support.TestResourceUnavailableException;
+import org.apache.brooklyn.core.test.policy.TestPolicy;
+import org.apache.brooklyn.entity.stock.BasicEntity;
 import org.testng.annotations.Test;
 
 import com.google.common.collect.Iterables;
 
 public class CatalogYamlPolicyTest extends AbstractYamlTest {
-    private static final String SIMPLE_POLICY_TYPE = "org.apache.brooklyn.test.osgi.entities.SimplePolicy";
-    private static final String SIMPLE_ENTITY_TYPE = "org.apache.brooklyn.test.osgi.entities.SimpleEntity";
-
-    @Override
-    protected boolean disableOsgi() {
-        return false;
-    }
+    private static final String POLICY_TYPE = TestPolicy.class.getName();
 
     @Test
     public void testAddCatalogItem() throws Exception {
         assertEquals(countCatalogPolicies(), 0);
 
         String symbolicName = "my.catalog.policy.id.load";
-        addCatalogOsgiPolicy(symbolicName, SIMPLE_POLICY_TYPE);
+        addCatalogPolicy(symbolicName, POLICY_TYPE);
 
         RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
         assertEquals(item.getSymbolicName(), symbolicName);
@@ -56,11 +49,11 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
     }
 
     @Test
-    public void testAddCatalogItemTopLevelSyntax() throws Exception {
+    public void testAddCatalogItemTopLevelLegacySyntax() throws Exception {
         assertEquals(countCatalogPolicies(), 0);
 
         String symbolicName = "my.catalog.policy.id.load";
-        addCatalogOsgiPolicyLegacySyntax(symbolicName, SIMPLE_POLICY_TYPE);
+        addCatalogPolicyLegacySyntax(symbolicName, POLICY_TYPE);
 
         RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
         assertEquals(item.getSymbolicName(), symbolicName);
@@ -72,13 +65,11 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
     @Test
     public void testLaunchApplicationReferencingPolicy() throws Exception {
         String symbolicName = "my.catalog.policy.id.launch";
-        addCatalogOsgiPolicy(symbolicName, SIMPLE_POLICY_TYPE);
+        addCatalogPolicy(symbolicName, POLICY_TYPE);
         Entity app = createAndStartApplication(
-            "name: simple-app-yaml",
-            "location: localhost",
             "services: ",
-            "  - type: org.apache.brooklyn.entity.stock.BasicEntity\n" +
-            "    brooklyn.policies:\n" +
+            "  - type: " + BasicEntity.class.getName(),
+            "    brooklyn.policies:",
             "    - type: " + ver(symbolicName),
             "      brooklyn.config:",
             "        config2: config2 override",
@@ -86,7 +77,7 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
 
         Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
         Policy policy = Iterables.getOnlyElement(simpleEntity.policies());
-        assertEquals(policy.getPolicyType().getName(), SIMPLE_POLICY_TYPE);
+        assertEquals(policy.getPolicyType().getName(), POLICY_TYPE);
         assertEquals(policy.getConfig(new BasicConfigKey<String>(String.class, "config1")), "config1");
         assertEquals(policy.getConfig(new BasicConfigKey<String>(String.class, "config2")), "config2 override");
         assertEquals(policy.getConfig(new BasicConfigKey<String>(String.class, "config3")), "config3");
@@ -97,13 +88,11 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
     @Test
     public void testLaunchApplicationReferencingPolicyTopLevelSyntax() throws Exception {
         String symbolicName = "my.catalog.policy.id.launch";
-        addCatalogOsgiPolicyLegacySyntax(symbolicName, SIMPLE_POLICY_TYPE);
+        addCatalogPolicyLegacySyntax(symbolicName, POLICY_TYPE);
         Entity app = createAndStartApplication(
-            "name: simple-app-yaml",
-            "location: localhost",
             "services: ",
-            "  - type: org.apache.brooklyn.entity.stock.BasicEntity\n" +
-            "    brooklyn.policies:\n" +
+            "  - type: " + BasicEntity.class.getName(), 
+            "    brooklyn.policies:",
             "    - type: " + ver(symbolicName),
             "      brooklyn.config:",
             "        config2: config2 override",
@@ -111,7 +100,7 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
 
         Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
         Policy policy = Iterables.getOnlyElement(simpleEntity.policies());
-        assertEquals(policy.getPolicyType().getName(), SIMPLE_POLICY_TYPE);
+        assertEquals(policy.getPolicyType().getName(), POLICY_TYPE);
         assertEquals(policy.getConfig(new BasicConfigKey<String>(String.class, "config1")), "config1");
         assertEquals(policy.getConfig(new BasicConfigKey<String>(String.class, "config2")), "config2 override");
         assertEquals(policy.getConfig(new BasicConfigKey<String>(String.class, "config3")), "config3");
@@ -123,7 +112,7 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
     public void testLaunchApplicationWithCatalogReferencingOtherCatalog() throws Exception {
         String referencedSymbolicName = "my.catalog.policy.id.referenced";
         String referrerSymbolicName = "my.catalog.policy.id.referring";
-        addCatalogOsgiPolicy(referencedSymbolicName, SIMPLE_POLICY_TYPE);
+        addCatalogPolicy(referencedSymbolicName, POLICY_TYPE);
 
         addCatalogItems(
             "brooklyn.catalog:",
@@ -133,30 +122,23 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
             "  name: My Catalog App",
             "  description: My description",
             "  icon_url: classpath://path/to/myicon.jpg",
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
             "  item:",
-            "    type: " + SIMPLE_ENTITY_TYPE,
+            "    type: " + BasicEntity.class.getName(),
             "    brooklyn.policies:",
             "    - type: " + ver(referencedSymbolicName));
 
-        String yaml = "name: simple-app-yaml\n" +
-                      "location: localhost\n" +
-                      "services: \n" +
-                      "- type: "+ ver(referrerSymbolicName);
-
-        Entity app = createAndStartApplication(yaml);
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: "+ ver(referrerSymbolicName));
 
         Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
         Policy policy = Iterables.getOnlyElement(simpleEntity.policies());
-        assertEquals(policy.getPolicyType().getName(), SIMPLE_POLICY_TYPE);
+        assertEquals(policy.getPolicyType().getName(), POLICY_TYPE);
 
         deleteCatalogEntity(referencedSymbolicName);
     }
 
-    private void addCatalogOsgiPolicy(String symbolicName, String policyType) {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
+    private void addCatalogPolicy(String symbolicName, String policyType) {
         addCatalogItems(
             "brooklyn.catalog:",
             "  id: " + symbolicName,
@@ -165,8 +147,6 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
             "  name: My Catalog Policy",
             "  description: My description",
             "  icon_url: classpath://path/to/myicon.jpg",
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
             "  item:",
             "    type: " + policyType,
             "    brooklyn.config:",
@@ -174,9 +154,7 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
             "      config2: config2");
     }
 
-    private void addCatalogOsgiPolicyLegacySyntax(String symbolicName, String policyType) {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
+    private void addCatalogPolicyLegacySyntax(String symbolicName, String policyType) {
         addCatalogItems(
             "brooklyn.catalog:",
             "  id: " + symbolicName,
@@ -184,8 +162,6 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
             "  description: My description",
             "  icon_url: classpath://path/to/myicon.jpg",
             "  version: " + TEST_VERSION,
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
             "",
             "brooklyn.policies:",
             "- type: " + policyType,
@@ -193,9 +169,4 @@ public class CatalogYamlPolicyTest extends AbstractYamlTest {
             "    config1: config1",
             "    config2: config2");
     }
-
-    private int countCatalogPolicies() {
-        return Iterables.size(mgmt().getCatalog().getCatalogItems(CatalogPredicates.IS_POLICY));
-    }
-
 }


[8/9] brooklyn-server git commit: slow tests moved to integration group

Posted by sv...@apache.org.
slow tests moved to integration group


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

Branch: refs/heads/master
Commit: 29191b76b403a5fea58e0e4fcab9a7eb926bbe8c
Parents: f73d2c8
Author: Alex Heneveld <al...@Alexs-MacBook-Pro.local>
Authored: Wed Dec 7 00:46:13 2016 +0000
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Feb 10 22:16:09 2017 +0000

----------------------------------------------------------------------
 .../java/org/apache/brooklyn/feed/function/FunctionFeedTest.java   | 2 +-
 .../policy/InvokeEffectorOnCollectionSensorChangeTest.java         | 2 +-
 .../java/org/apache/brooklyn/policy/ha/ServiceReplacerTest.java    | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/29191b76/core/src/test/java/org/apache/brooklyn/feed/function/FunctionFeedTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/brooklyn/feed/function/FunctionFeedTest.java b/core/src/test/java/org/apache/brooklyn/feed/function/FunctionFeedTest.java
index 19fc3d6..820757e 100644
--- a/core/src/test/java/org/apache/brooklyn/feed/function/FunctionFeedTest.java
+++ b/core/src/test/java/org/apache/brooklyn/feed/function/FunctionFeedTest.java
@@ -128,7 +128,7 @@ public class FunctionFeedTest extends BrooklynAppUnitTestSupport {
         Assert.assertFalse(feedAdded==feed0);
     }
     
-    @Test
+    @Test(groups="Integration")  // because slow
     public void testFeedDeDupeIgnoresSameObject() throws Exception {
         testPollsFunctionRepeatedlyToSetAttribute();
         entity.addFeed(feed);

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/29191b76/core/src/test/java/org/apache/brooklyn/policy/InvokeEffectorOnCollectionSensorChangeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/brooklyn/policy/InvokeEffectorOnCollectionSensorChangeTest.java b/core/src/test/java/org/apache/brooklyn/policy/InvokeEffectorOnCollectionSensorChangeTest.java
index 49d8c33..ac03b33 100644
--- a/core/src/test/java/org/apache/brooklyn/policy/InvokeEffectorOnCollectionSensorChangeTest.java
+++ b/core/src/test/java/org/apache/brooklyn/policy/InvokeEffectorOnCollectionSensorChangeTest.java
@@ -136,7 +136,7 @@ public class InvokeEffectorOnCollectionSensorChangeTest extends BrooklynAppUnitT
                 Predicates.<Collection<Object>>equalTo(ImmutableSet.<Object>of(4, 5)));
     }
 
-    @Test
+    @Test(groups="Integration")  // because slow
     public void testNothingHappensWhenSensorRepublishedUnchanged() {
         final ImmutableSet<Integer> input1 = ImmutableSet.of(1, 2, 3);
         testEntity.sensors().set(DEFAULT_SENSOR, input1);

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/29191b76/policy/src/test/java/org/apache/brooklyn/policy/ha/ServiceReplacerTest.java
----------------------------------------------------------------------
diff --git a/policy/src/test/java/org/apache/brooklyn/policy/ha/ServiceReplacerTest.java b/policy/src/test/java/org/apache/brooklyn/policy/ha/ServiceReplacerTest.java
index ff41f84..4aead1a 100644
--- a/policy/src/test/java/org/apache/brooklyn/policy/ha/ServiceReplacerTest.java
+++ b/policy/src/test/java/org/apache/brooklyn/policy/ha/ServiceReplacerTest.java
@@ -124,7 +124,7 @@ public class ServiceReplacerTest {
             }});
     }
 
-    @Test(invocationCount=100)
+    @Test(invocationCount=100, groups="Integration")
     public void testSetsOnFireWhenFailToReplaceMemberManyTimes() throws Exception {
         testSetsOnFireWhenFailToReplaceMember();
     }


[9/9] brooklyn-server git commit: Closes #556

Posted by sv...@apache.org.
Closes #556

Speed up tests

Starting an embedded OSGi container is slow - over half a second per `setUp()` call (and thus per test method) that uses it. It will likely be considerably slower on the jenkins build machine etc. This PR separates out those classes that include an embedded OSGi container into those tests that actually need it and those that don't. It primarily moves code around (e.g. separating out the osgi-based tests from `CatalogYamlLocationTest` into a new `CatalogOsgiYamlLocationTest`).

It also cherry-pick @ahgittin's commit that was included in https://github.com/apache/brooklyn-server/pull/480 (which marks some more slow tests as "integration").

If you're wondering why just moving the code has resulted in a bunch of extra lines (`+1,956 \u22121,038` according to the "Files changed" summary), there are two primary reasons: first there are 6 new classes which have the apache header and a bunch of imports; second the `CatalogOsgiYamlEntityTest` repeats some tests that are done in `CatalogOsgiEntityTest`. The reason is that there are subtleties in how bundles are used to load catalog items (especially when composing multiple items, and sub-typing). It's therefore worth having slow versions of some of these tests that use actual OSGi bundles.


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

Branch: refs/heads/master
Commit: 0f649fe175d6dfb1327d1db5bec420d71ac2a4c1
Parents: 6571bab 29191b7
Author: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Authored: Mon Feb 13 08:26:05 2017 +0200
Committer: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Committed: Mon Feb 13 08:26:05 2017 +0200

----------------------------------------------------------------------
 .../camp/brooklyn/AbstractYamlTest.java         |  20 +
 .../camp/brooklyn/ReferencedOsgiYamlTest.java   | 230 +++++
 .../camp/brooklyn/ReferencedYamlTest.java       | 191 ----
 .../catalog/CatalogOsgiYamlEntityTest.java      | 861 +++++++++++++++++++
 .../catalog/CatalogOsgiYamlLocationTest.java    | 198 +++++
 .../catalog/CatalogOsgiYamlPolicyTest.java      | 165 ++++
 .../catalog/CatalogOsgiYamlTemplateTest.java    | 113 +++
 .../brooklyn/catalog/CatalogYamlEntityTest.java | 796 +++++------------
 .../catalog/CatalogYamlLocationTest.java        |  81 +-
 .../brooklyn/catalog/CatalogYamlPolicyTest.java |  73 +-
 .../catalog/CatalogYamlTemplateTest.java        |  43 +-
 .../catalog/SpecParameterParsingOsgiTest.java   | 116 +++
 .../catalog/SpecParameterParsingTest.java       | 101 +--
 .../feed/function/FunctionFeedTest.java         |   2 +-
 ...okeEffectorOnCollectionSensorChangeTest.java |   2 +-
 .../brooklyn/policy/ha/ServiceReplacerTest.java |   2 +-
 16 files changed, 1956 insertions(+), 1038 deletions(-)
----------------------------------------------------------------------



[3/9] brooklyn-server git commit: Split CatalogYamlTemplateTest for non-osgi and osgi

Posted by sv...@apache.org.
Split CatalogYamlTemplateTest for non-osgi and osgi


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

Branch: refs/heads/master
Commit: 27ad37cd39b7b133a454f6e9b5d05eaedc0dc3d6
Parents: f2ea02d
Author: Aled Sage <al...@gmail.com>
Authored: Fri Feb 10 14:50:26 2017 +0000
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Feb 10 22:16:02 2017 +0000

----------------------------------------------------------------------
 .../catalog/CatalogOsgiYamlTemplateTest.java    | 113 +++++++++++++++++++
 .../catalog/CatalogYamlTemplateTest.java        |  43 +++----
 2 files changed, 130 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/27ad37cd/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlTemplateTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlTemplateTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlTemplateTest.java
new file mode 100644
index 0000000..8b2a561
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogOsgiYamlTemplateTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.catalog;
+
+import static org.testng.Assert.assertEquals;
+
+import java.util.List;
+
+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.typereg.RegisteredType;
+import org.apache.brooklyn.camp.brooklyn.AbstractYamlTest;
+import org.apache.brooklyn.core.mgmt.BrooklynTags;
+import org.apache.brooklyn.core.mgmt.BrooklynTags.NamedStringTag;
+import org.apache.brooklyn.core.mgmt.EntityManagementUtils;
+import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
+import org.apache.brooklyn.core.typereg.RegisteredTypePredicates;
+import org.apache.brooklyn.core.typereg.RegisteredTypes;
+import org.apache.brooklyn.test.Asserts;
+import org.apache.brooklyn.test.support.TestResourceUnavailableException;
+import org.apache.brooklyn.util.osgi.OsgiTestResources;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Iterables;
+
+
+public class CatalogOsgiYamlTemplateTest extends AbstractYamlTest {
+    
+    private static final String SIMPLE_ENTITY_TYPE = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY;
+
+    @Override
+    protected boolean disableOsgi() {
+        return false;
+    }
+
+    @Test
+    public void testAddCatalogItemOsgi() throws Exception {
+        RegisteredType item = makeItem("t1", SIMPLE_ENTITY_TYPE);
+        Assert.assertTrue(RegisteredTypePredicates.IS_APPLICATION.apply(item), "item: "+item);
+        String yaml = RegisteredTypes.getImplementationDataStringForSpec(item);
+        Assert.assertTrue(yaml.indexOf("sample comment")>=0,
+            "YAML did not include original comments; it was:\n"+yaml);
+        Assert.assertFalse(yaml.indexOf("description")>=0,
+            "YAML included metadata which should have been excluded; it was:\n"+yaml);
+
+        // Confirm can deploy an app using this template catalog item
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: t1");
+        waitForApplicationTasks(app);
+        
+        Entity entity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(entity.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+        
+        deleteCatalogEntity("t1");
+    }
+
+    @Test
+    public void testMetadataOnSpecCreatedFromItem() throws Exception {
+        makeItem("t1", SIMPLE_ENTITY_TYPE);
+        EntitySpec<? extends Application> spec = EntityManagementUtils.createEntitySpecForApplication(mgmt(), 
+            "services: [ { type: t1 } ]\n" +
+            "location: localhost");
+        
+        List<NamedStringTag> yamls = BrooklynTags.findAll(BrooklynTags.YAML_SPEC_KIND, spec.getTags());
+        Assert.assertEquals(yamls.size(), 1, "Expected 1 yaml tag; instead had: "+yamls);
+        String yaml = Iterables.getOnlyElement(yamls).getContents();
+        Asserts.assertStringContains(yaml, "services:", "t1", "localhost");
+        
+        EntitySpec<?> child = Iterables.getOnlyElement( spec.getChildren() );
+        Assert.assertEquals(child.getType().getName(), SIMPLE_ENTITY_TYPE);
+        Assert.assertEquals(child.getCatalogItemId(), "t1:"+TEST_VERSION);
+    }
+    
+    private RegisteredType makeItem(String symbolicName, String templateType) {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+        
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + symbolicName,
+            "  itemType: template",
+            "  name: My Catalog App",
+            "  description: My description",
+            "  icon_url: classpath://path/to/myicon.jpg",
+            "  version: " + TEST_VERSION,
+            "  libraries:",
+            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  item:",
+            "    services:",
+            "    # this sample comment should be included",
+            "    - type: " + templateType);
+
+        return mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/27ad37cd/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlTemplateTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlTemplateTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlTemplateTest.java
index 954d38d..cb585e0 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlTemplateTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlTemplateTest.java
@@ -32,36 +32,27 @@ import org.apache.brooklyn.core.entity.Entities;
 import org.apache.brooklyn.core.mgmt.BrooklynTags;
 import org.apache.brooklyn.core.mgmt.BrooklynTags.NamedStringTag;
 import org.apache.brooklyn.core.mgmt.EntityManagementUtils;
-import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
 import org.apache.brooklyn.core.test.entity.TestEntity;
 import org.apache.brooklyn.core.typereg.RegisteredTypePredicates;
 import org.apache.brooklyn.core.typereg.RegisteredTypes;
 import org.apache.brooklyn.entity.group.DynamicCluster;
 import org.apache.brooklyn.entity.stock.BasicApplication;
 import org.apache.brooklyn.test.Asserts;
-import org.apache.brooklyn.test.support.TestResourceUnavailableException;
 import org.apache.brooklyn.util.core.config.ConfigBag;
-import org.apache.brooklyn.util.osgi.OsgiTestResources;
 import org.testng.Assert;
 import org.testng.TestListenerAdapter;
 import org.testng.TestNG;
 import org.testng.annotations.Test;
 
+import com.google.common.base.Joiner;
 import com.google.common.collect.Iterables;
 
 
 public class CatalogYamlTemplateTest extends AbstractYamlTest {
     
-    private static final String SIMPLE_ENTITY_TYPE = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY;
-
-    @Override
-    protected boolean disableOsgi() {
-        return false;
-    }
-
     @Test
     public void testAddCatalogItem() throws Exception {
-        RegisteredType item = makeItem();
+        RegisteredType item = addCatalogItem("t1", TestEntity.class.getName());
         Assert.assertTrue(RegisteredTypePredicates.IS_APPLICATION.apply(item), "item: "+item);
         String yaml = RegisteredTypes.getImplementationDataStringForSpec(item);
         Assert.assertTrue(yaml.indexOf("sample comment")>=0,
@@ -74,10 +65,10 @@ public class CatalogYamlTemplateTest extends AbstractYamlTest {
 
     @Test
     public void testAddCatalogItemAndCheckSource() throws Exception {
-        // this will fail with the Eclipse TestNG plugin -- use the static main instead to run in eclipse!
+        // this may fail with old Eclipse TestNG plugins -- use the static main instead to run in eclipse!
         // see Yamls.KnownClassVersionException for details
         
-        RegisteredType item = makeItem();
+        RegisteredType item = addCatalogItem("t1", TestEntity.class.getName());
         String yaml = RegisteredTypes.getImplementationDataStringForSpec(item);
         Assert.assertTrue(yaml.indexOf("sample comment")>=0,
             "YAML did not include original comments; it was:\n"+yaml);
@@ -201,10 +192,14 @@ public class CatalogYamlTemplateTest extends AbstractYamlTest {
 
     @Test
     public void testMetadataOnSpecCreatedFromItem() throws Exception {
-        makeItem();
-        EntitySpec<? extends Application> spec = EntityManagementUtils.createEntitySpecForApplication(mgmt(), 
-            "services: [ { type: t1 } ]\n" +
-            "location: localhost");
+        addCatalogItem("t1", TestEntity.class.getName());
+
+        EntitySpec<? extends Application> spec = EntityManagementUtils.createEntitySpecForApplication(
+                mgmt(),
+                Joiner.on("\n").join(
+                        "location: localhost",
+                        "services:",
+                        "- type: t1"));
         
         List<NamedStringTag> yamls = BrooklynTags.findAll(BrooklynTags.YAML_SPEC_KIND, spec.getTags());
         Assert.assertEquals(yamls.size(), 1, "Expected 1 yaml tag; instead had: "+yamls);
@@ -212,7 +207,7 @@ public class CatalogYamlTemplateTest extends AbstractYamlTest {
         Asserts.assertStringContains(yaml, "services:", "t1", "localhost");
         
         EntitySpec<?> child = Iterables.getOnlyElement( spec.getChildren() );
-        Assert.assertEquals(child.getType().getName(), SIMPLE_ENTITY_TYPE);
+        Assert.assertEquals(child.getType().getName(), TestEntity.class.getName());
         Assert.assertEquals(child.getCatalogItemId(), "t1:"+TEST_VERSION);
     }
     
@@ -255,25 +250,21 @@ public class CatalogYamlTemplateTest extends AbstractYamlTest {
         Assert.assertEquals(spec.getCatalogItemId(), "app1r:1");
     }
     
-    private RegisteredType makeItem() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-        
+    private RegisteredType addCatalogItem(String symbolicName, String templateType) {
         addCatalogItems(
             "brooklyn.catalog:",
-            "  id: t1",
+            "  id: " + symbolicName,
             "  itemType: template",
             "  name: My Catalog App",
             "  description: My description",
             "  icon_url: classpath://path/to/myicon.jpg",
             "  version: " + TEST_VERSION,
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
             "  item:",
             "    services:",
             "    # this sample comment should be included",
-            "    - type: " + SIMPLE_ENTITY_TYPE);
+            "    - type: " + templateType);
 
-        return mgmt().getTypeRegistry().get("t1", TEST_VERSION);
+        return mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
     }
 
     // convenience for running in eclipse when the TestNG plugin drags in old version of snake yaml


[6/9] brooklyn-server git commit: Split SpecParameterParsingTest for non-osgi and osgi

Posted by sv...@apache.org.
Split SpecParameterParsingTest for non-osgi and osgi

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

Branch: refs/heads/master
Commit: f73d2c81402d098a184dbfe2e8af0dceb9bf6e64
Parents: 8f0021e
Author: Aled Sage <al...@gmail.com>
Authored: Fri Feb 10 21:56:32 2017 +0000
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Feb 10 22:16:09 2017 +0000

----------------------------------------------------------------------
 .../catalog/SpecParameterParsingOsgiTest.java   | 116 +++++++++++++++++++
 .../catalog/SpecParameterParsingTest.java       | 101 +---------------
 2 files changed, 118 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/f73d2c81/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/SpecParameterParsingOsgiTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/SpecParameterParsingOsgiTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/SpecParameterParsingOsgiTest.java
new file mode 100644
index 0000000..814aed7
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/SpecParameterParsingOsgiTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.catalog;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.brooklyn.api.entity.EntitySpec;
+import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
+import org.apache.brooklyn.api.objs.SpecParameter;
+import org.apache.brooklyn.api.typereg.RegisteredType;
+import org.apache.brooklyn.camp.brooklyn.AbstractYamlTest;
+import org.apache.brooklyn.core.config.ConfigKeys;
+import org.apache.brooklyn.core.entity.AbstractEntity;
+import org.apache.brooklyn.core.objs.BasicSpecParameter;
+import org.apache.brooklyn.entity.stock.BasicApplication;
+import org.apache.brooklyn.test.support.TestResourceUnavailableException;
+import org.apache.brooklyn.util.osgi.OsgiTestResources;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.ImmutableSet;
+
+public class SpecParameterParsingOsgiTest extends AbstractYamlTest {
+
+    private static final int NUM_APP_DEFAULT_CONFIG_KEYS = SpecParameterUnwrappingTest.NUM_APP_DEFAULT_CONFIG_KEYS;
+    
+    @Override
+    protected boolean disableOsgi() {
+        return false;
+    }
+
+    @Test
+    public void testOsgiType() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String itemId = ver("test.inputs", "0.0.1");
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: test.inputs",
+                "  version: 0.0.1",
+                "  itemType: entity",
+                "  libraries:",
+                "  - classpath://" + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_PATH,
+                "  item: ",
+                "    type: "+ BasicApplication.class.getName(),
+                "    brooklyn.parameters:",
+                "    - name: simple",
+                "      type: " + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
+        AbstractBrooklynObjectSpec<?,?> spec = createSpec(itemId);
+        List<SpecParameter<?>> inputs = spec.getParameters();
+        assertEquals(inputs.size(), NUM_APP_DEFAULT_CONFIG_KEYS + 1, "inputs="+inputs);
+        SpecParameter<?> firstInput = inputs.get(0);
+        assertEquals(firstInput.getLabel(), "simple");
+        assertTrue(firstInput.isPinned());
+        assertEquals(firstInput.getConfigKey().getName(), "simple");
+        assertEquals(firstInput.getConfigKey().getTypeToken().getRawType().getName(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
+    }
+
+    @Test
+    public void testOsgiClassScanned() {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiTestResources.BROOKLYN_TEST_MORE_ENTITIES_V2_PATH);
+
+        addCatalogItems("brooklyn.catalog:",
+            "    items:",
+            "    - scanJavaAnnotations: true",
+            "      version: 2.0.test_java",
+            "      libraries:",
+            "      - classpath://" + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_PATH,
+            "      - classpath://" + OsgiTestResources.BROOKLYN_TEST_MORE_ENTITIES_V2_PATH);
+
+        RegisteredType item = mgmt().getTypeRegistry().get(OsgiTestResources.BROOKLYN_TEST_MORE_ENTITIES_MORE_ENTITY);
+        assertEquals(item.getVersion(), "2.0.test_java");
+        assertEquals(item.getLibraries().size(), 2);
+        AbstractBrooklynObjectSpec<?,?> spec = createSpec(item);
+        List<SpecParameter<?>> inputs = spec.getParameters();
+        if (inputs.isEmpty()) Assert.fail("no inputs (if you're in the IDE, mvn clean install may need to be run to rebuild osgi test JARs)");
+        
+        Set<SpecParameter<?>> actual = ImmutableSet.copyOf(inputs);
+        Set<SpecParameter<?>> expected = ImmutableSet.<SpecParameter<?>>of(
+                new BasicSpecParameter<>("more_config", false, ConfigKeys.newStringConfigKey("more_config")),
+                new BasicSpecParameter<>(AbstractEntity.DEFAULT_DISPLAY_NAME.getName(), false, AbstractEntity.DEFAULT_DISPLAY_NAME));
+        assertEquals(actual, expected);
+    }
+
+    private AbstractBrooklynObjectSpec<?, ?> createSpec(String itemId) {
+        RegisteredType item = mgmt().getTypeRegistry().get(itemId);
+        Assert.assertNotNull(item, "Could not load: "+itemId);
+        return createSpec(item);
+    }
+    
+    private AbstractBrooklynObjectSpec<?, ?> createSpec(RegisteredType item) {
+        return mgmt().getTypeRegistry().createSpec(item, null, EntitySpec.class);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/f73d2c81/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/SpecParameterParsingTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/SpecParameterParsingTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/SpecParameterParsingTest.java
index 9ce057b..af5ce01 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/SpecParameterParsingTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/SpecParameterParsingTest.java
@@ -19,44 +19,25 @@
 package org.apache.brooklyn.camp.brooklyn.catalog;
 
 import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertTrue;
 
 import java.util.List;
-import java.util.Set;
 
-import org.apache.brooklyn.api.catalog.CatalogItem;
 import org.apache.brooklyn.api.entity.EntitySpec;
-import org.apache.brooklyn.api.internal.AbstractBrooklynObjectSpec;
 import org.apache.brooklyn.api.objs.SpecParameter;
-import org.apache.brooklyn.api.typereg.RegisteredType;
 import org.apache.brooklyn.camp.brooklyn.AbstractYamlTest;
-import org.apache.brooklyn.core.config.ConfigKeys;
-import org.apache.brooklyn.core.entity.AbstractEntity;
-import org.apache.brooklyn.core.objs.BasicSpecParameter;
 import org.apache.brooklyn.entity.stock.BasicApplication;
-import org.apache.brooklyn.test.support.TestResourceUnavailableException;
-import org.apache.brooklyn.util.osgi.OsgiTestResources;
-import org.testng.Assert;
 import org.testng.annotations.Test;
 
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
 import com.google.common.reflect.TypeToken;
 
 public class SpecParameterParsingTest  extends AbstractYamlTest {
 
     private static final int NUM_APP_DEFAULT_CONFIG_KEYS = SpecParameterUnwrappingTest.NUM_APP_DEFAULT_CONFIG_KEYS;
     
-    @Override
-    protected boolean disableOsgi() {
-        return false;
-    }
-
     @Test
     public void testYamlInputsParsed() {
-        String itemId = add(
+        String itemId = ver("test.inputs", "0.0.1");
+        addCatalogItems(
                 "brooklyn.catalog:",
                 "  id: test.inputs",
                 "  version: 0.0.1",
@@ -90,82 +71,4 @@ public class SpecParameterParsingTest  extends AbstractYamlTest {
         assertEquals(thirdInput.getConfigKey().getName(), "third_input");
         assertEquals(thirdInput.getConfigKey().getTypeToken(), TypeToken.of(Integer.class));
     }
-
-    @Test
-    public void testOsgiType() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String itemId = add(
-                "brooklyn.catalog:",
-                "  id: test.inputs",
-                "  version: 0.0.1",
-                "  itemType: entity",
-                "  libraries:",
-                "  - classpath://" + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_PATH,
-                "  item: ",
-                "    type: "+ BasicApplication.class.getName(),
-                "    brooklyn.parameters:",
-                "    - name: simple",
-                "      type: " + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
-        AbstractBrooklynObjectSpec<?,?> spec = createSpec(itemId);
-        List<SpecParameter<?>> inputs = spec.getParameters();
-        assertEquals(inputs.size(), NUM_APP_DEFAULT_CONFIG_KEYS + 1, "inputs="+inputs);
-        SpecParameter<?> firstInput = inputs.get(0);
-        assertEquals(firstInput.getLabel(), "simple");
-        assertTrue(firstInput.isPinned());
-        assertEquals(firstInput.getConfigKey().getName(), "simple");
-        assertEquals(firstInput.getConfigKey().getTypeToken().getRawType().getName(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
-    }
-
-    @Test
-    public void testOsgiClassScanned() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiTestResources.BROOKLYN_TEST_MORE_ENTITIES_V2_PATH);
-
-        addMulti("brooklyn.catalog:",
-            "    items:",
-            "    - scanJavaAnnotations: true",
-            "      version: 2.0.test_java",
-            "      libraries:",
-            "      - classpath://" + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_PATH,
-            "      - classpath://" + OsgiTestResources.BROOKLYN_TEST_MORE_ENTITIES_V2_PATH);
-
-        RegisteredType item = mgmt().getTypeRegistry().get(OsgiTestResources.BROOKLYN_TEST_MORE_ENTITIES_MORE_ENTITY);
-        assertEquals(item.getVersion(), "2.0.test_java");
-        assertEquals(item.getLibraries().size(), 2);
-        AbstractBrooklynObjectSpec<?,?> spec = createSpec(item);
-        List<SpecParameter<?>> inputs = spec.getParameters();
-        if (inputs.isEmpty()) Assert.fail("no inputs (if you're in the IDE, mvn clean install may need to be run to rebuild osgi test JARs)");
-        
-        Set<SpecParameter<?>> actual = ImmutableSet.copyOf(inputs);
-        Set<SpecParameter<?>> expected = ImmutableSet.<SpecParameter<?>>of(
-                new BasicSpecParameter<>("more_config", false, ConfigKeys.newStringConfigKey("more_config")),
-                new BasicSpecParameter<>(AbstractEntity.DEFAULT_DISPLAY_NAME.getName(), false, AbstractEntity.DEFAULT_DISPLAY_NAME));
-        assertEquals(actual, expected);
-    }
-
-    private String add(String... def) {
-        return Iterables.getOnlyElement(addMulti(def));
-    }
-
-    private Iterable<String> addMulti(String... def) {
-        return Iterables.transform(catalog.addItems(Joiner.on('\n').join(def)),
-            new Function<CatalogItem<?,?>, String>() {
-                @Override
-                public String apply(CatalogItem<?, ?> input) {
-                    return input.getId();
-                }
-            });
-    }
-
-    private AbstractBrooklynObjectSpec<?, ?> createSpec(String itemId) {
-        RegisteredType item = mgmt().getTypeRegistry().get(itemId);
-        Assert.assertNotNull(item, "Could not load: "+itemId);
-        return createSpec(item);
-    }
-    
-    private AbstractBrooklynObjectSpec<?, ?> createSpec(RegisteredType item) {
-        return mgmt().getTypeRegistry().createSpec(item, null, EntitySpec.class);
-    }
-
 }


[4/9] brooklyn-server git commit: Split CatalogYamlEntityTest for non-osgi and osgi

Posted by sv...@apache.org.
http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/c5ae189c/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java
index f9868f8..fc9347a 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/catalog/CatalogYamlEntityTest.java
@@ -18,16 +18,12 @@
  */
 package org.apache.brooklyn.camp.brooklyn.catalog;
 
+import static com.google.common.base.Preconditions.checkNotNull;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNotNull;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 
-import java.io.InputStream;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.brooklyn.api.catalog.BrooklynCatalog;
 import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.entity.EntitySpec;
@@ -38,44 +34,25 @@ import org.apache.brooklyn.camp.brooklyn.AbstractYamlTest;
 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.mgmt.osgi.OsgiStandaloneTest;
 import org.apache.brooklyn.core.test.entity.TestEntity;
 import org.apache.brooklyn.core.test.entity.TestEntityImpl;
 import org.apache.brooklyn.core.typereg.RegisteredTypes;
 import org.apache.brooklyn.entity.stock.BasicApplication;
 import org.apache.brooklyn.entity.stock.BasicEntity;
-import org.apache.brooklyn.test.support.TestResourceUnavailableException;
-import org.apache.brooklyn.util.collections.MutableList;
-import org.apache.brooklyn.util.core.ResourceUtils;
 import org.apache.brooklyn.util.exceptions.Exceptions;
-import org.apache.brooklyn.util.osgi.OsgiTestResources;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 
 
 public class CatalogYamlEntityTest extends AbstractYamlTest {
     
-    private static final String SIMPLE_ENTITY_TYPE = OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY;
-
-    @Override
-    protected boolean disableOsgi() {
-        return false;
-    }
-
     @Test
     public void testAddCatalogItemVerySimple() throws Exception {
         String symbolicName = "my.catalog.app.id.load";
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + symbolicName,
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  item:",
-            "    type: "+ BasicEntity.class.getName());
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), BasicEntity.class.getName());
 
         RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
         String planYaml = RegisteredTypes.getImplementationDataStringForSpec(item);
@@ -83,23 +60,10 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
 
         deleteCatalogEntity(symbolicName);
     }
-    @Test
-    public void testAddCatalogItem() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String symbolicName = "my.catalog.app.id.load";
-        addCatalogOSGiEntity(symbolicName);
-        RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
-        assertEquals(item.getSymbolicName(), symbolicName);
-
-        deleteCatalogEntity(symbolicName);
-    }
 
     // Legacy / backwards compatibility: should always specify itemType
     @Test
     public void testAddCatalogItemAsStringWithoutItemType() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String symbolicName = "my.catalog.app.id.load";
         addCatalogItems(
             "brooklyn.catalog:",
@@ -108,9 +72,7 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
             "  name: My Catalog App",
             "  description: My description",
             "  icon_url: classpath://path/to/myicon.jpg",
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  item: " + SIMPLE_ENTITY_TYPE);
+            "  item: " + BasicEntity.class.getName());
 
         RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
         assertEquals(item.getSymbolicName(), symbolicName);
@@ -120,20 +82,13 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
 
     @Test
     public void testAddCatalogItemTypeExplicitTypeAsString() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String symbolicName = "my.catalog.app.id.load";
         addCatalogItems(
             "brooklyn.catalog:",
             "  id: " + symbolicName,
             "  version: " + TEST_VERSION,
             "  itemType: entity",
-            "  name: My Catalog App",
-            "  description: My description",
-            "  icon_url: classpath://path/to/myicon.jpg",
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  item: " + SIMPLE_ENTITY_TYPE);
+            "  item: " + BasicEntity.class.getName());
 
         RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
         assertEquals(item.getSymbolicName(), symbolicName);
@@ -143,21 +98,14 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
 
     @Test
     public void testAddCatalogItemLegacySyntax() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String symbolicName = "my.catalog.app.id.load";
         addCatalogItems(
             "brooklyn.catalog:",
             "  id: " + symbolicName,
-            "  name: My Catalog App",
-            "  description: My description",
-            "  icon_url: classpath://path/to/myicon.jpg",
             "  version: " + TEST_VERSION,
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
             "",
             "services:",
-            "- type: " + SIMPLE_ENTITY_TYPE);
+            "- type: " + BasicEntity.class.getName());
 
         RegisteredType item = mgmt().getTypeRegistry().get(symbolicName, TEST_VERSION);
         assertEquals(item.getSymbolicName(), symbolicName);
@@ -168,17 +116,13 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
     // Legacy / backwards compatibility: should use id
     @Test
     public void testAddCatalogItemUsingNameInsteadOfIdWithoutVersion() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String id = "unversioned.app";
         addCatalogItems(
             "brooklyn.catalog:",
             "  name: " + id,
             "  itemType: entity",
-            "  libraries:",
-            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
             "  item:",
-            "    type: "+ SIMPLE_ENTITY_TYPE);
+            "    type: "+ BasicEntity.class.getName());
         RegisteredType catalogItem = mgmt().getTypeRegistry().get(id, BrooklynCatalog.DEFAULT_VERSION);
         assertEquals(catalogItem.getVersion(), "0.0.0.SNAPSHOT");
         mgmt().getCatalog().deleteCatalogItem(id, "0.0.0.SNAPSHOT");
@@ -187,17 +131,13 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
     // Legacy / backwards compatibility: should use id
     @Test
     public void testAddCatalogItemUsingNameInsteadOfIdWithInlinedVersion() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String id = "inline_version.app";
         addCatalogItems(
             "brooklyn.catalog:",
             "  name: " + id+":"+TEST_VERSION,
             "  itemType: entity",
-            "  libraries:",
-            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
             "services:",
-            "- type: " + SIMPLE_ENTITY_TYPE);
+            "- type: " + BasicEntity.class.getName());
         RegisteredType catalogItem = mgmt().getTypeRegistry().get(id, TEST_VERSION);
         assertEquals(catalogItem.getVersion(), TEST_VERSION);
         mgmt().getCatalog().deleteCatalogItem(id, TEST_VERSION);
@@ -205,51 +145,62 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
 
     @Test
     public void testLaunchApplicationReferencingCatalog() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+        String symbolicName = "myitem";
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), TestEntity.class.getName());
 
-        String symbolicName = "my.catalog.app.id.launch";
-        registerAndLaunchAndAssertSimpleEntity(symbolicName, SIMPLE_ENTITY_TYPE);
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: "+ver(symbolicName, TEST_VERSION));
+
+        Entity entity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(entity.getEntityType().getName(), TestEntity.class.getName());
+
+        deleteCatalogEntity(symbolicName);
     }
 
     @Test
     public void testLaunchApplicationUnversionedCatalogReference() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+        String symbolicName = "myitem";
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), TestEntity.class.getName());
 
-        String symbolicName = "my.catalog.app.id.fail";
-        addCatalogOSGiEntity(symbolicName, SIMPLE_ENTITY_TYPE);
-        try {
-            String yaml = "name: simple-app-yaml\n" +
-                          "location: localhost\n" +
-                          "services: \n" +
-                          "  - serviceType: " + symbolicName;
-            createAndStartApplication(yaml);
-        } finally {
-            deleteCatalogEntity(symbolicName);
-        }
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: "+symbolicName);
+
+        Entity entity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(entity.getEntityType().getName(), TestEntity.class.getName());
+
+        deleteCatalogEntity(symbolicName);
     }
 
     @Test
     public void testLaunchApplicationWithCatalogReferencingOtherCatalog() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String referencedSymbolicName = "my.catalog.app.id.referenced";
         String referrerSymbolicName = "my.catalog.app.id.referring";
-        addCatalogOSGiEntities(ImmutableMap.of(
-                referencedSymbolicName, SIMPLE_ENTITY_TYPE, 
-                referrerSymbolicName, ver(referencedSymbolicName)));
-
+        
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  name: My Catalog App",
+                "  description: My description",
+                "  icon_url: classpath://path/to/myicon.jpg",
+                "  version: " + TEST_VERSION,
+                "  items:",
+                "  - id: " + referencedSymbolicName,
+                "    item:",
+                "      type: " + TestEntity.class.getName(),
+                "  - id: " + referrerSymbolicName,
+                "    item:",
+                "      type: " + ver(referencedSymbolicName, TEST_VERSION));
+          
         RegisteredType referrer = mgmt().getTypeRegistry().get(referrerSymbolicName, TEST_VERSION);
         String planYaml = RegisteredTypes.getImplementationDataStringForSpec(referrer);
         Assert.assertTrue(planYaml.indexOf("services")>=0, "expected services in: "+planYaml);
         
-        String yaml = "name: simple-app-yaml\n" +
-                      "location: localhost\n" +
-                      "services: \n" +
-                      "  - type: " + ver(referrerSymbolicName);
-        Entity app = createAndStartApplication(yaml);
+        Entity app = createAndStartApplication("services:",
+                      "- type: " + ver(referrerSymbolicName, TEST_VERSION));
 
-        Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
-        assertEquals(simpleEntity.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+        Entity entity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(entity.getEntityType().getName(), TestEntity.class.getName());
 
         deleteCatalogEntity(referencedSymbolicName);
         deleteCatalogEntity(referrerSymbolicName);
@@ -257,21 +208,17 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
 
     @Test
     public void testLaunchApplicationWithCatalogReferencingOtherCatalogInTwoSteps() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String referencedSymbolicName = "my.catalog.app.id.referenced";
         String referrerSymbolicName = "my.catalog.app.id.referring";
-        addCatalogOSGiEntity(referencedSymbolicName, SIMPLE_ENTITY_TYPE);
-        addCatalogOSGiEntity(referrerSymbolicName, ver(referencedSymbolicName));
 
-        String yaml = "name: simple-app-yaml\n" +
-                      "location: localhost\n" +
-                      "services: \n" +
-                      "  - serviceType: " + ver(referrerSymbolicName);
-        Entity app = createAndStartApplication(yaml);
+        addCatalogEntity(IdAndVersion.of(referencedSymbolicName, TEST_VERSION), TestEntity.class.getName());
+        addCatalogEntity(IdAndVersion.of(referrerSymbolicName, TEST_VERSION), ver(referencedSymbolicName, TEST_VERSION));
 
-        Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
-        assertEquals(simpleEntity.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+        Entity app = createAndStartApplication("services:",
+                      "- type: " + ver(referrerSymbolicName, TEST_VERSION));
+
+        Entity entity = Iterables.getOnlyElement(app.getChildren());
+        assertEquals(entity.getEntityType().getName(), TestEntity.class.getName());
 
         deleteCatalogEntity(referencedSymbolicName);
         deleteCatalogEntity(referrerSymbolicName);
@@ -279,33 +226,34 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
 
     @Test
     public void testLaunchApplicationChildWithCatalogReferencingOtherCatalog() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String referencedSymbolicName = "my.catalog.app.id.child.referenced";
         String referrerSymbolicName = "my.catalog.app.id.child.referring";
-        addCatalogOSGiEntity(referencedSymbolicName, SIMPLE_ENTITY_TYPE);
-        addCatalogChildOSGiEntity(referrerSymbolicName, ver(referencedSymbolicName));
+        
+        addCatalogEntity(IdAndVersion.of(referencedSymbolicName, TEST_VERSION), TestEntity.class.getName());
 
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + referrerSymbolicName,
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  item:",
+                "    services:",
+                "    - type: " + BasicEntity.class.getName(),
+                "      brooklyn.children:",
+                "      - type: " + ver(referencedSymbolicName, TEST_VERSION));
+        
         Entity app = createAndStartApplication(
-            "name: simple-app-yaml",
-            "location: localhost",
-            "services:",
-            "- type: "+BasicEntity.class.getName(),
-            "  brooklyn.children:",
-            "  - type: " + ver(referrerSymbolicName));
+                "services:",
+                "- type: "+BasicEntity.class.getName(),
+                "  brooklyn.children:",
+                "  - type: " + ver(referrerSymbolicName));
 
-        Collection<Entity> children = app.getChildren();
-        assertEquals(children.size(), 1);
-        Entity child = Iterables.getOnlyElement(children);
+        Entity child = Iterables.getOnlyElement(app.getChildren());
         assertEquals(child.getEntityType().getName(), BasicEntity.class.getName());
-        Collection<Entity> grandChildren = child.getChildren();
-        assertEquals(grandChildren.size(), 1);
-        Entity grandChild = Iterables.getOnlyElement(grandChildren);
+        Entity grandChild = Iterables.getOnlyElement(child.getChildren());
         assertEquals(grandChild.getEntityType().getName(), BasicEntity.class.getName());
-        Collection<Entity> grandGrandChildren = grandChild.getChildren();
-        assertEquals(grandGrandChildren.size(), 1);
-        Entity grandGrandChild = Iterables.getOnlyElement(grandGrandChildren);
-        assertEquals(grandGrandChild.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+        Entity grandGrandChild = Iterables.getOnlyElement(grandChild.getChildren());
+        assertEquals(grandGrandChild.getEntityType().getName(), TestEntity.class.getName());
 
         deleteCatalogEntity(referencedSymbolicName);
         deleteCatalogEntity(referrerSymbolicName);
@@ -313,33 +261,33 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
 
     @Test
     public void testLaunchApplicationChildWithCatalogReferencingOtherCatalogServicesBlock() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String referencedSymbolicName = "my.catalog.app.id.child.referenced";
         String referrerSymbolicName = "my.catalog.app.id.child.referring";
-        addCatalogOSGiEntity(referencedSymbolicName, SIMPLE_ENTITY_TYPE);
-        addCatalogChildOSGiEntityWithServicesBlock(referrerSymbolicName, ver(referencedSymbolicName));
+        addCatalogEntity(IdAndVersion.of(referencedSymbolicName, TEST_VERSION), TestEntity.class.getName());
+
+        addCatalogItems(
+                "brooklyn.catalog:",
+                "  id: " + referrerSymbolicName,
+                "  version: " + TEST_VERSION,
+                "  itemType: entity",
+                "  item:",
+                "    services:",
+                "    - type: " + BasicEntity.class.getName(),
+                "      brooklyn.children:",
+                "      - type: " + ver(referencedSymbolicName, TEST_VERSION));
 
         Entity app = createAndStartApplication(
-            "name: simple-app-yaml",
-            "location: localhost",
             "services:",
-            "- serviceType: "+BasicEntity.class.getName(),
+            "- type: "+BasicEntity.class.getName(),
             "  brooklyn.children:",
             "  - type: " + ver(referrerSymbolicName));
 
-        Collection<Entity> children = app.getChildren();
-        assertEquals(children.size(), 1);
-        Entity child = Iterables.getOnlyElement(children);
+        Entity child = Iterables.getOnlyElement(app.getChildren());
         assertEquals(child.getEntityType().getName(), BasicEntity.class.getName());
-        Collection<Entity> grandChildren = child.getChildren();
-        assertEquals(grandChildren.size(), 1);
-        Entity grandChild = Iterables.getOnlyElement(grandChildren);
+        Entity grandChild = Iterables.getOnlyElement(child.getChildren());
         assertEquals(grandChild.getEntityType().getName(), BasicEntity.class.getName());
-        Collection<Entity> grandGrandChildren = grandChild.getChildren();
-        assertEquals(grandGrandChildren.size(), 1);
-        Entity grandGrandChild = Iterables.getOnlyElement(grandGrandChildren);
-        assertEquals(grandGrandChild.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+        Entity grandGrandChild = Iterables.getOnlyElement(grandChild.getChildren());
+        assertEquals(grandGrandChild.getEntityType().getName(), TestEntity.class.getName());
 
         deleteCatalogEntity(referencedSymbolicName);
         deleteCatalogEntity(referrerSymbolicName);
@@ -347,30 +295,34 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
     
     @Test
     public void testLaunchApplicationWithTypeUsingJavaColonPrefix() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String symbolicName = SIMPLE_ENTITY_TYPE;
-        String serviceName = "java:"+SIMPLE_ENTITY_TYPE;
-        registerAndLaunchAndAssertSimpleEntity(symbolicName, serviceName);
+        String symbolicName = "t1";
+        String actualType = TestEntity.class.getName();
+        String serviceType = "java:"+actualType;
+        registerAndLaunchAndAssertSimpleEntity(symbolicName, serviceType, actualType);
     }
 
     @Test
     public void testLaunchApplicationLoopWithJavaTypeName() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String symbolicName = SIMPLE_ENTITY_TYPE;
-        String serviceName = SIMPLE_ENTITY_TYPE;
+        String symbolicName = TestEntity.class.getName();
+        String serviceName = TestEntity.class.getName();
         registerAndLaunchAndAssertSimpleEntity(symbolicName, serviceName);
     }
 
     @Test
     public void testLaunchApplicationChildLoopCatalogIdFails() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String referrerSymbolicName = "my.catalog.app.id.child.referring";
         try {
             // TODO only fails if using 'services', because that forces plan parsing; should fail in all cases
-            addCatalogChildOSGiEntityWithServicesBlock(referrerSymbolicName, ver(referrerSymbolicName));
+            addCatalogItems(
+                    "brooklyn.catalog:",
+                    "  id: " + referrerSymbolicName,
+                    "  version: " + TEST_VERSION,
+                    "  itemType: entity",
+                    "  item:",
+                    "    services:",
+                    "    - type: " + BasicEntity.class.getName(),
+                    "      brooklyn.children:",
+                    "      - type: " + ver(referrerSymbolicName, TEST_VERSION));
             fail("Expected to throw");
         } catch (Exception e) {
             Exceptions.propagateIfFatal(e);
@@ -379,177 +331,33 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
     }
 
     @Test
-    public void testReferenceInstalledBundleByName() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String firstItemId = "my.catalog.app.id.register_bundle";
-        String secondItemId = "my.catalog.app.id.reference_bundle";
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + firstItemId,
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  item:",
-            "    type: " + SIMPLE_ENTITY_TYPE);
-        deleteCatalogEntity(firstItemId);
-
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + secondItemId,
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  libraries:",
-            "  - name: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_NAME,
-            "    version: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_VERSION,
-            "  item:",
-            "    type: " + SIMPLE_ENTITY_TYPE);
-
-        deleteCatalogEntity(secondItemId);
-    }
-
-    @Test
-    public void testReferenceNonInstalledBundledByNameFails() {
-        String nonExistentId = "none-existent-id";
-        String nonExistentVersion = "9.9.9";
-        try {
-            addCatalogItems(
-                "brooklyn.catalog:",
-                "  id: my.catalog.app.id.non_existing.ref",
-                "  version: " + TEST_VERSION,
-                "  itemType: entity",
-                "  libraries:",
-                "  - name: " + nonExistentId,
-                "    version: " + nonExistentVersion,
-                "  item:",
-                "    type: " + SIMPLE_ENTITY_TYPE);
-            fail();
-        } catch (IllegalStateException e) {
-            Assert.assertEquals(e.getMessage(), "Bundle from null failed to install: Bundle CatalogBundleDto{symbolicName=" + nonExistentId + ", version=" + nonExistentVersion + ", url=null} not previously registered, but URL is empty.");
-        }
-    }
-
-    @Test
-    public void testPartialBundleReferenceFails() {
-        try {
-            addCatalogItems(
-                "brooklyn.catalog:",
-                "  id: my.catalog.app.id.non_existing.ref",
-                "  version: " + TEST_VERSION,
-                "  itemType: entity",
-                "  libraries:",
-                "  - name: io.brooklyn.brooklyn-test-osgi-entities",
-                "  item:",
-                "    type: " + SIMPLE_ENTITY_TYPE);
-            fail();
-        } catch (NullPointerException e) {
-            Assert.assertEquals(e.getMessage(), "both name and version are required");
-        }
-        try {
-            addCatalogItems(
-                "brooklyn.catalog:",
-                "  id: my.catalog.app.id.non_existing.ref",
-                "  version: " + TEST_VERSION,
-                "  itemType: entity",
-                "  libraries:",
-                "  - version: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_VERSION,
-                "  item:",
-                "    type: " + SIMPLE_ENTITY_TYPE);
-            fail();
-        } catch (NullPointerException e) {
-            Assert.assertEquals(e.getMessage(), "both name and version are required");
-        }
-    }
-
-    @Test
-    public void testFullBundleReference() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String itemId = "my.catalog.app.id.full_ref";
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + itemId,
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  libraries:",
-            "  - name: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_NAME,
-            "    version: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_VERSION,
-            "    url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  item:",
-            "    type: " + SIMPLE_ENTITY_TYPE);
-        deleteCatalogEntity(itemId);
-    }
-
-    /**
-     * Test that the name:version contained in the OSGi bundle will
-     * override the values supplied in the YAML.
-     */
-    @Test
-    public void testFullBundleReferenceUrlMetaOverridesLocalNameVersion() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String firstItemId = "my.catalog.app.id.register_bundle";
-        String nonExistentId = "non_existent_id";
-        String nonExistentVersion = "9.9.9";
-        try {
-            addCatalogItems(
-                "brooklyn.catalog:",
-                "  id: " + firstItemId,
-                "  version: " + TEST_VERSION,
-                "  itemType: entity",
-                "  libraries:",
-                "  - name: " + nonExistentId,
-                "    version: " + nonExistentVersion,
-                "    url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-                "",
-                "  item:",
-                "    type: " + SIMPLE_ENTITY_TYPE);
-            fail();
-        } catch (IllegalStateException e) {
-            assertEquals(e.getMessage(), "Bundle from " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL + " failed to install: " +
-                    "Bundle already installed as " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_NAME + ":" +
-                    OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_VERSION + " but user explicitly requested " +
-                    "CatalogBundleDto{symbolicName=" + nonExistentId + ", version=" + nonExistentVersion + ", url=" +
-                    OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL + "}");
-        }
-    }
-
-    @Test
     public void testUpdatingItemAllowedIfSame() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String id = "my.catalog.app.id.duplicate";
-        addCatalogOSGiEntity(id);
-        addCatalogOSGiEntity(id);
+        String symbolicName = "my.catalog.app.id.duplicate";
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), TestEntity.class.getName());
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), TestEntity.class.getName());
     }
     
     @Test(expectedExceptions = IllegalStateException.class)
     public void testUpdatingItemFailsIfDifferent() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String id = "my.catalog.app.id.duplicate";
-        addCatalogOSGiEntity(id);
-        addCatalogOSGiEntity(id, SIMPLE_ENTITY_TYPE, true);
+        String symbolicName = "my.catalog.app.id.duplicate";
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), TestEntity.class.getName());
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), BasicEntity.class.getName());
     }
 
     @Test
     public void testForcedUpdatingItem() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String id = "my.catalog.app.id.duplicate";
-        addCatalogOSGiEntity(id);
+        String symbolicName = "my.catalog.app.id.duplicate";
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), TestEntity.class.getName());
         forceCatalogUpdate();
-        addCatalogOSGiEntity(id);
-        deleteCatalogEntity(id);
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), TestEntity.class.getName());
+        deleteCatalogEntity(symbolicName);
     }
 
     @Test
     public void testCreateSpecFromCatalogItem() {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String id = "my.catalog.app.id.create_spec";
-        addCatalogOSGiEntity(id);
+        addCatalogEntity(IdAndVersion.of(id, TEST_VERSION), TestEntity.class.getName());
+        
         BrooklynTypeRegistry catalog = mgmt().getTypeRegistry();
         RegisteredType item = catalog.get(id, TEST_VERSION);
         EntitySpec<?> spec = catalog.createSpec(item, null, EntitySpec.class);
@@ -559,65 +367,29 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
     }
     
     @Test
-    public void testLoadResourceFromBundle() throws Exception {
-        String id = "resource.test";
-        addCatalogOSGiEntity(id, SIMPLE_ENTITY_TYPE);
-        String yaml =
-                "services: \n" +
-                "  - serviceType: "+ver(id);
-        Entity app = createAndStartApplication(yaml);
-        Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
-        InputStream icon = new ResourceUtils(simpleEntity).getResourceFromUrl("classpath:/org/apache/brooklyn/test/osgi/entities/icon.gif");
-        assertTrue(icon != null);
-        icon.close();
-    }
-    
-    @Test
     public void testMissingTypeDoesNotRecurse() {
         String symbolicName = "my.catalog.app.id.basic";
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + symbolicName,
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  item:",
-            "    type: org.apache.brooklyn.entity.stock.BasicEntity");
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), TestEntity.class.getName());
 
         try {
-            addCatalogItems(
-                    "brooklyn.catalog:",
-                    "  id: " + symbolicName,
-                    "  version: " + TEST_VERSION + "-update",
-                    "  itemType: entity",
-                    "  item:",
-                    "    type: " + symbolicName);
-            fail("Catalog addition expected to fail due to non-existent java type " + symbolicName);
+            addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION + "-update"), symbolicName);
+            fail("Catalog addition expected to fail due to recursive reference to " + symbolicName);
         } catch (IllegalStateException e) {
             assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
         }
     }
     
     @Test
-    public void testVersionedTypeDoesNotRecurse() {
+    public void testVersionedTypeDoesNotRecurse() throws Exception {
+        // Alternatively, we could change this to tell foo:v2 reference foo:v1, but that feels 
+        // like a bad idea! 
         String symbolicName = "my.catalog.app.id.basic";
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + symbolicName,
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  item:",
-            "    type: org.apache.brooklyn.entity.stock.BasicEntity");
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), TestEntity.class.getName());
 
         String versionedId = CatalogUtils.getVersionedId(symbolicName, TEST_VERSION);
         try {
-            addCatalogItems(
-                "brooklyn.catalog:",
-                "  id: " + symbolicName,
-                "  version: " + TEST_VERSION + "-update",
-                "  itemType: entity",
-                "  item:",
-                "    type: " + versionedId);
-            fail("Catalog addition expected to fail due to non-existent java type " + versionedId);
+            addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION + "-update"), versionedId);
+            fail("Catalog addition expected to fail due to recursive reference to " + versionedId);
         } catch (IllegalStateException e) {
             assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
         }
@@ -625,33 +397,17 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
 
     @Test
     public void testIndirectRecursionFails() throws Exception {
-        String symbolicName = "my.catalog.app.id.basic";
+        String callerSymbolicName = "my.catalog.app.id.caller";
+        String calleeSymbolicName = "my.catalog.app.id.callee";
+        
         // 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",
-                "  itemType: entity",
-                "  item:",
-                "    type: "+BasicEntity.class.getName());
+        addCatalogEntity(IdAndVersion.of(callerSymbolicName, TEST_VERSION + "-pre"), TestEntity.class.getName());
 
-        addCatalogItems(
-                "brooklyn.catalog:",
-                "  id: " + symbolicName + ".callee",
-                "  version: " + TEST_VERSION,
-                "  itemType: entity",
-                "  item:",
-                "    type: " + symbolicName + ".caller");
+        addCatalogEntity(IdAndVersion.of(calleeSymbolicName, TEST_VERSION), callerSymbolicName);
 
         try {
-            addCatalogItems(
-                    "brooklyn.catalog:",
-                    "  id: " + symbolicName + ".caller",
-                    "  version: " + TEST_VERSION,
-                    "  itemType: entity",
-                    "  item:",
-                    "    type: " + symbolicName + ".callee");
+            addCatalogEntity(IdAndVersion.of(callerSymbolicName, TEST_VERSION), calleeSymbolicName);
             fail();
         } catch (IllegalStateException e) {
             assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
@@ -660,38 +416,29 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
 
     @Test
     public void testChildItemsDoNotRecurse() throws Exception {
-        String symbolicName = "my.catalog.app.id.basic";
+        String callerSymbolicName = "my.catalog.app.id.caller";
+        String calleeSymbolicName = "my.catalog.app.id.callee";
+
         // 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",
-                "  itemType: entity",
-                "  item:",
-                "    type: org.apache.brooklyn.entity.stock.BasicEntity");
+        
+        addCatalogEntity(IdAndVersion.of(callerSymbolicName, TEST_VERSION + "-pre"), TestEntity.class.getName());
 
-        addCatalogItems(
-                "brooklyn.catalog:",
-                "  id: " + symbolicName + ".callee",
-                "  version: " + TEST_VERSION,
-                "  itemType: entity",
-                "  item:",
-                "    type: " + symbolicName + ".caller");
+        addCatalogEntity(IdAndVersion.of(calleeSymbolicName, TEST_VERSION), callerSymbolicName);
 
         try {
             // TODO Only passes if include "services:" and if itemType=entity, rather than "template"!
             // Being a child is important, triggers the case where: we allow retrying with other transformers.
             addCatalogItems(
                     "brooklyn.catalog:",
-                    "  id: " + symbolicName + ".caller",
+                    "  id: " + callerSymbolicName,
                     "  version: " + TEST_VERSION,
                     "  itemType: entity",
                     "  item:",
                     "    services:",
-                    "    - type: org.apache.brooklyn.entity.stock.BasicEntity",
+                    "    - type: " + BasicEntity.class.getName(),
                     "      brooklyn.children:",
-                    "      - type: " + symbolicName + ".callee");
+                    "      - type: " + calleeSymbolicName);
             fail();
         } catch (IllegalStateException e) {
             assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
@@ -701,116 +448,92 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
     @Test
     public void testRecursiveCheckForDepenentsOnly() throws Exception {
         String symbolicName = "my.catalog.app.id.basic";
-        addCatalogItems(
-                "brooklyn.catalog:",
-                "  id: " + symbolicName,
-                "  version: " + TEST_VERSION,
-                "  itemType: entity",
-                "  item:",
-                "    type: org.apache.brooklyn.entity.stock.BasicEntity");
+        addCatalogEntity(symbolicName, TestEntity.class.getName());
 
         createAndStartApplication(
                 "services:",
-                "- type: " + ver(symbolicName),
+                "- type: " + symbolicName,
                 "  brooklyn.children:",
-                "  - type: " + ver(symbolicName),
-                "- type: " + ver(symbolicName),
+                "  - type: " + symbolicName,
+                "- type: " + symbolicName,
                 "  brooklyn.children:",
-                "  - type: " + ver(symbolicName));
-    }
-
-    @Test
-    public void testOsgiNotLeakingToParent() {
-        addCatalogOSGiEntity(SIMPLE_ENTITY_TYPE);
-        try {
-            addCatalogItems(
-                    "brooklyn.catalog:",
-                    "  id: " + SIMPLE_ENTITY_TYPE,
-                    "  version: " + TEST_VERSION + "-update",
-                    "  itemType: entity",
-                    "  item:",
-                    "    type: " + SIMPLE_ENTITY_TYPE);
-            fail("Catalog addition expected to fail due to non-existent java type " + SIMPLE_ENTITY_TYPE);
-        } catch (IllegalStateException e) {
-            assertTrue(e.toString().contains("recursive"), "Unexpected error message: "+e);
-        }
+                "  - type: " + symbolicName);
     }
 
     @Test
     public void testConfigAppliedToCatalogItem() throws Exception {
-        addCatalogOSGiEntity("test", TestEntity.class.getName());
-        String testName = "test-applies-config-on-catalog-item";
+        addCatalogEntity("test", TestEntity.class.getName());
+        String val = "test-applies-config-on-catalog-item";
         Entity app = createAndStartApplication(
                 "services:",
-                "- type: " + ver("test"),
+                "- type: test",
                 "  brooklyn.config:",
-                "    test.confName: " + testName);
+                "    test.confName: " + val);
         Entity testEntity = Iterables.getOnlyElement(app.getChildren());
-        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), val);
     }
 
     @Test
     public void testFlagsAppliesToCatalogItem() throws Exception {
-        addCatalogOSGiEntity("test", TestEntity.class.getName());
-        String testName = "test-applies-config-on-catalog-item";
+        addCatalogEntity("test", TestEntity.class.getName());
+        String val = "test-applies-config-on-catalog-item";
         Entity app = createAndStartApplication(
                 "services:",
                 "- type: " + ver("test"),
-                "  confName: " + testName);
+                "  confName: " + val);
         Entity testEntity = Iterables.getOnlyElement(app.getChildren());
-        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), val);
     }
 
     @Test
     public void testExplicitFlagsAppliesToCatalogItem() throws Exception {
-        addCatalogOSGiEntity("test", TestEntity.class.getName());
-        String testName = "test-applies-config-on-catalog-item";
+        addCatalogEntity("test", TestEntity.class.getName());
+        String val = "test-applies-config-on-catalog-item";
         Entity app = createAndStartApplication(
                 "services:",
                 "- type: " + ver("test"),
                 "  brooklyn.flags:",
-                "    confName: " + testName);
+                "    confName: " + val);
         Entity testEntity = Iterables.getOnlyElement(app.getChildren());
-        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), val);
     }
     
-
     @Test
     public void testConfigAppliedToCatalogItemImpl() throws Exception {
-        addCatalogOSGiEntity("test", TestEntityImpl.class.getName());
-        String testName = "test-applies-config-on-catalog-item";
+        addCatalogEntity("test", TestEntityImpl.class.getName());
+        String val = "test-applies-config-on-catalog-item";
         Entity app = createAndStartApplication(
                 "services:",
                 "- type: " + ver("test"),
                 "  brooklyn.config:",
-                "    test.confName: " + testName);
+                "    test.confName: " + val);
         Entity testEntity = Iterables.getOnlyElement(app.getChildren());
-        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), val);
     }
 
     @Test
     public void testFlagsAppliesToCatalogItemImpl() throws Exception {
-        addCatalogOSGiEntity("test", TestEntityImpl.class.getName());
-        String testName = "test-applies-config-on-catalog-item";
+        addCatalogEntity("test", TestEntityImpl.class.getName());
+        String val = "test-applies-config-on-catalog-item";
         Entity app = createAndStartApplication(
                 "services:",
                 "- type: " + ver("test"),
-                "  confName: " + testName);
+                "  confName: " + val);
         Entity testEntity = Iterables.getOnlyElement(app.getChildren());
-        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), val);
     }
 
     @Test
     public void testExplicitFlagsAppliesToCatalogItemImpl() throws Exception {
-        addCatalogOSGiEntity("test", TestEntityImpl.class.getName());
-        String testName = "test-applies-config-on-catalog-item";
+        addCatalogEntity("test", TestEntityImpl.class.getName());
+        String val = "test-applies-config-on-catalog-item";
         Entity app = createAndStartApplication(
                 "services:",
                 "- type: " + ver("test"),
                 "  brooklyn.flags:",
-                "    confName: " + testName);
+                "    confName: " + val);
         Entity testEntity = Iterables.getOnlyElement(app.getChildren());
-        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), testName);
+        assertEquals(testEntity.config().get(TestEntity.CONF_NAME), val);
     }
 
     @Test
@@ -890,70 +613,12 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
         mgmt().getCatalog().deleteCatalogItem(id, version);
     }
 
-    @Test
-    public void testCreateOsgiSpecFromRegistry() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String symbolicName = "my.catalog.app.id.registry.spec";
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + symbolicName,
-            "  name: My Catalog App",
-            "  description: My description",
-            "  icon_url: classpath://path/to/myicon.jpg",
-            "  version: " + TEST_VERSION,
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  item: " + SIMPLE_ENTITY_TYPE);
-
-        BrooklynTypeRegistry registry = mgmt().getTypeRegistry();
-        RegisteredType item = registry.get(symbolicName, TEST_VERSION);
-        AbstractBrooklynObjectSpec<?, ?> spec = registry.createSpec(item, null, null);
-        assertEquals(spec.getCatalogItemId(), ver(symbolicName));
-
-        deleteCatalogEntity(symbolicName);
-    }
-
-    @Test
-    public void testIndirectCatalogItemCanLoadResources() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String symbolicNameInner = "my.catalog.app.id.inner";
-        String symbolicNameOuter = "my.catalog.app.id.outer";
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  version: " + TEST_VERSION,
-            "  items:",
-            "  - id: " + symbolicNameInner,
-            "    name: My Catalog App",
-            "    description: My description",
-            "    icon_url: classpath://path/to/myicon.jpg",
-            "    libraries:",
-            "    - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "    item: " + SIMPLE_ENTITY_TYPE,
-            "  - id: " + symbolicNameOuter,
-            "    item: " + symbolicNameInner);
-
-        String yaml = "name: " + symbolicNameOuter + "\n" +
-                "services: \n" +
-                "  - serviceType: "+ver(symbolicNameOuter);
-        Entity app = createAndStartApplication(yaml);
-        Entity entity = app.getChildren().iterator().next();
-
-        ResourceUtils.create(entity).getResourceAsString("classpath://yaml-ref-osgi-entity.yaml");
-
-        deleteCatalogEntity(symbolicNameInner);
-        deleteCatalogEntity(symbolicNameOuter);
-    }
-
     // The test is disabled as it fails. The entity will get assigned the outer-most catalog
     // item which doesn't have the necessary libraries with visibility to the entity's classpath
     // When loading resources from inside the entity then we will use the wrong BCLCS. A workaround
     // has been implemented which explicitly adds the entity's class loader to the fallbacks.
     @Test(groups="WIP")
     public void testCatalogItemIdInReferencedItems() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
         String symbolicNameInner = "my.catalog.app.id.inner";
         String symbolicNameOuter = "my.catalog.app.id.outer";
         addCatalogItems(
@@ -961,12 +626,7 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
             "  version: " + TEST_VERSION,
             "  items:",
             "  - id: " + symbolicNameInner,
-            "    name: My Catalog App",
-            "    description: My description",
-            "    icon_url: classpath://path/to/myicon.jpg",
-            "    libraries:",
-            "    - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "    item: " + SIMPLE_ENTITY_TYPE,
+            "    item: " + TestEntity.class.getName(),
             "  - id: " + symbolicNameOuter,
             "    item: " + symbolicNameInner);
 
@@ -985,97 +645,47 @@ public class CatalogYamlEntityTest extends AbstractYamlTest {
     }
 
     private void registerAndLaunchAndAssertSimpleEntity(String symbolicName, String serviceType) throws Exception {
-        addCatalogOSGiEntity(symbolicName, serviceType);
-        String yaml = "name: simple-app-yaml\n" +
-                      "location: localhost\n" +
-                      "services: \n" +
-                      "  - serviceType: "+ver(symbolicName);
-        Entity app = createAndStartApplication(yaml);
+        registerAndLaunchAndAssertSimpleEntity(symbolicName, serviceType, serviceType);
+    }
+    
+    private void registerAndLaunchAndAssertSimpleEntity(String symbolicName, String serviceType, String expectedType) throws Exception {
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), serviceType);
+        
+        Entity app = createAndStartApplication(
+                "services:",
+                "- type: "+ver(symbolicName, TEST_VERSION));
 
         Entity simpleEntity = Iterables.getOnlyElement(app.getChildren());
-        assertEquals(simpleEntity.getEntityType().getName(), SIMPLE_ENTITY_TYPE);
+        assertEquals(simpleEntity.getEntityType().getName(), expectedType);
 
         deleteCatalogEntity(symbolicName);
     }
 
-    private void addCatalogOSGiEntity(String symbolicName) {
-        addCatalogOSGiEntity(symbolicName, SIMPLE_ENTITY_TYPE);
-    }
-
-    private void addCatalogOSGiEntity(String symbolicName, String serviceType) {
-        addCatalogOSGiEntity(symbolicName, serviceType, false);
-    }
-    
-    private void addCatalogOSGiEntity(String symbolicName, String serviceType, boolean extraLib) {
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + symbolicName,
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  name: My Catalog App",
-            "  description: My description",
-            "  icon_url: classpath://path/to/myicon.jpg",
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL +
-            (extraLib ? "\n"+"  - url: "+OsgiStandaloneTest.BROOKLYN_OSGI_TEST_A_0_1_0_URL : ""),
-            "  item:",
-            "    type: " + serviceType);
-    }
-
-    private void addCatalogOSGiEntities(Map<String, String> idAndTypes) {
-        List<String> lines = MutableList.of(
-            "brooklyn.catalog:",
-            "  name: My Catalog App",
-            "  description: My description",
-            "  icon_url: classpath://path/to/myicon.jpg",
-            "  version: " + TEST_VERSION,
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  items:");
+    public static class IdAndVersion {
+        public final String id;
+        public final String version;
         
-        for (Map.Entry<String, String> entry : idAndTypes.entrySet()) {
-            lines.addAll(MutableList.of(
-            "  - id: " + entry.getKey(),
-            "    item:",
-            "      type: " + entry.getValue()));
+        public static IdAndVersion of(String id, String version) {
+            return new IdAndVersion(id, version);
+        }
+        
+        public IdAndVersion(String id, String version) {
+            this.id = checkNotNull(id, "id");
+            this.version = checkNotNull(version, "version");
         }
-            
-        addCatalogItems(lines);
     }
     
-    private void addCatalogChildOSGiEntityWithServicesBlock(String symbolicName, String serviceType) {
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + symbolicName,
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  name: My Catalog App",
-            "  description: My description",
-            "  icon_url: classpath://path/to/myicon.jpg",
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  item:",
-            "    services:",
-            "    - type: " + BasicEntity.class.getName(),
-            "      brooklyn.children:",
-            "      - type: " + serviceType);
+    private void addCatalogEntity(String symbolicName, String entityType) {
+        addCatalogEntity(IdAndVersion.of(symbolicName, TEST_VERSION), entityType);
     }
     
-    private void addCatalogChildOSGiEntity(String symbolicName, String serviceType) {
+    private void addCatalogEntity(IdAndVersion idAndVersion, String serviceType) {
         addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + symbolicName,
-            "  itemType: entity",
-            "  name: My Catalog App",
-            "  description: My description",
-            "  icon_url: classpath://path/to/myicon.jpg",
-            "  version: " + TEST_VERSION,
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  item:",
-            "    type: " + BasicEntity.class.getName(),
-            "    brooklyn.children:",
-            "    - type: " + serviceType);
+                "brooklyn.catalog:",
+                "  id: " + idAndVersion.id,
+                "  version: " + idAndVersion.version,
+                "  itemType: entity",
+                "  item:",
+                "    type: " + serviceType);
     }
-
 }


[7/9] brooklyn-server git commit: Split ReferencedYamlTest for non-osgi and osgi

Posted by sv...@apache.org.
Split ReferencedYamlTest for non-osgi and osgi

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

Branch: refs/heads/master
Commit: 8f0021eddb3df379dad227c31b9f4b29e9dfaeba
Parents: c5ae189
Author: Aled Sage <al...@gmail.com>
Authored: Fri Feb 10 21:55:56 2017 +0000
Committer: Aled Sage <al...@gmail.com>
Committed: Fri Feb 10 22:16:09 2017 +0000

----------------------------------------------------------------------
 .../camp/brooklyn/ReferencedOsgiYamlTest.java   | 230 +++++++++++++++++++
 .../camp/brooklyn/ReferencedYamlTest.java       | 191 ---------------
 2 files changed, 230 insertions(+), 191 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8f0021ed/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ReferencedOsgiYamlTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ReferencedOsgiYamlTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ReferencedOsgiYamlTest.java
new file mode 100644
index 0000000..4fc647d
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ReferencedOsgiYamlTest.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.brooklyn.camp.brooklyn;
+
+import java.util.Collection;
+
+import org.apache.brooklyn.api.catalog.BrooklynCatalog;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
+import org.apache.brooklyn.entity.stock.BasicEntity;
+import org.apache.brooklyn.test.support.TestResourceUnavailableException;
+import org.apache.brooklyn.util.osgi.OsgiTestResources;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Iterables;
+
+public class ReferencedOsgiYamlTest extends AbstractYamlTest {
+
+    @Override
+    protected boolean disableOsgi() {
+        return false;
+    }
+
+    @Test
+    public void testCatalogReferencingYamlUrlFromOsgiBundle() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: yaml.reference",
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  libraries:",
+            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  item:",
+            "    type: classpath://yaml-ref-osgi-entity.yaml");
+        
+        String entityName = "YAML -> catalog item -> yaml url (osgi)";
+        Entity app = createAndStartApplication(
+            "services:",
+            "- name: " + entityName,
+            "  type: " + ver("yaml.reference"));
+        
+        checkChildEntitySpec(app, entityName);
+    }
+
+    /**
+     * Tests that a YAML referenced by URL from a catalog item
+     * will have access to the catalog item's bundles.
+     */
+    @Test
+    public void testCatalogLeaksBundlesToReferencedYaml() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        String parentCatalogId = "my.catalog.app.id.url.parent";
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  id: " + parentCatalogId,
+            "  version: " + TEST_VERSION,
+            "  itemType: entity",
+            "  libraries:",
+            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  item:",
+            "    type: classpath://yaml-ref-bundle-without-libraries.yaml");
+
+        Entity app = createAndStartApplication(
+            "services:",
+                "- type: " + ver(parentCatalogId));
+        
+        Collection<Entity> children = app.getChildren();
+        Assert.assertEquals(children.size(), 1);
+        Entity child = Iterables.getOnlyElement(children);
+        Assert.assertEquals(child.getEntityType().getName(), "org.apache.brooklyn.test.osgi.entities.SimpleEntity");
+
+        deleteCatalogEntity(parentCatalogId);
+    }
+
+    @Test
+    public void testCatalogReference() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  brooklyn.libraries:",
+            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  items:",
+            "  - classpath://yaml-ref-parent-catalog.bom");
+
+        assertCatalogReference();
+    }
+
+    @Test
+    public void testCatalogReferenceByExplicitUrl() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  brooklyn.libraries:",
+            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  items:",
+            "  - include: classpath://yaml-ref-parent-catalog.bom");
+
+        assertCatalogReference();
+    }
+
+    @Test
+    public void testCatalogReferenceByMultipleUrls() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  items:",
+            "  - include: classpath://yaml-ref-simple.bom",
+            "  - include: classpath://yaml-ref-more.bom"
+        );
+
+        assertCatalogReference();
+    }
+
+    @Test
+    public void testCatalogReferenceByMultipleUrlsSimplerSyntax() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  items:",
+            "  - classpath://yaml-ref-simple.bom",
+            "  - classpath://yaml-ref-more.bom"
+        );
+
+        assertCatalogReference();
+    }
+
+
+    @Test
+    public void testCatalogReferenceSeesPreviousItems() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  brooklyn.libraries:",
+            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  items:",
+            "  - id: yaml.nested.catalog.simple",
+            "    itemType: entity",
+            "    item:",
+            "      type: " + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY,
+            "  - include: classpath://yaml-ref-back-catalog.bom");
+
+        String entityNameSimple = "YAML -> catalog -> catalog (osgi)";
+        Entity app = createAndStartApplication(
+            "services:",
+            "- name: " + entityNameSimple,
+            "  type: back-reference");
+        
+        Collection<Entity> children = app.getChildren();
+        Assert.assertEquals(children.size(), 1);
+        Entity childSimple = Iterables.getOnlyElement(children);
+        Assert.assertEquals(childSimple.getDisplayName(), entityNameSimple);
+        Assert.assertEquals(childSimple.getEntityType().getName(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
+    }
+
+    @Test
+    public void testCatalogReferenceMixesMetaAndUrl() throws Exception {
+        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
+
+        addCatalogItems(
+            "brooklyn.catalog:",
+            "  brooklyn.libraries:",
+            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
+            "  items:",
+            "  - include: classpath://yaml-ref-parent-catalog.bom",
+            "    items:",
+            "    - id: yaml.nested.catalog.nested",
+            "      itemType: entity",
+            "      item:",
+            "        type: " + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
+
+        BrooklynCatalog catalog = mgmt().getCatalog();
+        Assert.assertNotNull(catalog.getCatalogItem("yaml.nested.catalog.nested", BrooklynCatalog.DEFAULT_VERSION));
+        Assert.assertNotNull(catalog.getCatalogItem("yaml.nested.catalog.simple", BrooklynCatalog.DEFAULT_VERSION));
+        Assert.assertNotNull(catalog.getCatalogItem("yaml.nested.catalog.more", BrooklynCatalog.DEFAULT_VERSION));
+    }
+
+    protected void assertCatalogReference() throws Exception {
+        String entityNameSimple = "YAML -> catalog -> catalog simple (osgi)";
+        String entityNameMore = "YAML -> catalog -> catalog more (osgi)";
+        Entity app = createAndStartApplication(
+            "services:",
+            "- name: " + entityNameSimple,
+            "  type: yaml.nested.catalog.simple",
+            "- name: " + entityNameMore,
+            "  type: yaml.nested.catalog.more");
+        
+        Collection<Entity> children = app.getChildren();
+        Assert.assertEquals(children.size(), 2);
+        Entity childSimple = Iterables.get(children, 0);
+        Assert.assertEquals(childSimple.getDisplayName(), entityNameSimple);
+        Assert.assertEquals(childSimple.getEntityType().getName(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
+
+        Entity childMore = Iterables.get(children, 1);
+        Assert.assertEquals(childMore.getDisplayName(), entityNameMore);
+        Assert.assertEquals(childMore.getEntityType().getName(), OsgiTestResources.BROOKLYN_TEST_MORE_ENTITIES_MORE_ENTITY);
+    }
+
+    private void checkChildEntitySpec(Entity app, String entityName) {
+        Collection<Entity> children = app.getChildren();
+        Assert.assertEquals(children.size(), 1);
+        Entity child = Iterables.getOnlyElement(children);
+        Assert.assertEquals(child.getDisplayName(), entityName);
+        Assert.assertEquals(child.getEntityType().getName(), BasicEntity.class.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/8f0021ed/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ReferencedYamlTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ReferencedYamlTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ReferencedYamlTest.java
index 8775496..8de36fe 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ReferencedYamlTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ReferencedYamlTest.java
@@ -20,13 +20,9 @@ package org.apache.brooklyn.camp.brooklyn;
 
 import java.util.Collection;
 
-import org.apache.brooklyn.api.catalog.BrooklynCatalog;
 import org.apache.brooklyn.api.entity.Entity;
-import org.apache.brooklyn.core.mgmt.osgi.OsgiStandaloneTest;
 import org.apache.brooklyn.entity.stock.BasicApplication;
 import org.apache.brooklyn.entity.stock.BasicEntity;
-import org.apache.brooklyn.test.support.TestResourceUnavailableException;
-import org.apache.brooklyn.util.osgi.OsgiTestResources;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -34,11 +30,6 @@ import com.google.common.collect.Iterables;
 
 public class ReferencedYamlTest extends AbstractYamlTest {
 
-    @Override
-    protected boolean disableOsgi() {
-        return false;
-    }
-
     @Test
     public void testReferenceEntityYamlAsPlatformComponent() throws Exception {
         String entityName = "Reference child name";
@@ -118,29 +109,6 @@ public class ReferencedYamlTest extends AbstractYamlTest {
     }
 
     @Test
-    public void testCatalogReferencingYamlUrlFromOsgiBundle() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: yaml.reference",
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  item:",
-            "    type: classpath://yaml-ref-osgi-entity.yaml");
-        
-        String entityName = "YAML -> catalog item -> yaml url (osgi)";
-        Entity app = createAndStartApplication(
-            "services:",
-            "- name: " + entityName,
-            "  type: " + ver("yaml.reference"));
-        
-        checkChildEntitySpec(app, entityName);
-    }
-
-    @Test
     public void testYamlUrlReferencingCatalog() throws Exception {
         addCatalogItems(
             "brooklyn.catalog:",
@@ -185,165 +153,6 @@ public class ReferencedYamlTest extends AbstractYamlTest {
         checkChildEntitySpec(app, entityName);
     }
 
-    /**
-     * Tests that a YAML referenced by URL from a catalog item
-     * will have access to the catalog item's bundles.
-     */
-    @Test
-    public void testCatalogLeaksBundlesToReferencedYaml() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        String parentCatalogId = "my.catalog.app.id.url.parent";
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  id: " + parentCatalogId,
-            "  version: " + TEST_VERSION,
-            "  itemType: entity",
-            "  libraries:",
-            "  - url: " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  item:",
-            "    type: classpath://yaml-ref-bundle-without-libraries.yaml");
-
-        Entity app = createAndStartApplication(
-            "services:",
-                "- type: " + ver(parentCatalogId));
-        
-        Collection<Entity> children = app.getChildren();
-        Assert.assertEquals(children.size(), 1);
-        Entity child = Iterables.getOnlyElement(children);
-        Assert.assertEquals(child.getEntityType().getName(), "org.apache.brooklyn.test.osgi.entities.SimpleEntity");
-
-        deleteCatalogEntity(parentCatalogId);
-    }
-
-    @Test
-    public void testCatalogReference() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  brooklyn.libraries:",
-            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  items:",
-            "  - classpath://yaml-ref-parent-catalog.bom");
-
-        assertCatalogReference();
-    }
-
-    @Test
-    public void testCatalogReferenceByExplicitUrl() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  brooklyn.libraries:",
-            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  items:",
-            "  - include: classpath://yaml-ref-parent-catalog.bom");
-
-        assertCatalogReference();
-    }
-
-    @Test
-    public void testCatalogReferenceByMultipleUrls() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  items:",
-            "  - include: classpath://yaml-ref-simple.bom",
-            "  - include: classpath://yaml-ref-more.bom"
-        );
-
-        assertCatalogReference();
-    }
-
-    @Test
-    public void testCatalogReferenceByMultipleUrlsSimplerSyntax() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  items:",
-            "  - classpath://yaml-ref-simple.bom",
-            "  - classpath://yaml-ref-more.bom"
-        );
-
-        assertCatalogReference();
-    }
-
-
-    @Test
-    public void testCatalogReferenceSeesPreviousItems() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  brooklyn.libraries:",
-            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  items:",
-            "  - id: yaml.nested.catalog.simple",
-            "    itemType: entity",
-            "    item:",
-            "      type: " + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY,
-            "  - include: classpath://yaml-ref-back-catalog.bom");
-
-        String entityNameSimple = "YAML -> catalog -> catalog (osgi)";
-        Entity app = createAndStartApplication(
-            "services:",
-            "- name: " + entityNameSimple,
-            "  type: back-reference");
-        
-        Collection<Entity> children = app.getChildren();
-        Assert.assertEquals(children.size(), 1);
-        Entity childSimple = Iterables.getOnlyElement(children);
-        Assert.assertEquals(childSimple.getDisplayName(), entityNameSimple);
-        Assert.assertEquals(childSimple.getEntityType().getName(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
-    }
-
-    @Test
-    public void testCatalogReferenceMixesMetaAndUrl() throws Exception {
-        TestResourceUnavailableException.throwIfResourceUnavailable(getClass(), OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_PATH);
-
-        addCatalogItems(
-            "brooklyn.catalog:",
-            "  brooklyn.libraries:",
-            "  - " + OsgiStandaloneTest.BROOKLYN_TEST_OSGI_ENTITIES_URL,
-            "  items:",
-            "  - include: classpath://yaml-ref-parent-catalog.bom",
-            "    items:",
-            "    - id: yaml.nested.catalog.nested",
-            "      itemType: entity",
-            "      item:",
-            "        type: " + OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
-
-        BrooklynCatalog catalog = mgmt().getCatalog();
-        Assert.assertNotNull(catalog.getCatalogItem("yaml.nested.catalog.nested", BrooklynCatalog.DEFAULT_VERSION));
-        Assert.assertNotNull(catalog.getCatalogItem("yaml.nested.catalog.simple", BrooklynCatalog.DEFAULT_VERSION));
-        Assert.assertNotNull(catalog.getCatalogItem("yaml.nested.catalog.more", BrooklynCatalog.DEFAULT_VERSION));
-    }
-
-    protected void assertCatalogReference() throws Exception {
-        String entityNameSimple = "YAML -> catalog -> catalog simple (osgi)";
-        String entityNameMore = "YAML -> catalog -> catalog more (osgi)";
-        Entity app = createAndStartApplication(
-            "services:",
-            "- name: " + entityNameSimple,
-            "  type: yaml.nested.catalog.simple",
-            "- name: " + entityNameMore,
-            "  type: yaml.nested.catalog.more");
-        
-        Collection<Entity> children = app.getChildren();
-        Assert.assertEquals(children.size(), 2);
-        Entity childSimple = Iterables.get(children, 0);
-        Assert.assertEquals(childSimple.getDisplayName(), entityNameSimple);
-        Assert.assertEquals(childSimple.getEntityType().getName(), OsgiTestResources.BROOKLYN_TEST_OSGI_ENTITIES_SIMPLE_ENTITY);
-
-        Entity childMore = Iterables.get(children, 1);
-        Assert.assertEquals(childMore.getDisplayName(), entityNameMore);
-        Assert.assertEquals(childMore.getEntityType().getName(), OsgiTestResources.BROOKLYN_TEST_MORE_ENTITIES_MORE_ENTITY);
-    }
-
     private void checkChildEntitySpec(Entity app, String entityName) {
         Collection<Entity> children = app.getChildren();
         Assert.assertEquals(children.size(), 1);