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 2016/11/23 12:17:38 UTC

[1/2] brooklyn-server git commit: Factor out the functionality for security groups.

Repository: brooklyn-server
Updated Branches:
  refs/heads/master 836cc2ba1 -> 95cf0f9a6


Factor out the functionality for security groups.

Brings the create/delete and add/remove permissions functionality
for security groups from JcloudsLocationSecurityGroupCustomizer
out into its own class. The aim is to make it more convenient
to write other customizers etc. that might need security groups.

This is intended as nearly as possible as a behaviour-preserving
change. I have permitted myself one change to the behaviour where
this seemed sensible; the loop in `runOperationWithRetry`
was effectively an infinite loop because of its 100 iteration
exponential back-off, so I have changed this to a back-off that
tries for about five minutes.


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

Branch: refs/heads/master
Commit: a27656c431a38cd7926288ddded7516532dd6ecf
Parents: a569463
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Authored: Tue Nov 8 21:37:48 2016 +0000
Committer: Geoff Macartney <ge...@cloudsoftcorp.com>
Committed: Mon Nov 21 10:49:53 2016 +0000

----------------------------------------------------------------------
 .../JcloudsLocationSecurityGroupCustomizer.java | 260 +++++++----------
 .../jclouds/networking/SecurityGroupEditor.java | 284 +++++++++++++++++++
 .../SharedLocationSecurityGroupCustomizer.java  |   2 +-
 ...oudsLocationSecurityGroupCustomizerTest.java | 112 ++++----
 .../networking/SecurityGroupLiveTest.java       |  32 ---
 ...aredLocationSecurityGroupCustomizerTest.java |   2 +-
 .../test/location/SecurityGroupLiveTest.java    | 229 +++++++++++++++
 7 files changed, 682 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a27656c4/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizer.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizer.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizer.java
index f88106a..4f7ee5a 100644
--- a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizer.java
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizer.java
@@ -171,18 +171,24 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
     }
 
     /** @see #addPermissionsToLocation(JcloudsMachineLocation, java.lang.Iterable) */
-    public JcloudsLocationSecurityGroupCustomizer addPermissionsToLocation(final JcloudsMachineLocation location, IpPermission... permissions) {
+    public JcloudsLocationSecurityGroupCustomizer addPermissionsToLocation(final JcloudsMachineLocation location,
+            IpPermission... permissions) {
+
         addPermissionsToLocation(location, ImmutableList.copyOf(permissions));
         return this;
     }
 
     /** @see #addPermissionsToLocation(JcloudsMachineLocation, java.lang.Iterable) */
-    public JcloudsLocationSecurityGroupCustomizer addPermissionsToLocation(final JcloudsMachineLocation location, SecurityGroupDefinition securityGroupDefinition) {
+    public JcloudsLocationSecurityGroupCustomizer addPermissionsToLocation(final JcloudsMachineLocation location,
+             SecurityGroupDefinition securityGroupDefinition) {
+
         addPermissionsToLocation(location, securityGroupDefinition.getPermissions());
         return this;
     }
 
-    private SecurityGroup getSecurityGroup(final String nodeId, final SecurityGroupExtension securityApi, final String locationId) {
+    private SecurityGroup getMachineUniqueSecurityGroup(final String nodeId, final String locationId,
+            final SecurityGroupEditor groupEditor) {
+
         // Expect to have two security groups on the node: one shared between all nodes in the location,
         // that is cached in sharedGroupCache, and one created by Jclouds that is unique to the node.
         // Relies on customize having been called before. This should be safe because the arguments
@@ -192,7 +198,8 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
         try {
             machineUniqueSecurityGroup = uniqueGroupCache.get(nodeId, new Callable<SecurityGroup>() {
                 @Override public SecurityGroup call() throws Exception {
-                    SecurityGroup sg = getUniqueSecurityGroupForNodeCachingSharedGroupIfPreviouslyUnknown(nodeId, locationId, securityApi);
+                    SecurityGroup sg =
+                        getUniqueSecurityGroupForNodeCachingSharedGroupIfUnknown(nodeId, locationId, groupEditor);
                     if (sg == null) {
                         throw new IllegalStateException("Failed to find machine-unique group on node: " + nodeId);
                     }
@@ -219,60 +226,62 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
      * the changes may not be picked up by later customizations, meaning the same rule could possibly be
      * added twice, which would fail. A finer grained mechanism would be preferable here, but
      * we have no access to the information required, so this brute force serializing is required.
+     * TODO investigate whether this can be improved. Can the synchronization be moved to
+     * the class org.apache.brooklyn.location.jclouds.networking.SecurityGroupEditor?
      *
      * @param location Location to gain permissions
      * @param permissions The set of permissions to be applied to the location
      */
-    public JcloudsLocationSecurityGroupCustomizer addPermissionsToLocation(final JcloudsMachineLocation location, final Iterable<IpPermission> permissions) {
-        ComputeService computeService = location.getParent().getComputeService();
-        addPermissionsToLocationAndReturnSecurityGroup(computeService, location, permissions);
+    public JcloudsLocationSecurityGroupCustomizer addPermissionsToLocation(final JcloudsMachineLocation location,
+              final Iterable<IpPermission> permissions) {
+        addPermissionsToLocationAndReturnSecurityGroup(location, permissions);
         return this;
     }
 
-    public Collection<SecurityGroup> addPermissionsToLocationAndReturnSecurityGroup(ComputeService computeService, final JcloudsMachineLocation location, final Iterable<IpPermission> permissions) {
+    public Collection<SecurityGroup> addPermissionsToLocationAndReturnSecurityGroup(
+        final JcloudsMachineLocation location, final Iterable<IpPermission> permissions) {
+
         synchronized (JcloudsLocationSecurityGroupCustomizer.class) {
-            String nodeId = location.getNode().getId();
-            return addPermissionsToLocation(permissions, nodeId, computeService).values();
+            return addPermissionsInternal(permissions, location).values();
         }
     }
+
     /**
-     * Removes the given security group permissions from the given node with the given compute service.
+     * Removes the given security group permissions from the given node.
      * <p>
      * Takes no action if the compute service does not have a security group extension.
-     * @param permissions The set of permissions to be removed from the location
-     * @param location Location to remove permissions from
+     * @param location Location of the node to remove permissions from
+     * @param permissions The set of permissions to be removed from the node
      */
-    public void removePermissionsFromLocation(final JcloudsMachineLocation location, final Iterable<IpPermission> permissions) {
+    public void removePermissionsFromLocation(JcloudsMachineLocation location, Iterable<IpPermission> permissions) {
+
         synchronized (JcloudsLocationSecurityGroupCustomizer.class) {
-            ComputeService computeService = location.getParent().getComputeService();
-            String nodeId = location.getNode().getId();
-            removePermissionsFromLocation(permissions, nodeId, computeService);
+            removePermissionsInternal(location, permissions);
         }
     }
 
     /**
-     * Removes the given security group permissions from the given node with the given compute service.
+     * Removes the given security group permissions from the given node.
      * <p>
      * Takes no action if the compute service does not have a security group extension.
+     * @param location Location of the node to remove permissions from
      * @param permissions The set of permissions to be removed from the node
-     * @param nodeId The id of the node to update
-     * @param computeService The compute service to use to apply the changes
      */
-    @VisibleForTesting
-    void removePermissionsFromLocation(Iterable<IpPermission> permissions, final String nodeId, ComputeService computeService) {
-        if (!computeService.getSecurityGroupExtension().isPresent()) {
+    private void removePermissionsInternal(JcloudsMachineLocation location, Iterable<IpPermission> permissions) {
+        ComputeService computeService = location.getParent().getComputeService();
+        String nodeId = location.getNode().getId();
+
+        final Optional<SecurityGroupExtension> securityApi = computeService.getSecurityGroupExtension();
+        if (!securityApi.isPresent()) {
             LOG.warn("Security group extension for {} absent; cannot update node {} with {}",
                     new Object[] {computeService, nodeId, permissions});
             return;
         }
 
-        final SecurityGroupExtension securityApi = computeService.getSecurityGroupExtension().get();
-        final String locationId = computeService.getContext().unwrap().getId();
-        SecurityGroup machineUniqueSecurityGroup = getSecurityGroup(nodeId, securityApi, locationId);
-
-        for (IpPermission permission : permissions) {
-            removePermission(permission, machineUniqueSecurityGroup, securityApi);
-        }
+        final SecurityGroupEditor editor = createSecurityGroupEditor(securityApi.get(), location.getNode().getLocation());
+        String locationId = computeService.getContext().unwrap().getId();
+        SecurityGroup machineUniqueSecurityGroup = getMachineUniqueSecurityGroup(nodeId, locationId, editor);
+        editor.removePermissions(machineUniqueSecurityGroup, permissions);
     }
 
 
@@ -282,32 +291,34 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
      * <p>
      * Takes no action if the compute service does not have a security group extension.
      * @param permissions The set of permissions to be applied to the node
-     * @param nodeId The id of the node to update
-     * @param computeService The compute service to use to apply the changes
+     * @param location
      */
-    @VisibleForTesting
-    Map<String, SecurityGroup> addPermissionsToLocation(Iterable<IpPermission> permissions, final String nodeId, ComputeService computeService) {
-        if (!computeService.getSecurityGroupExtension().isPresent()) {
+    private Map<String, SecurityGroup> addPermissionsInternal(Iterable<IpPermission> permissions,
+            JcloudsMachineLocation location) {
+
+        String nodeId = location.getNode().getId();
+        final Location nodeLocation = location.getNode().getLocation();
+        ComputeService computeService = location.getParent().getComputeService();
+
+        final Optional<SecurityGroupExtension> securityApi = computeService.getSecurityGroupExtension();
+        if (!securityApi.isPresent()) {
             LOG.warn("Security group extension for {} absent; cannot update node {} with {}",
-                    new Object[] {computeService, nodeId, permissions});
+                new Object[] {computeService, nodeId, permissions});
             return ImmutableMap.of();
         }
-        final SecurityGroupExtension securityApi = computeService.getSecurityGroupExtension().get();
-        final String locationId = computeService.getContext().unwrap().getId();
+
+        final SecurityGroupEditor groupEditor = createSecurityGroupEditor(securityApi.get(), nodeLocation);
 
         // Expect to have two security groups on the node: one shared between all nodes in the location,
         // that is cached in sharedGroupCache, and one created by Jclouds that is unique to the node.
         // Relies on customize having been called before. This should be safe because the arguments
         // needed to call this method are not available until post-instance creation.
-        SecurityGroup machineUniqueSecurityGroup = getSecurityGroup(nodeId, securityApi, locationId);
+        String locationId = computeService.getContext().unwrap().getId();
+        SecurityGroup machineUniqueSecurityGroup = getMachineUniqueSecurityGroup(nodeId, locationId, groupEditor);
         MutableList<IpPermission> newPermissions = MutableList.copyOf(permissions);
         Iterables.removeAll(newPermissions, machineUniqueSecurityGroup.getIpPermissions());
-        MutableMap<String, SecurityGroup> addedSecurityGroups = MutableMap.of();
-        for (IpPermission permission : newPermissions) {
-            SecurityGroup addedPermission = addPermission(permission, machineUniqueSecurityGroup, securityApi);
-            addedSecurityGroups.put(addedPermission.getId(), addedPermission);
-        }
-        return addedSecurityGroups;
+        machineUniqueSecurityGroup = groupEditor.addPermissions(machineUniqueSecurityGroup, newPermissions);
+        return MutableMap.of(machineUniqueSecurityGroup.getId(), machineUniqueSecurityGroup);
     }
 
     /**
@@ -320,14 +331,16 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
      * look for the uniqueSecurityGroup rather than the shared securityGroup.
      *
      * @param nodeId The id of the node in question
-     * @param locationId The id of the location in question
-     * @param securityApi The API to use to list security groups
+     * @param locationId The ID of the location of this node
+     * @param groupEditor security group tool
      * @return the security group unique to the given node, or null if one could not be determined.
      */
-    private SecurityGroup getUniqueSecurityGroupForNodeCachingSharedGroupIfPreviouslyUnknown(String nodeId, String locationId, SecurityGroupExtension securityApi) {
-        Set<SecurityGroup> groupsOnNode = securityApi.listSecurityGroupsForNode(nodeId);
+    private SecurityGroup getUniqueSecurityGroupForNodeCachingSharedGroupIfUnknown(String nodeId,
+                                                                                   final String locationId,
+                                                                                   final SecurityGroupEditor groupEditor) {
 
-        if(groupsOnNode == null || groupsOnNode.isEmpty()){
+        Set<SecurityGroup> groupsOnNode = groupEditor.listSecurityGroupsForNode(nodeId);
+        if(groupsOnNode.isEmpty()){
             return null;
         }
 
@@ -380,24 +393,39 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
      */
     @Override
     public void customize(JcloudsLocation location, ComputeService computeService, Template template) {
-        if (!computeService.getSecurityGroupExtension().isPresent()) {
-            LOG.warn("Security group extension for {} absent; cannot configure security groups in context: {}", computeService, applicationId);
+        final Optional<SecurityGroupExtension> securityApi = computeService.getSecurityGroupExtension();
+        if (!securityApi.isPresent()) {
+            LOG.warn("Security group extension for {} absent; cannot configure security groups in context: {}",
+                computeService, applicationId);
+
         } else if (template.getLocation() == null) {
-            LOG.warn("No location has been set on {}; cannot configure security groups in context: {}", template, applicationId);
+            LOG.warn("No location has been set on {}; cannot configure security groups in context: {}",
+                template, applicationId);
+
         } else {
             LOG.info("Configuring security groups on location {} in context {}", location, applicationId);
-            setSecurityGroupOnTemplate(location, template, computeService.getSecurityGroupExtension().get());
+            SecurityGroupEditor groupEditor = createSecurityGroupEditor(securityApi.get(), template.getLocation());
+
+            setSecurityGroupOnTemplate(location, template, groupEditor);
         }
     }
 
-    private void setSecurityGroupOnTemplate(final JcloudsLocation location, final Template template, final SecurityGroupExtension securityApi) {
+    private SecurityGroupEditor createSecurityGroupEditor(SecurityGroupExtension securityApi,
+                                                          Location location) {
+
+        return new SecurityGroupEditor(location, securityApi, isExceptionRetryable);
+    }
+
+    private void setSecurityGroupOnTemplate(final JcloudsLocation location, final Template template,
+            final SecurityGroupEditor groupEditor) {
+
         SecurityGroup shared;
         Tasks.setBlockingDetails("Loading security group shared by instances in " + template.getLocation() +
                 " in app " + applicationId);
         try {
             shared = sharedGroupCache.get(template.getLocation(), new Callable<SecurityGroup>() {
                 @Override public SecurityGroup call() throws Exception {
-                    return getOrCreateSharedSecurityGroup(template.getLocation(), securityApi);
+                    return getOrCreateSharedSecurityGroup(template.getLocation(), groupEditor);
                 }
             });
         } catch (ExecutionException e) {
@@ -409,7 +437,8 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
         Set<String> originalGroups = template.getOptions().getGroups();
         template.getOptions().securityGroups(shared.getName());
         if (!originalGroups.isEmpty()) {
-            LOG.info("Replaced configured security groups: configured={}, replaced with={}", originalGroups, template.getOptions().getGroups());
+            LOG.info("Replaced configured security groups: configured={}, replaced with={}",
+                originalGroups, template.getOptions().getGroups());
         } else {
             LOG.debug("Configured security groups at {} to: {}", location, template.getOptions().getGroups());
         }
@@ -423,16 +452,12 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
      * @param securityApi The API to use to list and create security groups
      * @return the security group to share between instances in the given location in this application
      */
-    private SecurityGroup getOrCreateSharedSecurityGroup(Location location, SecurityGroupExtension securityApi) {
+    private SecurityGroup getOrCreateSharedSecurityGroup(Location location,
+            SecurityGroupEditor groupEditor) {
+
         final String groupName = getNameForSharedSecurityGroup();
         // Could sort-and-search if straight search is too expensive
-        Optional<SecurityGroup> shared = Iterables.tryFind(securityApi.listSecurityGroupsInLocation(location), new Predicate<SecurityGroup>() {
-            @Override
-            public boolean apply(final SecurityGroup input) {
-                // endsWith because Jclouds prepends 'jclouds#' to security group names.
-                return input.getName().endsWith(groupName);
-            }
-        });
+        Optional<SecurityGroup> shared = groupEditor.findSecurityGroupByName(groupName);
         if (shared.isPresent()) {
             LOG.info("Found existing shared security group in {} for app {}: {}",
                     new Object[]{location, applicationId, groupName});
@@ -440,7 +465,7 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
         } else {
             LOG.info("Creating new shared security group in {} for app {}: {}",
                     new Object[]{location, applicationId, groupName});
-            return createBaseSecurityGroupInLocation(groupName, location, securityApi);
+            return createBaseSecurityGroupInLocation(groupName, groupEditor);
         }
     }
 
@@ -456,17 +481,18 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
      *
      *
      * @param groupName The name of the security group to create
-     * @param location The location in which the security group will be created
      * @param securityApi The API to use to create the security group
      *
      * @return the created security group
      */
-    private SecurityGroup createBaseSecurityGroupInLocation(String groupName, Location location, SecurityGroupExtension securityApi) {
-        SecurityGroup group = addSecurityGroupInLocation(groupName, location, securityApi);
+    private SecurityGroup createBaseSecurityGroupInLocation(String groupName,
+            SecurityGroupEditor groupEditor) {
+
+        SecurityGroup group = groupEditor.createSecurityGroup(groupName);
 
         String groupId = group.getProviderId();
         int fromPort = 0;
-        if (isOpenstackNova(location)) {
+        if (isOpenstackNova(groupEditor.getLocation())) {
             groupId = group.getId();
             fromPort = 1;
         }
@@ -476,10 +502,11 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
                 .groupId(groupId)
                 .fromPort(fromPort)
                 .toPort(65535);
-        addPermission(allWithinGroup.ipProtocol(IpProtocol.TCP).build(), group, securityApi);
-        addPermission(allWithinGroup.ipProtocol(IpProtocol.UDP).build(), group, securityApi);
-        if (!isAzure(location)) {
-            addPermission(allWithinGroup.ipProtocol(IpProtocol.ICMP).fromPort(-1).toPort(-1).build(), group, securityApi);
+        group = groupEditor.addPermission(group, allWithinGroup.ipProtocol(IpProtocol.TCP).build());
+        group = groupEditor.addPermission(group, allWithinGroup.ipProtocol(IpProtocol.UDP).build());
+        if (!isAzure(groupEditor.getLocation())) {
+            group = groupEditor.addPermission(group,
+                allWithinGroup.ipProtocol(IpProtocol.ICMP).fromPort(-1).toPort(-1).build());
         }
 
         IpPermission sshPermission = IpPermission.builder()
@@ -488,7 +515,7 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
                 .ipProtocol(IpProtocol.TCP)
                 .cidrBlock(getBrooklynCidrBlock())
                 .build();
-        addPermission(sshPermission, group, securityApi);
+        group = groupEditor.addPermission(group, sshPermission);
 
         return group;
     }
@@ -519,7 +546,8 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
     }
 
     private boolean isOpenstackNova(Location location) {
-        Set<String> computeIds = getJcloudsLocationIds(ImmutableList.of("openstack-nova", "openstack-mitaka-nova", "openstack-devtest-compute"));
+        Set<String> computeIds = getJcloudsLocationIds(
+            ImmutableList.of("openstack-nova", "openstack-mitaka-nova", "openstack-devtest-compute"));
         return location.getParent() != null && Iterables.contains(computeIds, location.getParent().getId());
     }
     
@@ -528,59 +556,7 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
         return location.getParent() != null && Iterables.contains(computeIds, location.getParent().getId());
     }
     
-    protected SecurityGroup addSecurityGroupInLocation(final String groupName, final Location location, final SecurityGroupExtension securityApi) {
-        LOG.debug("Creating security group {} in {}", groupName, location);
-        Callable<SecurityGroup> callable = new Callable<SecurityGroup>() {
-            @Override
-            public SecurityGroup call() throws Exception {
-                return securityApi.createSecurityGroup(groupName, location);
-            }
-        };
-        return runOperationWithRetry(callable);
-    }
-
-    protected SecurityGroup addPermission(final IpPermission permission, final SecurityGroup group, final SecurityGroupExtension securityApi) {
-        LOG.debug("Adding permission to security group {}: {}", group.getName(), permission);
-        Callable<SecurityGroup> callable = new Callable<SecurityGroup>() {
-            @Override
-            public SecurityGroup call() throws Exception {
-                try {
-                    return securityApi.addIpPermission(permission, group);
-                } catch (AWSResponseException e) {
-                    if ("InvalidPermission.Duplicate".equals(e.getError().getCode())) {
-                        // already exists
-                        LOG.info("Permission already exists for security group; continuing (logging underlying exception at debug): permission="+permission+"; group="+group);
-                        LOG.debug("Permission already exists for security group; continuing: permission="+permission+"; group="+group, e);
-                        return null;
-                    } else {
-                        throw e;
-                    }
-                } catch (Exception e) {
-                    Exceptions.propagateIfFatal(e);
-                    if (e.toString().contains("InvalidPermission.Duplicate")) {
-                        // belt-and-braces, in case 
-                        // already exists
-                        LOG.info("Permission already exists for security group; continuing (but unexpected exception type): permission="+permission+"; group="+group, e);
-                        return null;
-                    } else {
-                        throw Exceptions.propagate(e);
-                    }
-                }
-            }
-        };
-        return runOperationWithRetry(callable);
-    }
 
-    protected SecurityGroup removePermission(final IpPermission permission, final SecurityGroup group, final SecurityGroupExtension securityApi) {
-        LOG.debug("Removing permission from security group {}: {}", group.getName(), permission);
-        Callable<SecurityGroup> callable = new Callable<SecurityGroup>() {
-            @Override
-            public SecurityGroup call() throws Exception {
-                return securityApi.removeIpPermission(permission, group);
-            }
-        };
-        return runOperationWithRetry(callable);
-    }
 
     /** @return the CIDR block used to configure Brooklyn's in security groups */
     public String getBrooklynCidrBlock() {
@@ -607,34 +583,6 @@ public class JcloudsLocationSecurityGroupCustomizer extends BasicJcloudsLocation
         uniqueGroupCache.invalidateAll();
     }
 
-    /**
-     * Runs the given callable. Repeats until the operation succeeds or {@link #isExceptionRetryable} indicates
-     * that the request cannot be retried.
-     */
-    protected <T> T runOperationWithRetry(Callable<T> operation) {
-        int backoff = 64;
-        Exception lastException = null;
-        for (int retries = 0; retries < 100; retries++) {
-            try {
-                return operation.call();
-            } catch (Exception e) {
-                lastException = e;
-                if (isExceptionRetryable.apply(e)) {
-                    LOG.debug("Attempt #{} failed to add security group: {}", retries + 1, e.getMessage());
-                    try {
-                        Thread.sleep(backoff);
-                    } catch (InterruptedException e1) {
-                        throw Exceptions.propagate(e1);
-                    }
-                    backoff = backoff << 1;
-                } else {
-                    break;
-                }
-            }
-        }
-
-        throw new RuntimeException("Unable to add security group rule; repeated errors from provider", lastException);
-    }
 
     /**
      * @return

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a27656c4/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/SecurityGroupEditor.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/SecurityGroupEditor.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/SecurityGroupEditor.java
new file mode 100644
index 0000000..31ac532
--- /dev/null
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/SecurityGroupEditor.java
@@ -0,0 +1,284 @@
+/*
+ * 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.location.jclouds.networking;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.apache.brooklyn.util.exceptions.Exceptions;
+import org.jclouds.aws.AWSResponseException;
+import org.jclouds.compute.domain.SecurityGroup;
+import org.jclouds.compute.extensions.SecurityGroupExtension;
+import org.jclouds.domain.Location;
+import org.jclouds.net.domain.IpPermission;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+import java.util.concurrent.Callable;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Utility for manipulating security groups.
+ */
+public class SecurityGroupEditor {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SecurityGroupEditor.class);
+    public static final java.lang.String JCLOUDS_PREFIX = "jclouds#";
+
+    private final Location location;
+    private final SecurityGroupExtension securityApi;
+    private final Predicate<Exception> isExceptionRetryable;
+
+    /**
+     * Constructor for editor that never retries requests if the attempted operation fails.
+     * @param location JClouds location where security groups will be managed.
+     * @param securityGroupExtension The JClouds security group extension from the compute service for this location.
+     */
+    public SecurityGroupEditor(Location location, SecurityGroupExtension securityGroupExtension) {
+        this.location = checkNotNull(location, "location");
+        this.securityApi = checkNotNull(securityGroupExtension, "securityGroupExtension");
+        this.isExceptionRetryable = Predicates.alwaysFalse();
+    }
+
+    /**
+     * Constructor for editor that may retry operations upon exceptions.
+     * @deprecated since 0.10.0 Ideally it should be possible to determine a suitable predicate internally to
+     * this class by instantiating a type appropriate to the underlying cloud. TODO investigate and implement.
+     *
+     * @param location JClouds location where security groups will be managed.
+     * @param securityGroupExtension The JClouds security group extension from the compute service for this location.
+     * @param isExceptionRetryable used to determine for an exception whether to retry the operation that failed.
+     */
+    @Deprecated
+    public SecurityGroupEditor(Location location, SecurityGroupExtension securityGroupExtension,
+            Predicate<Exception> isExceptionRetryable) {
+        this.location = checkNotNull(location, "location");
+        this.securityApi = checkNotNull(securityGroupExtension, "securityGroupExtension");
+        this.isExceptionRetryable = isExceptionRetryable;
+    }
+
+
+    /**
+     * Get the location in which security groups will be created or searched.
+     */
+    public Location getLocation() {
+        return location;
+    }
+
+    public Set<SecurityGroup> getSecurityGroupsForNode(final String nodeId) {
+        return securityApi.listSecurityGroupsForNode(nodeId);
+    }
+
+    /**
+     * Create the security group. As we use jclouds, groups are created with names prefixed
+     * with {@link #JCLOUDS_PREFIX}. This method is idempotent.
+     * @param name Name of the group to create
+     * @return The created group.
+     */
+    public SecurityGroup createSecurityGroup(final String name) {
+
+        LOG.debug("Creating security group {} in {}", name, location);
+        Callable<SecurityGroup> callable = new Callable<SecurityGroup>() {
+            @Override
+            public SecurityGroup call() throws Exception {
+                return securityApi.createSecurityGroup(name, location);
+            }
+        };
+        return runOperationWithRetry(callable);
+    }
+
+    /**
+     * Removes a security group and its permissions.
+     * @return true if the group was found and removed.
+     */
+    public boolean removeSecurityGroup(final SecurityGroup group) {
+
+        LOG.debug("Removing security group {} in {}", group.getName(), location);
+        Callable<Boolean> callable = new Callable<Boolean>() {
+            @Override
+            public Boolean call() throws Exception {
+                return securityApi.removeSecurityGroup(group.getId());
+            }
+        };
+        return runOperationWithRetry(callable);
+    }
+
+    public Set<SecurityGroup> listSecurityGroupsForNode(final String nodeId) {
+        return securityApi.listSecurityGroupsForNode(nodeId);
+    }
+
+    public Iterable<SecurityGroup> findSecurityGroupsMatching(Predicate predicate) {
+        final Set<SecurityGroup> locationGroups = securityApi.listSecurityGroupsInLocation(location);
+        return Iterables.filter(locationGroups, predicate);
+    }
+
+    /**
+     * @see #findSecurityGroupByName(String)
+     */
+    public static class AmbiguousGroupName extends IllegalArgumentException {
+        public AmbiguousGroupName(String s) {
+            super(s);
+        }
+    }
+
+    /**
+     * Find a security group with the given name. As we use jclouds, groups are created with names prefixed
+     * with {@link #JCLOUDS_PREFIX}. For convenience this method accepts names either with or without the prefix.
+     * @param name Name of the group to find.
+     * @return An optional of the group.
+     * @throws AmbiguousGroupName in the unexpected case that the cloud returns more than one matching group.
+     */
+    public Optional<SecurityGroup> findSecurityGroupByName(final String name) {
+        final String query = name.startsWith(JCLOUDS_PREFIX) ? name : JCLOUDS_PREFIX + name;
+        final Iterable<SecurityGroup> groupsMatching = findSecurityGroupsMatching(new Predicate<SecurityGroup>() {
+            @Override
+            public boolean apply(final SecurityGroup input) {
+                return input.getName().equals(query);
+            }
+        });
+        final ImmutableList<SecurityGroup> matches = ImmutableList.copyOf(groupsMatching);
+        if (matches.size() == 0) {
+            return Optional.absent();
+        } else if (matches.size() == 1) {
+            return Optional.of(matches.get(0));
+        } else {
+            throw new AmbiguousGroupName("Unexpected result of multiple groups matching " + name);
+        }
+    }
+
+    /**
+     * Add permissions to the security group, using {@link #addPermission(SecurityGroup, IpPermission)}.
+     * @param group The group to update
+     * @param permissions The new permissions
+     * @return The updated group with the added permissions.
+     */
+    public SecurityGroup addPermissions(final SecurityGroup group, final Iterable<IpPermission> permissions) {
+        SecurityGroup lastGroup = group;
+        for (IpPermission permission : permissions) {
+            lastGroup = addPermission(group, permission);
+        }
+        return lastGroup;
+    }
+
+    /**
+     * Add a permission to the security group. This operation is idempotent (will return the group unmodified if the
+     * permission already exists on it).
+     * @param group The group to update
+     * @param permissions The new permissions
+     * @return The updated group with the added permissions.
+     */
+    public SecurityGroup addPermission(final SecurityGroup group, final IpPermission permission) {
+        LOG.debug("Adding permission to security group {}: {}", group.getName(), permission);
+        Callable<SecurityGroup> callable = new Callable<SecurityGroup>() {
+            @Override
+            public SecurityGroup call() throws Exception {
+                try {
+                    return securityApi.addIpPermission(permission, group);
+                } catch (Exception e) {
+                    Exceptions.propagateIfFatal(e);
+
+                    if (isDuplicate(e)) {
+                        return group;
+                    }
+
+                    throw Exceptions.propagate(e);
+                }
+            }
+        };
+        return runOperationWithRetry(callable);
+    }
+
+    @Deprecated // TODO improve this - shouldn't have AWS specifics in here
+    private boolean isDuplicate(Exception e) {
+        // Sometimes AWSResponseException is wrapped in an IllegalStateException
+        AWSResponseException cause = Exceptions.getFirstThrowableOfType(e, AWSResponseException.class);
+        if (cause != null) {
+            if ("InvalidPermission.Duplicate".equals(cause.getError().getCode())) {
+                return true;
+            }
+        }
+
+        if (e.toString().contains("already exists")) {
+            return true;
+        }
+
+        return false;
+    }
+
+
+    public SecurityGroup removePermission(final SecurityGroup group, final IpPermission permission) {
+        LOG.debug("Removing permission from security group {}: {}", group.getName(), permission);
+        Callable<SecurityGroup> callable = new Callable<SecurityGroup>() {
+            @Override
+            public SecurityGroup call() throws Exception {
+                return securityApi.removeIpPermission(permission, group);
+            }
+        };
+        return runOperationWithRetry(callable);
+    }
+
+    public SecurityGroup removePermissions(SecurityGroup group, final Iterable<IpPermission> permissions) {
+        for (IpPermission permission : permissions) {
+            group = removePermission(group, permission);
+        }
+        return group;
+    }
+
+    /**
+     * Runs the given callable. Repeats until the operation succeeds or {@link #isExceptionRetryable} indicates
+     * that the request cannot be retried.
+     */
+    protected <T> T runOperationWithRetry(Callable<T> operation) {
+        int backoff = 64;
+        Exception lastException = null;
+        for (int retries = 0; retries < 12; retries++) { // 12 = keep trying for about 5 minutes
+            try {
+                return operation.call();
+            } catch (Exception e) {
+                lastException = e;
+                if (isExceptionRetryable.apply(e)) {
+                    LOG.debug("Attempt #{} failed to add security group: {}", retries + 1, e.getMessage());
+                    try {
+                        Thread.sleep(backoff);
+                    } catch (InterruptedException e1) {
+                        throw Exceptions.propagate(e1);
+                    }
+                    backoff = backoff << 1;
+                } else {
+                    break;
+                }
+            }
+        }
+
+        throw new RuntimeException("Unable to add security group rule; repeated errors from provider", lastException);
+    }
+
+    @Override
+    public String toString() {
+        return "JcloudsLocationSecurityGroupEditor{" +
+            "location=" + location +
+            ", securityApi=" + securityApi +
+            ", isExceptionRetryable=" + isExceptionRetryable +
+            '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a27656c4/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizer.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizer.java b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizer.java
index 2511d74..f6323d7 100644
--- a/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizer.java
+++ b/locations/jclouds/src/main/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizer.java
@@ -182,7 +182,7 @@ public class SharedLocationSecurityGroupCustomizer extends BasicJcloudsLocationC
                 builder.add(ipPermission);
             }
         }
-        return instance.addPermissionsToLocationAndReturnSecurityGroup(computeService, machine, builder.build());
+        return instance.addPermissionsToLocationAndReturnSecurityGroup(machine, builder.build());
     }
 
     private List<IpPermission> getIpPermissions(JcloudsLocationSecurityGroupCustomizer instance, RangeSet<Integer> portRanges, IpProtocol protocol) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a27656c4/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizerTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizerTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizerTest.java
index afc722a..de0f02d 100644
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizerTest.java
+++ b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/JcloudsLocationSecurityGroupCustomizerTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.brooklyn.location.jclouds.networking;
 
+import static org.apache.brooklyn.location.jclouds.networking.SecurityGroupEditor.JCLOUDS_PREFIX;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.eq;
@@ -36,6 +37,7 @@ import java.net.URI;
 import java.util.Collections;
 import java.util.Set;
 
+import org.apache.brooklyn.location.jclouds.JcloudsMachineLocation;
 import org.jclouds.aws.AWSResponseException;
 import org.jclouds.aws.domain.AWSError;
 import org.jclouds.compute.ComputeService;
@@ -47,6 +49,8 @@ import org.jclouds.domain.Location;
 import org.jclouds.net.domain.IpPermission;
 import org.jclouds.net.domain.IpProtocol;
 import org.mockito.Answers;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
@@ -55,7 +59,6 @@ import com.google.common.base.Predicate;
 import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.util.concurrent.UncheckedExecutionException;
 
 import org.apache.brooklyn.location.jclouds.JcloudsLocation;
 import org.apache.brooklyn.util.collections.MutableMap;
@@ -64,9 +67,11 @@ import org.apache.brooklyn.util.net.Cidr;
 public class JcloudsLocationSecurityGroupCustomizerTest {
 
     JcloudsLocationSecurityGroupCustomizer customizer;
-    ComputeService computeService;
-    Location location;
-    SecurityGroupExtension securityApi;
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS) ComputeService computeService;
+    @Mock(answer = Answers.RETURNS_SMART_NULLS) Location location;
+    @Mock(answer = Answers.RETURNS_SMART_NULLS) SecurityGroupExtension securityApi;
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS) JcloudsMachineLocation jcloudsMachineLocation;
+    private static final String NODE_ID = "node";
 
     /** Used to skip external checks in unit tests. */
     private static class TestCidrSupplier implements Supplier<Cidr> {
@@ -77,11 +82,14 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
 
     @BeforeMethod
     public void setUp() {
+        MockitoAnnotations.initMocks(this);
         customizer = new JcloudsLocationSecurityGroupCustomizer("testapp", new TestCidrSupplier());
-        location = mock(Location.class);
-        securityApi = mock(SecurityGroupExtension.class);
-        computeService = mock(ComputeService.class, Answers.RETURNS_DEEP_STUBS.get());
         when(computeService.getSecurityGroupExtension()).thenReturn(Optional.of(securityApi));
+        when(location.getParent()).thenReturn(location);
+        when(location.getId()).thenReturn("aws-ec2");
+        when(jcloudsMachineLocation.getNode().getId()).thenReturn(NODE_ID);
+        when(jcloudsMachineLocation.getNode().getLocation()).thenReturn(location);
+        when(jcloudsMachineLocation.getParent().getComputeService()).thenReturn(computeService);
     }
 
     @Test
@@ -100,6 +108,7 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
         when(template.getOptions()).thenReturn(templateOptions);
         SecurityGroup group = newGroup("id");
         when(securityApi.createSecurityGroup(anyString(), eq(location))).thenReturn(group);
+        when(securityApi.addIpPermission(any(IpPermission.class), eq(group))).thenReturn(group);
 
         // Two Brooklyn.JcloudsLocations added to same Jclouds.Location
         JcloudsLocation jcloudsLocationA = new JcloudsLocation(MutableMap.of("deferConstruction", true));
@@ -124,7 +133,11 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
         JcloudsLocation jcloudsLocation = new JcloudsLocation(MutableMap.of("deferConstruction", true));
         SecurityGroup shared = newGroup(customizer.getNameForSharedSecurityGroup());
         SecurityGroup irrelevant = newGroup("irrelevant");
+        when(securityApi.createSecurityGroup(shared.getName(), location)).thenReturn(shared);
+        when(securityApi.createSecurityGroup(irrelevant.getName(), location)).thenReturn(irrelevant);
         when(securityApi.listSecurityGroupsInLocation(location)).thenReturn(ImmutableSet.of(irrelevant, shared));
+        when(securityApi.addIpPermission(any(IpPermission.class), eq(shared))).thenReturn(shared);
+        when(securityApi.addIpPermission(any(IpPermission.class), eq(irrelevant))).thenReturn(irrelevant);
 
         customizer.customize(jcloudsLocation, computeService, template);
 
@@ -136,16 +149,15 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
     public void testAddPermissionsToNode() {
         IpPermission ssh = newPermission(22);
         IpPermission jmx = newPermission(31001);
-        String nodeId = "node";
         SecurityGroup sharedGroup = newGroup(customizer.getNameForSharedSecurityGroup());
         SecurityGroup group = newGroup("id");
-        when(securityApi.listSecurityGroupsForNode(nodeId)).thenReturn(ImmutableSet.of(sharedGroup, group));
+        when(securityApi.listSecurityGroupsForNode(NODE_ID)).thenReturn(ImmutableSet.of(sharedGroup, group));
         SecurityGroup updatedSecurityGroup = newGroup("id", ImmutableSet.of(ssh, jmx));
         when(securityApi.addIpPermission(ssh, group)).thenReturn(updatedSecurityGroup);
         when(securityApi.addIpPermission(jmx, group)).thenReturn(updatedSecurityGroup);
         when(computeService.getContext().unwrap().getId()).thenReturn("aws-ec2");
 
-        customizer.addPermissionsToLocation(ImmutableList.of(ssh, jmx), nodeId, computeService);
+        customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableList.of(ssh, jmx));
 
         verify(securityApi, never()).createSecurityGroup(anyString(), any(Location.class));
         verify(securityApi, times(1)).addIpPermission(ssh, group);
@@ -156,17 +168,16 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
     public void testRemovePermissionsFromNode() {
         IpPermission ssh = newPermission(22);
         IpPermission jmx = newPermission(31001);
-        String nodeId = "node";
         SecurityGroup sharedGroup = newGroup(customizer.getNameForSharedSecurityGroup());
         SecurityGroup group = newGroup("id");
-        when(securityApi.listSecurityGroupsForNode(nodeId)).thenReturn(ImmutableSet.of(sharedGroup, group));
+        when(securityApi.listSecurityGroupsForNode(NODE_ID)).thenReturn(ImmutableSet.of(sharedGroup, group));
         SecurityGroup updatedSecurityGroup = newGroup("id", ImmutableSet.of(ssh, jmx));
         when(securityApi.addIpPermission(ssh, group)).thenReturn(updatedSecurityGroup);
         when(securityApi.addIpPermission(jmx, group)).thenReturn(updatedSecurityGroup);
         when(computeService.getContext().unwrap().getId()).thenReturn("aws-ec2");
 
-        customizer.addPermissionsToLocation(ImmutableList.of(ssh, jmx), nodeId, computeService);
-        customizer.removePermissionsFromLocation(ImmutableList.of(jmx), nodeId, computeService);
+        customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableList.of(ssh, jmx));
+        customizer.removePermissionsFromLocation(jcloudsMachineLocation, ImmutableList.of(jmx));
 
         verify(securityApi, never()).removeIpPermission(ssh, group);
         verify(securityApi, times(1)).removeIpPermission(jmx, group);
@@ -176,17 +187,19 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
     public void testRemoveMultiplePermissionsFromNode() {
         IpPermission ssh = newPermission(22);
         IpPermission jmx = newPermission(31001);
-        String nodeId = "node";
         SecurityGroup sharedGroup = newGroup(customizer.getNameForSharedSecurityGroup());
         SecurityGroup group = newGroup("id");
-        when(securityApi.listSecurityGroupsForNode(nodeId)).thenReturn(ImmutableSet.of(sharedGroup, group));
+        when(securityApi.listSecurityGroupsForNode(NODE_ID)).thenReturn(ImmutableSet.of(sharedGroup, group));
         SecurityGroup updatedSecurityGroup = newGroup("id", ImmutableSet.of(ssh, jmx));
         when(securityApi.addIpPermission(ssh, group)).thenReturn(updatedSecurityGroup);
         when(securityApi.addIpPermission(jmx, group)).thenReturn(updatedSecurityGroup);
         when(computeService.getContext().unwrap().getId()).thenReturn("aws-ec2");
 
-        customizer.addPermissionsToLocation(ImmutableList.of(ssh, jmx), nodeId, computeService);
-        customizer.removePermissionsFromLocation(ImmutableList.of(ssh, jmx), nodeId, computeService);
+        customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableList.of(ssh, jmx));
+
+        when(securityApi.removeIpPermission(ssh, group)).thenReturn(updatedSecurityGroup);
+        when(securityApi.removeIpPermission(jmx, group)).thenReturn(updatedSecurityGroup);
+        customizer.removePermissionsFromLocation(jcloudsMachineLocation, ImmutableList.of(ssh, jmx));
 
         verify(securityApi, times(1)).removeIpPermission(ssh, group);
         verify(securityApi, times(1)).removeIpPermission(jmx, group);
@@ -197,13 +210,12 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
     public void testAddPermissionWhenNoExtension() {
         IpPermission ssh = newPermission(22);
         IpPermission jmx = newPermission(31001);
-        String nodeId = "node";
 
-        when(securityApi.listSecurityGroupsForNode(nodeId)).thenReturn(Collections.<SecurityGroup>emptySet());
+        when(securityApi.listSecurityGroupsForNode(NODE_ID)).thenReturn(Collections.<SecurityGroup>emptySet());
 
         RuntimeException exception = null;
         try {
-            customizer.addPermissionsToLocation(ImmutableList.of(ssh, jmx), nodeId, computeService);
+            customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableList.of(ssh, jmx));
         } catch(RuntimeException e){
             exception = e;
         }
@@ -214,8 +226,6 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
     @Test
     public void testAddPermissionsToNodeUsesUncachedSecurityGroup() {
         JcloudsLocation jcloudsLocation = new JcloudsLocation(MutableMap.of("deferConstruction", true));
-        IpPermission ssh = newPermission(22);
-        String nodeId = "nodeId";
         SecurityGroup sharedGroup = newGroup(customizer.getNameForSharedSecurityGroup());
         SecurityGroup uniqueGroup = newGroup("unique");
 
@@ -224,17 +234,21 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
         when(template.getLocation()).thenReturn(location);
         when(template.getOptions()).thenReturn(templateOptions);
         when(securityApi.createSecurityGroup(anyString(), eq(location))).thenReturn(sharedGroup);
+        when(securityApi.addIpPermission(any(IpPermission.class), eq(uniqueGroup))).thenReturn(uniqueGroup);
+        when(securityApi.addIpPermission(any(IpPermission.class), eq(sharedGroup))).thenReturn(sharedGroup);
+
         when(computeService.getContext().unwrap().getId()).thenReturn("aws-ec2");
 
         // Call customize to cache the shared group
         customizer.customize(jcloudsLocation, computeService, template);
         reset(securityApi);
-        when(securityApi.listSecurityGroupsForNode(nodeId)).thenReturn(ImmutableSet.of(uniqueGroup, sharedGroup));
+        when(securityApi.listSecurityGroupsForNode(NODE_ID)).thenReturn(ImmutableSet.of(uniqueGroup, sharedGroup));
+        IpPermission ssh = newPermission(22);
         SecurityGroup updatedSharedSecurityGroup = newGroup(sharedGroup.getId(), ImmutableSet.of(ssh));
         when(securityApi.addIpPermission(ssh, uniqueGroup)).thenReturn(updatedSharedSecurityGroup);
         SecurityGroup updatedUniqueSecurityGroup = newGroup("unique", ImmutableSet.of(ssh));
         when(securityApi.addIpPermission(ssh, sharedGroup)).thenReturn(updatedUniqueSecurityGroup);
-        customizer.addPermissionsToLocation(ImmutableSet.of(ssh), nodeId, computeService);
+        customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableSet.of(ssh));
 
         // Expect the per-machine group to have been altered, not the shared group
         verify(securityApi).addIpPermission(ssh, uniqueGroup);
@@ -244,11 +258,10 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
     @Test
     public void testSecurityGroupsLoadedWhenAddingPermissionsToUncachedNode() {
         IpPermission ssh = newPermission(22);
-        String nodeId = "nodeId";
         SecurityGroup sharedGroup = newGroup(customizer.getNameForSharedSecurityGroup());
         SecurityGroup uniqueGroup = newGroup("unique");
 
-        when(securityApi.listSecurityGroupsForNode(nodeId)).thenReturn(ImmutableSet.of(sharedGroup, uniqueGroup));
+        when(securityApi.listSecurityGroupsForNode(NODE_ID)).thenReturn(ImmutableSet.of(sharedGroup, uniqueGroup));
         when(computeService.getContext().unwrap().getId()).thenReturn("aws-ec2");
         SecurityGroup updatedSecurityGroup = newGroup(uniqueGroup.getId(), ImmutableSet.of(ssh));
         when(securityApi.addIpPermission(ssh, sharedGroup)).thenReturn(updatedSecurityGroup);
@@ -256,10 +269,10 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
         when(securityApi.addIpPermission(ssh, updatedUniqueSecurityGroup)).thenReturn(updatedUniqueSecurityGroup);
 
         // Expect first call to list security groups on nodeId, second to use cached version
-        customizer.addPermissionsToLocation(ImmutableSet.of(ssh), nodeId, computeService);
-        customizer.addPermissionsToLocation(ImmutableSet.of(ssh), nodeId, computeService);
+        customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableSet.of(ssh));
+        customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableSet.of(ssh));
 
-        verify(securityApi, times(1)).listSecurityGroupsForNode(nodeId);
+        verify(securityApi, times(1)).listSecurityGroupsForNode(NODE_ID);
         verify(securityApi, times(2)).addIpPermission(ssh, uniqueGroup);
         verify(securityApi, never()).addIpPermission(any(IpPermission.class), eq(sharedGroup));
     }
@@ -267,16 +280,15 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
     @Test
     public void testAddRuleNotRetriedByDefault() {
         IpPermission ssh = newPermission(22);
-        String nodeId = "node";
         SecurityGroup sharedGroup = newGroup(customizer.getNameForSharedSecurityGroup());
         SecurityGroup uniqueGroup = newGroup("unique");
-        when(securityApi.listSecurityGroupsForNode(nodeId)).thenReturn(ImmutableSet.of(sharedGroup, uniqueGroup));
+        when(securityApi.listSecurityGroupsForNode(NODE_ID)).thenReturn(ImmutableSet.of(sharedGroup, uniqueGroup));
         when(securityApi.addIpPermission(eq(ssh), eq(uniqueGroup)))
                 .thenThrow(new RuntimeException("exception creating " + ssh));
         when(computeService.getContext().unwrap().getId()).thenReturn("aws-ec2");
 
         try {
-            customizer.addPermissionsToLocation(ImmutableList.of(ssh), nodeId, computeService);
+            customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableList.of(ssh));
         } catch (Exception e) {
             assertTrue(e.getMessage().contains("repeated errors from provider"), "message=" + e.getMessage());
         }
@@ -305,16 +317,15 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
         when(computeService.getContext().unwrap().getId()).thenReturn("aws-ec2");
 
         IpPermission ssh = newPermission(22);
-        String nodeId = "node";
         SecurityGroup sharedGroup = newGroup(customizer.getNameForSharedSecurityGroup());
         SecurityGroup uniqueGroup = newGroup("unique");
-        when(securityApi.listSecurityGroupsForNode(nodeId)).thenReturn(ImmutableSet.of(sharedGroup, uniqueGroup));
+        when(securityApi.listSecurityGroupsForNode(NODE_ID)).thenReturn(ImmutableSet.of(sharedGroup, uniqueGroup));
         when(securityApi.addIpPermission(eq(ssh), eq(uniqueGroup)))
                 .thenThrow(new RuntimeException(new Exception(message)))
                 .thenThrow(new RuntimeException(new Exception(message)))
                 .thenReturn(sharedGroup);
 
-        customizer.addPermissionsToLocation(ImmutableList.of(ssh), nodeId, computeService);
+        customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableList.of(ssh));
 
         verify(securityApi, never()).createSecurityGroup(anyString(), any(Location.class));
         verify(securityApi, times(3)).addIpPermission(ssh, uniqueGroup);
@@ -323,11 +334,10 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
     @Test
     public void testAddRuleRetriedOnAwsFailure() {
         IpPermission ssh = newPermission(22);
-        String nodeId = "nodeId";
         SecurityGroup sharedGroup = newGroup(customizer.getNameForSharedSecurityGroup());
         SecurityGroup uniqueGroup = newGroup("unique");
         customizer.setRetryExceptionPredicate(JcloudsLocationSecurityGroupCustomizer.newAwsExceptionRetryPredicate());
-        when(securityApi.listSecurityGroupsForNode(nodeId)).thenReturn(ImmutableSet.of(sharedGroup, uniqueGroup));
+        when(securityApi.listSecurityGroupsForNode(NODE_ID)).thenReturn(ImmutableSet.of(sharedGroup, uniqueGroup));
         when(securityApi.addIpPermission(any(IpPermission.class), eq(uniqueGroup)))
                 .thenThrow(newAwsResponseExceptionWithCode("InvalidGroup.InUse"))
                 .thenThrow(newAwsResponseExceptionWithCode("DependencyViolation"))
@@ -337,7 +347,7 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
         when(computeService.getContext().unwrap().getId()).thenReturn("aws-ec2");
 
         try {
-            customizer.addPermissionsToLocation(ImmutableList.of(ssh), nodeId, computeService);
+            customizer.addPermissionsToLocation(jcloudsMachineLocation, ImmutableList.of(ssh));
         } catch (Exception e) {
             String expected = "repeated errors from provider";
             assertTrue(e.getMessage().contains(expected), "expected exception message to contain " + expected + ", was: " + e.getMessage());
@@ -351,19 +361,23 @@ public class JcloudsLocationSecurityGroupCustomizerTest {
         return newGroup(id, ImmutableSet.<IpPermission>of());
     }
 
-    private SecurityGroup newGroup(String id, Set<IpPermission> ipPermissions) {
+    private SecurityGroup newGroup(String name, Set<IpPermission> ipPermissions) {
+        String id = name;
+        if (!name.startsWith(JCLOUDS_PREFIX)) {
+            id = JCLOUDS_PREFIX + name;
+        }
         URI uri = null;
         String ownerId = null;
         return new SecurityGroup(
-                "providerId",
-                id,
-                id,
-                location,
-                uri,
-                Collections.<String, String>emptyMap(),
-                ImmutableSet.<String>of(),
-                ipPermissions,
-                ownerId);
+            "providerId",
+            id,
+            id,
+            location,
+            uri,
+            Collections.<String, String>emptyMap(),
+            ImmutableSet.<String>of(),
+            ipPermissions,
+            ownerId);
     }
 
     private IpPermission newPermission(int port) {

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a27656c4/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/SecurityGroupLiveTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/SecurityGroupLiveTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/SecurityGroupLiveTest.java
deleted file mode 100644
index 0eed616..0000000
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/SecurityGroupLiveTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.location.jclouds.networking;
-
-import org.testng.annotations.Test;
-
-@Test(groups = {"Live", "WIP"})
-public class SecurityGroupLiveTest {
-
-    public void testCreateEc2WithSecurityGroup() {
-        SecurityGroupDefinition sgDef = new SecurityGroupDefinition()
-            .allowingInternalPorts(8097, 8098).allowingInternalPortRange(6000, 7999)
-            .allowingPublicPort(8099);
-        // TODO create machine and test
-    }
-}

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a27656c4/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizerTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizerTest.java b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizerTest.java
index 587f0f1..dfd35ce 100644
--- a/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizerTest.java
+++ b/locations/jclouds/src/test/java/org/apache/brooklyn/location/jclouds/networking/SharedLocationSecurityGroupCustomizerTest.java
@@ -141,7 +141,7 @@ public class SharedLocationSecurityGroupCustomizerTest {
 
     private void assertPermissionsAdded(int expectedFrom, int expectedTo, IpProtocol expectedProtocol) {
         ArgumentCaptor<List> listArgumentCaptor = ArgumentCaptor.forClass(List.class);
-        verify(sgCustomizer).addPermissionsToLocationAndReturnSecurityGroup(any(ComputeService.class), any(JcloudsMachineLocation.class), listArgumentCaptor.capture());
+        verify(sgCustomizer).addPermissionsToLocationAndReturnSecurityGroup(any(JcloudsMachineLocation.class), listArgumentCaptor.capture());
         IpPermission ipPermission = (IpPermission) listArgumentCaptor.getValue().get(0);
         assertEquals(ipPermission.getFromPort(), expectedFrom);
         assertEquals(ipPermission.getToPort(), expectedTo);

http://git-wip-us.apache.org/repos/asf/brooklyn-server/blob/a27656c4/software/base/src/test/java/org/apache/brooklyn/entity/software/base/test/location/SecurityGroupLiveTest.java
----------------------------------------------------------------------
diff --git a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/test/location/SecurityGroupLiveTest.java b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/test/location/SecurityGroupLiveTest.java
new file mode 100644
index 0000000..12a5a30
--- /dev/null
+++ b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/test/location/SecurityGroupLiveTest.java
@@ -0,0 +1,229 @@
+/*
+ * 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.entity.software.base.test.location;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.entity.EntitySpec;
+import org.apache.brooklyn.api.location.Location;
+import org.apache.brooklyn.core.entity.EntityAsserts;
+import org.apache.brooklyn.core.entity.trait.Startable;
+import org.apache.brooklyn.core.internal.BrooklynProperties;
+import org.apache.brooklyn.core.location.Locations;
+import org.apache.brooklyn.core.test.BrooklynAppLiveTestSupport;
+import org.apache.brooklyn.core.test.entity.LocalManagementContextForTests;
+import org.apache.brooklyn.entity.software.base.EmptySoftwareProcess;
+import org.apache.brooklyn.location.jclouds.JcloudsMachineLocation;
+import org.apache.brooklyn.location.jclouds.networking.SecurityGroupEditor;
+import org.apache.brooklyn.location.jclouds.networking.SecurityGroupDefinition;
+import org.apache.brooklyn.location.ssh.SshMachineLocation;
+import org.apache.brooklyn.util.collections.MutableMap;
+import org.apache.brooklyn.util.text.Identifiers;
+import org.jclouds.compute.ComputeService;
+import org.jclouds.compute.domain.SecurityGroup;
+import org.jclouds.compute.extensions.SecurityGroupExtension;
+import org.jclouds.net.domain.IpPermission;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.brooklyn.core.entity.DependentConfigurationTest.TIMEOUT_MS;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+
+@Test(groups = {"Live"})
+public class SecurityGroupLiveTest extends BrooklynAppLiveTestSupport  {
+
+    private static final Logger LOG = LoggerFactory.getLogger(SecurityGroupLiveTest.class);
+
+    public static final String PROVIDER = "aws-ec2";
+    public static final String REGION_NAME = "us-east-1";
+    public static final String LOCATION_SPEC = PROVIDER + (REGION_NAME == null ? "" : ":" + REGION_NAME);
+    public static final String UBUNTU_12 = "us-east-1/ami-d0f89fb9";
+
+    private BrooklynProperties brooklynProperties;
+
+    private Location loc;
+    private List<Location> locs;
+    private Entity testEntity;
+    private JcloudsMachineLocation jcloudsMachineLocation;
+    private ComputeService computeService;
+
+    @BeforeMethod(alwaysRun=true)
+    @Override
+    public void setUp() throws Exception {
+        // Don't let any defaults from brooklyn.properties (except credentials) interfere with test
+        brooklynProperties = BrooklynProperties.Factory.newDefault();
+        brooklynProperties.remove("brooklyn.jclouds."+PROVIDER+".image-description-regex");
+        brooklynProperties.remove("brooklyn.jclouds."+PROVIDER+".image-name-regex");
+        brooklynProperties.remove("brooklyn.jclouds."+PROVIDER+".image-id");
+        brooklynProperties.remove("brooklyn.jclouds."+PROVIDER+".inboundPorts");
+        brooklynProperties.remove("brooklyn.jclouds."+PROVIDER+".hardware-id");
+
+        // Also removes scriptHeader (e.g. if doing `. ~/.bashrc` and `. ~/.profile`, then that can cause "stdin: is not a tty")
+        brooklynProperties.remove("brooklyn.ssh.config.scriptHeader");
+
+        mgmt = new LocalManagementContextForTests(brooklynProperties);
+
+        super.setUp();
+
+        Map<String,?> allFlags = MutableMap.<String,Object>builder()
+            .put("tags", ImmutableList.of(getClass().getName()))
+            .putAll(ImmutableMap.of("imageId", UBUNTU_12, "loginUser", "ubuntu", "hardwareId", "m1.small"))
+            .build();
+        loc = mgmt.getLocationRegistry().getLocationManaged(LOCATION_SPEC, allFlags);
+        testEntity = app.createAndManageChild(EntitySpec.create(EmptySoftwareProcess.class));
+        app.start(ImmutableList.of(loc));
+        EntityAsserts.assertAttributeEqualsEventually(MutableMap.of("timeout", TIMEOUT_MS),
+            testEntity, Startable.SERVICE_UP, true);
+        SshMachineLocation sshLoc = Locations.findUniqueSshMachineLocation(testEntity.getLocations()).get();
+        jcloudsMachineLocation = (JcloudsMachineLocation)sshLoc;
+        computeService = jcloudsMachineLocation.getParent().getComputeService();
+
+    }
+
+
+    @AfterMethod(alwaysRun=true)
+    @Override
+    public void tearDown() throws Exception {
+        try {
+            if (app != null) {
+                app.stop();
+            }
+        } catch (Exception e) {
+            LOG.error("error stopping app; continuing with shutdown...", e);
+        } finally {
+            super.tearDown();
+        }
+    }
+
+    @Test
+    public void testCreateGroupAddPermissionsAndDelete() {
+        SecurityGroupDefinition sgDef = new SecurityGroupDefinition()
+            .allowingInternalPorts(8097, 8098)
+            .allowingInternalPortRange(6000, 7999)
+            .allowingPublicPort(8099);
+        final String securityGroupName = Identifiers.makeRandomLowercaseId(15);
+        final SecurityGroupEditor editor = makeEditor();
+        final SecurityGroup testGroup = createTestGroup(securityGroupName, editor);
+        assertEquals(testGroup.getName(), "jclouds#" + securityGroupName);
+
+        final SecurityGroup updated = editor.addPermissions(testGroup, sgDef.getPermissions());
+
+        final Optional<SecurityGroup> fromCloud = editor.findSecurityGroupByName(securityGroupName);
+        assertTrue(fromCloud.isPresent());
+        final SecurityGroup cloudGroup = fromCloud.get();
+
+        assertPermissionsEqual(updated.getIpPermissions(), cloudGroup.getIpPermissions());
+
+        editor.removeSecurityGroup(updated);
+
+        final Optional<SecurityGroup> afterRemove = editor.findSecurityGroupByName(securityGroupName);
+        assertFalse(afterRemove.isPresent());
+
+    }
+
+    @Test
+    public void testGroupAddIsIdempotent() {
+        SecurityGroupDefinition sgDef = new SecurityGroupDefinition()
+            .allowingInternalPorts(8097, 8098)
+            .allowingInternalPortRange(6000, 7999)
+            .allowingPublicPort(8099);
+        final String securityGroupName = Identifiers.makeRandomLowercaseId(15);
+        final SecurityGroupEditor editor = makeEditor();
+
+        SecurityGroup group1 = createTestGroup(securityGroupName, editor);
+        assertEquals(group1.getName(), "jclouds#" + securityGroupName);
+        group1 = editor.addPermissions(group1, sgDef.getPermissions());
+
+        final SecurityGroup group2 = createTestGroup(securityGroupName, editor);
+        assertEquals(group2.getName(), group1.getName());
+        assertPermissionsEqual(group2.getIpPermissions(), group1.getIpPermissions());
+
+        editor.removeSecurityGroup(group2);
+
+        final Optional<SecurityGroup> afterRemove = editor.findSecurityGroupByName(securityGroupName);
+        assertFalse(afterRemove.isPresent());
+    }
+
+
+
+    @Test
+    public void testPermissionsAddIsIdempotent() {
+        SecurityGroupDefinition sgDef = new SecurityGroupDefinition()
+            .allowingInternalPorts(8097, 8098)
+            .allowingInternalPortRange(6000, 7999)
+            .allowingPublicPort(8099);
+        final String securityGroupName = Identifiers.makeRandomLowercaseId(15);
+        final SecurityGroupEditor editor = makeEditor();
+
+        SecurityGroup group1 = createTestGroup(securityGroupName, editor);
+        assertEquals(group1.getName(), "jclouds#" + securityGroupName);
+
+        final SecurityGroup before = editor.addPermissions(group1, sgDef.getPermissions());
+        assertPermissionsEqual(ImmutableSet.copyOf(sgDef.getPermissions()), before.getIpPermissions());
+
+        try {
+            final SecurityGroup after = editor.addPermissions(before, sgDef.getPermissions());
+            assertPermissionsEqual(before.getIpPermissions(), after.getIpPermissions());
+        } catch (Exception e) {
+            // catch here so as to give us the chance to delete the group rather than leak it (this is a live test)
+            fail("Exception repeating group permissions", e);
+        }
+
+        editor.removeSecurityGroup(group1);
+
+        final Optional<SecurityGroup> afterRemove = editor.findSecurityGroupByName(securityGroupName);
+        assertFalse(afterRemove.isPresent());
+    }
+
+    private void assertPermissionsEqual(Set<IpPermission> actuals, Set<IpPermission> expecteds) {
+        assertEquals(actuals.size(), expecteds.size());
+        assertTrue(actuals.containsAll(expecteds));
+    }
+
+    private SecurityGroupEditor makeEditor() {
+        final org.jclouds.domain.Location nodeLocation = jcloudsMachineLocation.getNode().getLocation();
+        ComputeService computeService = jcloudsMachineLocation.getParent().getComputeService();
+        final Optional<SecurityGroupExtension> securityGroupExtension = computeService.getSecurityGroupExtension();
+        if (securityGroupExtension.isPresent()) {
+            return new SecurityGroupEditor(nodeLocation, securityGroupExtension.get());
+        } else {
+            throw new IllegalArgumentException("Expected SecurityGroupExtension not found in " + computeService);
+        }
+    }
+
+    private SecurityGroup createTestGroup(String securityGroupName, SecurityGroupEditor editor) {
+
+        LOG.info("Creating security group {} in {}", securityGroupName, jcloudsMachineLocation);
+        return editor.createSecurityGroup(securityGroupName);
+    }
+
+}


[2/2] brooklyn-server git commit: Closes #425

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

Factor security group code out into separate utility class

Factor out the functionality for security groups.

Brings the create/delete and add/remove permissions functionality
for security groups from JcloudsLocationSecurityGroupCustomizer
out into its own class. The aim is to make it more convenient
to write other customizers etc. that might need security groups.

This is intended as nearly as possible as a behaviour-preserving
change. I have permitted myself one change to the behaviour where
this seemed sensible; the loop in `runOperationWithRetry`
was effectively an infinite loop because of its 100 iteration
exponential back-off, so I have changed this to a back-off that
tries for about five minutes.


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

Branch: refs/heads/master
Commit: 95cf0f9a60934bf7c0c25069f52480699aa2d4aa
Parents: 836cc2b a27656c
Author: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Authored: Wed Nov 23 14:17:06 2016 +0200
Committer: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Committed: Wed Nov 23 14:17:06 2016 +0200

----------------------------------------------------------------------
 .../JcloudsLocationSecurityGroupCustomizer.java | 260 +++++++----------
 .../jclouds/networking/SecurityGroupEditor.java | 284 +++++++++++++++++++
 .../SharedLocationSecurityGroupCustomizer.java  |   2 +-
 ...oudsLocationSecurityGroupCustomizerTest.java | 112 ++++----
 .../networking/SecurityGroupLiveTest.java       |  32 ---
 ...aredLocationSecurityGroupCustomizerTest.java |   2 +-
 .../test/location/SecurityGroupLiveTest.java    | 229 +++++++++++++++
 7 files changed, 682 insertions(+), 239 deletions(-)
----------------------------------------------------------------------