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

[1/3] brooklyn-server git commit: One instance of customizer at each customize step

Repository: brooklyn-server
Updated Branches:
  refs/heads/master 717fe3561 -> bc4942380


One instance of customizer at each customize step

Change so that getCustomizers is only called once during provisioning.
This ensures that only a single instance of a customizer configured
with DSL will be called.  This means that state can be kept between
steps.


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

Branch: refs/heads/master
Commit: c63a3c1cadba2e6ab935ccbed4c3e3245ab91cff
Parents: 717fe35
Author: Duncan Grant <du...@cloudsoftcorp.com>
Authored: Tue Oct 4 16:26:29 2016 +0100
Committer: Duncan Grant <du...@cloudsoftcorp.com>
Committed: Thu Oct 6 14:53:06 2016 +0100

----------------------------------------------------------------------
 .../ConfigLocationInheritanceYamlTest.java      |   6 +-
 ...loudsCustomizerInstantiationYamlDslTest.java | 178 +++++++++++++++++++
 .../command/support/CloudExplorerSupport.java   |  30 ++--
 .../location/jclouds/JcloudsLocation.java       |  17 +-
 .../jclouds/JcloudsLocationCustomizer.java      |   4 +
 .../jclouds/BailOutJcloudsLocation.java         |   9 +-
 6 files changed, 218 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/c63a3c1c/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java
index ff5dcae..6d49947 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java
@@ -21,6 +21,7 @@ package org.apache.brooklyn.camp.brooklyn;
 import static org.testng.Assert.assertEquals;
 
 import java.net.URI;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
@@ -36,6 +37,7 @@ import org.apache.brooklyn.core.location.Machines;
 import org.apache.brooklyn.location.jclouds.ComputeServiceRegistry;
 import org.apache.brooklyn.location.jclouds.JcloudsLocation;
 import org.apache.brooklyn.location.jclouds.JcloudsLocationConfig;
+import org.apache.brooklyn.location.jclouds.JcloudsLocationCustomizer;
 import org.apache.brooklyn.location.jclouds.JcloudsLocationResolver;
 import org.apache.brooklyn.location.jclouds.StubbedComputeServiceRegistry;
 import org.apache.brooklyn.location.jclouds.StubbedComputeServiceRegistry.SingleNodeCreator;
@@ -139,9 +141,9 @@ public class ConfigLocationInheritanceYamlTest extends AbstractYamlTest {
     public static class RecordingJcloudsLocation extends JcloudsLocation {
         public final List<ConfigBag> templateConfigs = Lists.newCopyOnWriteArrayList();
         
-        public Template buildTemplate(ComputeService computeService, ConfigBag config) {
+        public Template buildTemplate(ComputeService computeService, ConfigBag config, Collection<JcloudsLocationCustomizer> customizers) {
             templateConfigs.add(config);
-            return super.buildTemplate(computeService, config);
+            return super.buildTemplate(computeService, config, customizers);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/c63a3c1c/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsCustomizerInstantiationYamlDslTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsCustomizerInstantiationYamlDslTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsCustomizerInstantiationYamlDslTest.java
new file mode 100644
index 0000000..9533280
--- /dev/null
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsCustomizerInstantiationYamlDslTest.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.brooklyn.camp.brooklyn;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertSame;
+
+import java.util.List;
+import java.util.Map;
+
+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.camp.brooklyn.spi.creation.CampTypePlanTransformer;
+import org.apache.brooklyn.core.entity.trait.Startable;
+import org.apache.brooklyn.core.location.Machines;
+import org.apache.brooklyn.core.typereg.RegisteredTypeLoadingContexts;
+import org.apache.brooklyn.entity.machine.MachineEntity;
+import org.apache.brooklyn.location.jclouds.BasicJcloudsLocationCustomizer;
+import org.apache.brooklyn.location.jclouds.ComputeServiceRegistry;
+import org.apache.brooklyn.location.jclouds.JcloudsLocation;
+import org.apache.brooklyn.location.jclouds.JcloudsMachineLocation;
+import org.apache.brooklyn.location.ssh.SshMachineLocation;
+import org.apache.brooklyn.util.collections.MutableList;
+import org.apache.brooklyn.util.core.internal.ssh.RecordingSshTool;
+import org.jclouds.compute.ComputeService;
+import org.jclouds.compute.domain.Template;
+import org.jclouds.compute.domain.TemplateBuilder;
+import org.jclouds.compute.options.TemplateOptions;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+/**
+ * The test is designed to ensure that when a customizer is configured in
+ * yaml with fields that are configured via DSL (forcing brooklyn to
+ * return a {@link org.apache.brooklyn.camp.brooklyn.spi.dsl.methods.BrooklynDslCommon.DslObject})
+ * that only one customizer is instantiated so that state may be maintained between customize calls.
+ *
+ * e.g.
+ *
+ * brooklyn.config:
+ *   provisioning.properties:
+ *     customizers:
+ *     - $brooklyn:object:
+ *       type: org.apache.brooklyn.location.jclouds.networking.SharedLocationSecurityGroupCustomizer
+ *       object.fields:
+ *         - enabled: $brooklyn:config("kubernetes.sharedsecuritygroup.create")
+ *
+ */
+@Test(groups = {"Live", "Live-sanity"})
+public class JcloudsCustomizerInstantiationYamlDslTest extends JcloudsRebindStubYamlTest {
+
+    protected Entity origApp;
+
+    @Override
+    protected JcloudsLocation newJcloudsLocation(ComputeServiceRegistry computeServiceRegistry) throws Exception {
+        ByonComputeServiceStaticRef.setInstance(computeServiceRegistry);
+
+        String yaml = Joiner.on("\n").join(
+                "location:",
+                "  " + LOCATION_SPEC + ":",
+                "    imageId: " + IMAGE_ID,
+                "    jclouds.computeServiceRegistry:",
+                "      $brooklyn:object:",
+                "        type: " + ByonComputeServiceStaticRef.class.getName(),
+                "    " + SshMachineLocation.SSH_TOOL_CLASS.getName() + ": " + RecordingSshTool.class.getName(),
+                "    waitForSshable: false",
+                "    useJcloudsSshInit: false",
+                "services:\n" +
+                "- type: " + MachineEntity.class.getName(),
+                "  brooklyn.config:",
+                "    onbox.base.dir.skipResolution: true",
+                "    sshMonitoring.enabled: false",
+                "    metrics.usage.retrieve: false",
+                "    enabled: true",
+                "    provisioning.properties:",
+                "      customizer:",
+                "        $brooklyn:object:",
+                "          type: " + RecordingLocationCustomizer.class.getName(),
+                "          object.fields:",
+                "            enabled: $brooklyn:config(\"enabled\")");
+
+        EntitySpec<?> spec = mgmt().getTypeRegistry().createSpecFromPlan(CampTypePlanTransformer.FORMAT, yaml, RegisteredTypeLoadingContexts.spec(Application.class), EntitySpec.class);
+        origApp = mgmt().getEntityManager().createEntity(spec);
+
+        return (JcloudsLocation) Iterables.getOnlyElement(origApp.getLocations());
+    }
+
+    @Override
+    protected JcloudsMachineLocation obtainMachine(JcloudsLocation jcloudsLoc, Map<?, ?> props) throws Exception {
+        final MachineEntity entity = (MachineEntity) Iterables.getOnlyElement(origApp.getChildren());
+        origApp.invoke(Startable.START, ImmutableMap.<String, Object>of()).get();
+
+        // Assert all customize functions called
+        assertEquals(RecordingLocationCustomizer.calls.size(), 4,
+                "size=" + RecordingLocationCustomizer.calls.size() + "; calls=" + RecordingLocationCustomizer.calls);
+
+        // Assert same instance used for all calls
+        RecordingLocationCustomizer firstInstance = RecordingLocationCustomizer.calls.get(0).instance;
+        for (RecordingLocationCustomizer.CallParams call : RecordingLocationCustomizer.calls) {
+            assertSame(call.instance, firstInstance);
+        }
+
+        JcloudsMachineLocation machine =
+                Machines.findUniqueMachineLocation(entity.getLocations(), JcloudsMachineLocation.class).get();
+
+        return machine;
+    }
+
+
+    public static class RecordingLocationCustomizer extends BasicJcloudsLocationCustomizer {
+
+        public static final List<CallParams> calls = Lists.newCopyOnWriteArrayList();
+        private Boolean enabled;
+
+        @Override
+        public void customize(JcloudsLocation location, ComputeService computeService, TemplateBuilder templateBuilder) {
+            calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, templateBuilder)));
+        }
+
+        @Override
+        public void customize(JcloudsLocation location, ComputeService computeService, Template template) {
+            calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, template)));
+        }
+
+        @Override
+        public void customize(JcloudsLocation location, ComputeService computeService, TemplateOptions templateOptions) {
+            calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, templateOptions)));
+        }
+
+        @Override
+        public void customize(JcloudsLocation location, ComputeService computeService, JcloudsMachineLocation machine) {
+            calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, machine)));
+        }
+
+        @Override
+        public void preRelease(JcloudsMachineLocation machine) {
+            calls.add(new CallParams(this, "preRelease", MutableList.of(machine)));
+        }
+
+        @Override
+        public void postRelease(JcloudsMachineLocation machine) {
+            calls.add(new CallParams(this, "postRelease", MutableList.of(machine)));
+        }
+
+        public static class CallParams {
+            RecordingLocationCustomizer instance;
+            String method;
+            List<?> args;
+
+            public CallParams(RecordingLocationCustomizer instance, String method, List<?> args) {
+                this.instance = instance;
+                this.method = method;
+                this.args = args;
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/c63a3c1c/launcher-common/src/main/java/org/apache/brooklyn/launcher/command/support/CloudExplorerSupport.java
----------------------------------------------------------------------
diff --git a/launcher-common/src/main/java/org/apache/brooklyn/launcher/command/support/CloudExplorerSupport.java b/launcher-common/src/main/java/org/apache/brooklyn/launcher/command/support/CloudExplorerSupport.java
index 5904ae5..6708d81 100644
--- a/launcher-common/src/main/java/org/apache/brooklyn/launcher/command/support/CloudExplorerSupport.java
+++ b/launcher-common/src/main/java/org/apache/brooklyn/launcher/command/support/CloudExplorerSupport.java
@@ -18,16 +18,25 @@
  */
 package org.apache.brooklyn.launcher.command.support;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.google.common.collect.Lists;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+
 import org.apache.brooklyn.api.location.Location;
 import org.apache.brooklyn.api.location.LocationDefinition;
 import org.apache.brooklyn.api.mgmt.ManagementContext;
 import org.apache.brooklyn.core.location.LocationConfigKeys;
 import org.apache.brooklyn.core.location.cloud.CloudLocationConfig;
 import org.apache.brooklyn.location.jclouds.JcloudsLocation;
+import org.apache.brooklyn.location.jclouds.JcloudsLocationCustomizer;
 import org.apache.brooklyn.location.jclouds.JcloudsUtil;
+import org.apache.brooklyn.util.core.config.ConfigBag;
 import org.apache.brooklyn.util.exceptions.FatalConfigurationRuntimeException;
 import org.apache.brooklyn.util.stream.Streams;
 import org.jclouds.blobstore.BlobStore;
@@ -42,14 +51,9 @@ import org.jclouds.compute.domain.NodeMetadata;
 import org.jclouds.compute.domain.Template;
 import org.jclouds.compute.options.TemplateOptions;
 
-import java.io.InputStream;
-import java.io.PrintStream;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-
-import static com.google.common.base.Preconditions.checkNotNull;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
 
 /**
  * Convenience for listing Cloud Compute and BlobStore details.
@@ -272,8 +276,10 @@ public abstract class CloudExplorerSupport implements Callable<Void> {
         protected void doCall(JcloudsLocation loc, String indent) throws Exception {
 
             ComputeService computeService = loc.getComputeService();
+            ConfigBag setup = loc.config().getBag();
+            Collection<JcloudsLocationCustomizer> customizers = loc.getCustomizers(setup);
 
-            Template template = loc.buildTemplate(computeService, loc.config().getBag());
+            Template template = loc.buildTemplate(computeService, setup, customizers);
             Image image = template.getImage();
             Hardware hardware = template.getHardware();
             org.jclouds.domain.Location location = template.getLocation();

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/c63a3c1c/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
index 1429a54..8469086 100644
--- a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
@@ -410,7 +410,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         }
     }
 
-    protected Collection<JcloudsLocationCustomizer> getCustomizers(ConfigBag setup) {
+    public Collection<JcloudsLocationCustomizer> getCustomizers(ConfigBag setup) {
         @SuppressWarnings("deprecation")
         JcloudsLocationCustomizer customizer = setup.get(JCLOUDS_LOCATION_CUSTOMIZER);
         Collection<JcloudsLocationCustomizer> customizers = setup.get(JCLOUDS_LOCATION_CUSTOMIZERS);
@@ -691,9 +691,10 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
             LoginCredentials userCredentials = null;
             Set<? extends NodeMetadata> nodes;
             Template template;
+            Collection<JcloudsLocationCustomizer> customizers = getCustomizers(setup);
             try {
                 // Setup the template
-                template = buildTemplate(computeService, setup);
+                template = buildTemplate(computeService, setup, customizers);
                 boolean expectWindows = isWindows(template, setup);
                 if (!skipJcloudsSshing) {
                     if (expectWindows) {
@@ -725,7 +726,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                     }
                 }
                 
-                customizeTemplate(setup, computeService, template);
+                customizeTemplate(computeService, template, customizers);
                 
                 LOG.debug("jclouds using template {} / options {} to provision machine in {}",
                         new Object[] {template, template.getOptions(), setup.getDescription()});
@@ -1051,7 +1052,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
             }
 
             // Apply any optional app-specific customization.
-            for (JcloudsLocationCustomizer customizer : getCustomizers(setup)) {
+            for (JcloudsLocationCustomizer customizer : customizers) {
                 LOG.debug("Customizing machine {}, using customizer {}", machineLocation, customizer);
                 customizer.customize(this, computeService, machineLocation);
             }
@@ -1574,8 +1575,8 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
     }
 
     /** hook whereby template customizations can be made for various clouds */
-    protected void customizeTemplate(ConfigBag setup, ComputeService computeService, Template template) {
-        for (JcloudsLocationCustomizer customizer : getCustomizers(setup)) {
+    protected void customizeTemplate(ComputeService computeService, Template template, Collection<JcloudsLocationCustomizer> customizers) {
+        for (JcloudsLocationCustomizer customizer : customizers) {
             customizer.customize(this, computeService, template);
             customizer.customize(this, computeService, template.getOptions());
         }
@@ -1641,7 +1642,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
     }
     
     /** returns the jclouds Template which describes the image to be built, for the given config and compute service */
-    public Template buildTemplate(ComputeService computeService, ConfigBag config) {
+    public Template buildTemplate(ComputeService computeService, ConfigBag config, Collection<JcloudsLocationCustomizer> customizers) {
         TemplateBuilder templateBuilder = (TemplateBuilder) config.get(TEMPLATE_BUILDER);
         if (templateBuilder==null) {
             templateBuilder = new PortableTemplateBuilder<PortableTemplateBuilder<?>>();
@@ -1687,7 +1688,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         }
 
         // Then apply any optional app-specific customization.
-        for (JcloudsLocationCustomizer customizer : getCustomizers(config)) {
+        for (JcloudsLocationCustomizer customizer : customizers) {
             customizer.customize(this, computeService, templateBuilder);
         }
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/c63a3c1c/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java
index d768cb4..89e22bb 100644
--- a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java
@@ -32,6 +32,10 @@ import org.apache.brooklyn.util.core.config.ConfigBag;
  * Instances will be invoked with the {@link ConfigBag} being used to obtain a machine by the
  * {@link JcloudsLocation} if such a constructor exists. If not, the default no argument constructor
  * will be invoked.
+ *
+ * Customizers are not persisted so the pre and postRelease will not be called on the same instance as was used in
+ * provisioning.  However the customize functions will be called on the same instance unless brooklyn is stopped, in which
+ * case vm provisioning would fail anyway.
  */
 public interface JcloudsLocationCustomizer {
 

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/c63a3c1c/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/BailOutJcloudsLocation.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/BailOutJcloudsLocation.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/BailOutJcloudsLocation.java
index 28d1c7b..6ef5656 100644
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/BailOutJcloudsLocation.java
+++ b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/BailOutJcloudsLocation.java
@@ -19,6 +19,7 @@
 
 package org.apache.brooklyn.location.jclouds;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 
@@ -73,13 +74,13 @@ public class BailOutJcloudsLocation extends JcloudsLocation {
     }
 
     @Override
-    public Template buildTemplate(ComputeService computeService, ConfigBag config) {
+    public Template buildTemplate(ComputeService computeService, ConfigBag config, Collection<JcloudsLocationCustomizer> customizers) {
         lastConfigBag = config;
         if (getConfig(BUILD_TEMPLATE_INTERCEPTOR) != null) {
             getConfig(BUILD_TEMPLATE_INTERCEPTOR).apply(config);
         }
         if (Boolean.TRUE.equals(getConfig(BUILD_TEMPLATE))) {
-            template = super.buildTemplate(computeService, config);
+            template = super.buildTemplate(computeService, config, customizers);
         }
         throw new RuntimeException(BAIL_OUT_FOR_TESTING);
     }
@@ -149,9 +150,9 @@ public class BailOutJcloudsLocation extends JcloudsLocation {
     public static class CountingBailOutJcloudsLocation extends BailOutJcloudsLocation {
         int buildTemplateCount = 0;
         @Override
-        public Template buildTemplate(ComputeService computeService, ConfigBag config) {
+        public Template buildTemplate(ComputeService computeService, ConfigBag config, Collection<JcloudsLocationCustomizer> customizers) {
             buildTemplateCount++;
-            return super.buildTemplate(computeService, config);
+            return super.buildTemplate(computeService, config, customizers);
         }
     }
 


[2/3] brooklyn-server git commit: pr371 tidies: instantiating jclouds customizers

Posted by al...@apache.org.
pr371 tidies: instantiating jclouds customizers


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

Branch: refs/heads/master
Commit: a5470a8b31baca9194353ffebaed8c9be8cedd06
Parents: c63a3c1
Author: Aled Sage <al...@gmail.com>
Authored: Thu Oct 6 11:04:04 2016 +0100
Committer: Duncan Grant <du...@cloudsoftcorp.com>
Committed: Thu Oct 6 14:53:11 2016 +0100

----------------------------------------------------------------------
 .../api/location/MachineLocationCustomizer.java |  5 ++
 .../ConfigLocationInheritanceYamlTest.java      |  2 +
 ...loudsCustomizerInstantiationYamlDslTest.java | 59 +++++++++++++++++---
 .../location/jclouds/JcloudsLocation.java       | 15 +++--
 .../location/jclouds/JcloudsLocationConfig.java |  4 +-
 .../jclouds/JcloudsLocationCustomizer.java      | 20 ++++---
 6 files changed, 81 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a5470a8b/api/src/main/java/org/apache/brooklyn/api/location/MachineLocationCustomizer.java
----------------------------------------------------------------------
diff --git a/api/src/main/java/org/apache/brooklyn/api/location/MachineLocationCustomizer.java b/api/src/main/java/org/apache/brooklyn/api/location/MachineLocationCustomizer.java
index a9b4e2e..028efa1 100644
--- a/api/src/main/java/org/apache/brooklyn/api/location/MachineLocationCustomizer.java
+++ b/api/src/main/java/org/apache/brooklyn/api/location/MachineLocationCustomizer.java
@@ -25,6 +25,11 @@ import com.google.common.annotations.Beta;
  * <p>
  * Users are strongly encouraged to sub-class {@link BasicMachineLocationCustomizer}, to give
  * some protection against this {@link Beta} API changing in future releases.
+ * 
+ * Customizers can be instantiated on-demand, so the {@link #customize(MachineLocation)}
+ * and {@link #preRelease(MachineLocation)} methods may not be called on the same instance. 
+ * This is always true after a Brooklyn restart, and may be true at other times depending 
+ * how the customizer has been wired in.
  */
 @Beta
 public interface MachineLocationCustomizer {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a5470a8b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java
index 6d49947..db84b59 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/ConfigLocationInheritanceYamlTest.java
@@ -141,6 +141,7 @@ public class ConfigLocationInheritanceYamlTest extends AbstractYamlTest {
     public static class RecordingJcloudsLocation extends JcloudsLocation {
         public final List<ConfigBag> templateConfigs = Lists.newCopyOnWriteArrayList();
         
+        @Override
         public Template buildTemplate(ComputeService computeService, ConfigBag config, Collection<JcloudsLocationCustomizer> customizers) {
             templateConfigs.add(config);
             return super.buildTemplate(computeService, config, customizers);
@@ -161,6 +162,7 @@ public class ConfigLocationInheritanceYamlTest extends AbstractYamlTest {
                 return "jclouds-config-test";
             }
 
+            @Override
             protected Class<? extends JcloudsLocation> getLocationClass() {
                 return RecordingJcloudsLocation.class;
             }

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a5470a8b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsCustomizerInstantiationYamlDslTest.java
----------------------------------------------------------------------
diff --git a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsCustomizerInstantiationYamlDslTest.java b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsCustomizerInstantiationYamlDslTest.java
index 9533280..7083fc2 100644
--- a/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsCustomizerInstantiationYamlDslTest.java
+++ b/camp/camp-brooklyn/src/test/java/org/apache/brooklyn/camp/brooklyn/JcloudsCustomizerInstantiationYamlDslTest.java
@@ -43,6 +43,8 @@ import org.jclouds.compute.ComputeService;
 import org.jclouds.compute.domain.Template;
 import org.jclouds.compute.domain.TemplateBuilder;
 import org.jclouds.compute.options.TemplateOptions;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
 import com.google.common.base.Joiner;
@@ -58,6 +60,8 @@ import com.google.common.collect.Lists;
  *
  * e.g.
  *
+ * <pre>
+ * {@code
  * brooklyn.config:
  *   provisioning.properties:
  *     customizers:
@@ -65,13 +69,31 @@ import com.google.common.collect.Lists;
  *       type: org.apache.brooklyn.location.jclouds.networking.SharedLocationSecurityGroupCustomizer
  *       object.fields:
  *         - enabled: $brooklyn:config("kubernetes.sharedsecuritygroup.create")
- *
+ * }
+ * </pre>
  */
 @Test(groups = {"Live", "Live-sanity"})
 public class JcloudsCustomizerInstantiationYamlDslTest extends JcloudsRebindStubYamlTest {
 
     protected Entity origApp;
 
+    @BeforeMethod(alwaysRun=true)
+    @Override
+    public void setUp() throws Exception {
+        RecordingLocationCustomizer.clear();
+        super.setUp();
+    }
+    
+    @AfterMethod(alwaysRun=true)
+    @Override
+    public void tearDown() throws Exception {
+        try {
+            super.tearDown();
+        } finally {
+            RecordingLocationCustomizer.clear();
+        }
+    }
+    
     @Override
     protected JcloudsLocation newJcloudsLocation(ComputeServiceRegistry computeServiceRegistry) throws Exception {
         ByonComputeServiceStaticRef.setInstance(computeServiceRegistry);
@@ -102,7 +124,7 @@ public class JcloudsCustomizerInstantiationYamlDslTest extends JcloudsRebindStub
 
         EntitySpec<?> spec = mgmt().getTypeRegistry().createSpecFromPlan(CampTypePlanTransformer.FORMAT, yaml, RegisteredTypeLoadingContexts.spec(Application.class), EntitySpec.class);
         origApp = mgmt().getEntityManager().createEntity(spec);
-
+        
         return (JcloudsLocation) Iterables.getOnlyElement(origApp.getLocations());
     }
 
@@ -131,36 +153,57 @@ public class JcloudsCustomizerInstantiationYamlDslTest extends JcloudsRebindStub
     public static class RecordingLocationCustomizer extends BasicJcloudsLocationCustomizer {
 
         public static final List<CallParams> calls = Lists.newCopyOnWriteArrayList();
+
+        public static void clear() {
+            calls.clear();
+        }
+
         private Boolean enabled;
 
+        public void setEnabled(Boolean val) {
+            this.enabled = val;
+        }
+        
         @Override
         public void customize(JcloudsLocation location, ComputeService computeService, TemplateBuilder templateBuilder) {
-            calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, templateBuilder)));
+            if (Boolean.TRUE.equals(enabled)) {
+                calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, templateBuilder)));
+            }
         }
 
         @Override
         public void customize(JcloudsLocation location, ComputeService computeService, Template template) {
-            calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, template)));
+            if (Boolean.TRUE.equals(enabled)) {
+                calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, template)));
+            }
         }
 
         @Override
         public void customize(JcloudsLocation location, ComputeService computeService, TemplateOptions templateOptions) {
-            calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, templateOptions)));
+            if (Boolean.TRUE.equals(enabled)) {
+                calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, templateOptions)));
+            }
         }
 
         @Override
         public void customize(JcloudsLocation location, ComputeService computeService, JcloudsMachineLocation machine) {
-            calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, machine)));
+            if (Boolean.TRUE.equals(enabled)) {
+                calls.add(new CallParams(this, "customize", MutableList.of(location, computeService, machine)));
+            }
         }
 
         @Override
         public void preRelease(JcloudsMachineLocation machine) {
-            calls.add(new CallParams(this, "preRelease", MutableList.of(machine)));
+            if (Boolean.TRUE.equals(enabled)) {
+                calls.add(new CallParams(this, "preRelease", MutableList.of(machine)));
+            }
         }
 
         @Override
         public void postRelease(JcloudsMachineLocation machine) {
-            calls.add(new CallParams(this, "postRelease", MutableList.of(machine)));
+            if (Boolean.TRUE.equals(enabled)) {
+                calls.add(new CallParams(this, "postRelease", MutableList.of(machine)));
+            }
         }
 
         public static class CallParams {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a5470a8b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
index 8469086..c58bdbd 100644
--- a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocation.java
@@ -692,6 +692,8 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
             Set<? extends NodeMetadata> nodes;
             Template template;
             Collection<JcloudsLocationCustomizer> customizers = getCustomizers(setup);
+            Collection<MachineLocationCustomizer> machineCustomizers = getMachineCustomizers(setup);
+            
             try {
                 // Setup the template
                 template = buildTemplate(computeService, setup, customizers);
@@ -1056,7 +1058,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                 LOG.debug("Customizing machine {}, using customizer {}", machineLocation, customizer);
                 customizer.customize(this, computeService, machineLocation);
             }
-            for (MachineLocationCustomizer customizer : getMachineCustomizers(setup)) {
+            for (MachineLocationCustomizer customizer : machineCustomizers) {
                 LOG.debug("Customizing machine {}, using customizer {}", machineLocation, customizer);
                 customizer.customize(machineLocation);
             }
@@ -1640,7 +1642,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         }
         return BrooklynImageChooser.cloneFor(chooser, computeService, config);
     }
-    
+
     /** returns the jclouds Template which describes the image to be built, for the given config and compute service */
     public Template buildTemplate(ComputeService computeService, ConfigBag config, Collection<JcloudsLocationCustomizer> customizers) {
         TemplateBuilder templateBuilder = (TemplateBuilder) config.get(TEMPLATE_BUILDER);
@@ -2727,7 +2729,10 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
         Exception tothrow = null;
 
         ConfigBag setup = config().getBag();
-        for (JcloudsLocationCustomizer customizer : getCustomizers(setup)) {
+        Collection<JcloudsLocationCustomizer> customizers = getCustomizers(setup);
+        Collection<MachineLocationCustomizer> machineCustomizers = getMachineCustomizers(setup);
+        
+        for (JcloudsLocationCustomizer customizer : customizers) {
             try {
                 customizer.preRelease(machine);
             } catch (Exception e) {
@@ -2737,7 +2742,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
                 if (tothrow==null) tothrow = e;
             }
         }
-        for (MachineLocationCustomizer customizer : getMachineCustomizers(setup)) {
+        for (MachineLocationCustomizer customizer : machineCustomizers) {
             customizer.preRelease(machine);
         }
 
@@ -2764,7 +2769,7 @@ public class JcloudsLocation extends AbstractCloudMachineProvisioningLocation im
 
         removeChild(machine);
 
-        for (JcloudsLocationCustomizer customizer : getCustomizers(setup)) {
+        for (JcloudsLocationCustomizer customizer : customizers) {
             try {
                 customizer.postRelease(machine);
             } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a5470a8b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationConfig.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationConfig.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationConfig.java
index d14c866..3c07f4b 100644
--- a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationConfig.java
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationConfig.java
@@ -203,13 +203,13 @@ public interface JcloudsLocationConfig extends CloudLocationConfig {
     /** @deprecated since 0.7.0; use {@link #JCLOUDS_LOCATION_CUSTOMIZERS} instead */
     @Deprecated
     public static final ConfigKey<String> JCLOUDS_LOCATION_CUSTOMIZER_TYPE = ConfigKeys.newStringConfigKey(
-            "customizerType", "Optional location customizer type (to be class-loaded and constructed with no-arg constructor)");
+            "customizerType", "Optional location customizer type (to be class-loaded and constructed with either a ConfigBag or no-arg constructor)");
 
     /** @deprecated since 0.7.0; use {@link #JCLOUDS_LOCATION_CUSTOMIZERS} instead */
     @Deprecated
     public static final ConfigKey<String> JCLOUDS_LOCATION_CUSTOMIZERS_SUPPLIER_TYPE = ConfigKeys.newStringConfigKey(
             "customizersSupplierType", "Optional type of a Supplier<Collection<JcloudsLocationCustomizer>> " +
-            "(to be class-loaded and constructed with ConfigBag or no-arg constructor)");
+            "(to be class-loaded and constructed with either a ConfigBag or no-arg constructor)");
 
     public static final ConfigKey<String> LOCAL_TEMP_DIR = SshTool.PROP_LOCAL_TEMP_DIR;
     

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a5470a8b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java
index 89e22bb..192077b 100644
--- a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/JcloudsLocationCustomizer.java
@@ -23,19 +23,21 @@ import org.jclouds.compute.domain.Template;
 import org.jclouds.compute.domain.TemplateBuilder;
 import org.jclouds.compute.options.TemplateOptions;
 
-import org.apache.brooklyn.util.core.config.ConfigBag;
-
 /**
  * Customization hooks to allow apps to perform specific customisation at each stage of jclouds machine provisioning.
  * For example, an app could attach an EBS volume to an EC2 node, or configure a desired availability zone.
  * <p>
- * Instances will be invoked with the {@link ConfigBag} being used to obtain a machine by the
- * {@link JcloudsLocation} if such a constructor exists. If not, the default no argument constructor
- * will be invoked.
- *
- * Customizers are not persisted so the pre and postRelease will not be called on the same instance as was used in
- * provisioning.  However the customize functions will be called on the same instance unless brooklyn is stopped, in which
- * case vm provisioning would fail anyway.
+ * Users are strongly encouraged to sub-class {@link org.apache.brooklyn.location.jclouds.BasicJcloudsLocationCustomizer}, 
+ * to give some protection against this API changing in future releases.
+ * <p>
+ * Customizers can be instantiated on-demand, so the {@link #postRelease(JcloudsMachineLocation)}
+ * and {@link #postRelease(JcloudsMachineLocation)} methods may not be called on the same instance 
+ * as was used in provisioning. This is always true after a Brooklyn restart, and may be true at 
+ * other times depending how the customizer has been wired in.
+ * <p>
+ * However the customize functions will be called sequentially on the same instance during provisioning,
+ * unless Brooklyn is stopped (or fails over to a high-availability standby), in which case VM 
+ * provisioning would abort anyway.
  */
 public interface JcloudsLocationCustomizer {
 


[3/3] brooklyn-server git commit: This closes #371

Posted by al...@apache.org.
This closes #371


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

Branch: refs/heads/master
Commit: bc49423805ee70758ec7f5b11849bff8ecdd51f1
Parents: 717fe35 a5470a8
Author: Aled Sage <al...@gmail.com>
Authored: Thu Oct 6 17:17:43 2016 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Thu Oct 6 17:17:43 2016 +0100

----------------------------------------------------------------------
 .../api/location/MachineLocationCustomizer.java |   5 +
 .../ConfigLocationInheritanceYamlTest.java      |   8 +-
 ...loudsCustomizerInstantiationYamlDslTest.java | 221 +++++++++++++++++++
 .../command/support/CloudExplorerSupport.java   |  30 ++-
 .../location/jclouds/JcloudsLocation.java       |  32 +--
 .../location/jclouds/JcloudsLocationConfig.java |   4 +-
 .../jclouds/JcloudsLocationCustomizer.java      |  16 +-
 .../jclouds/BailOutJcloudsLocation.java         |   9 +-
 8 files changed, 287 insertions(+), 38 deletions(-)
----------------------------------------------------------------------