You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jclouds.apache.org by na...@apache.org on 2017/01/26 21:40:02 UTC

[4/4] jclouds-labs git commit: JCLOUDS-1231: Implement the SecurityGroupExtension in ARM

JCLOUDS-1231: Implement the SecurityGroupExtension in ARM


Project: http://git-wip-us.apache.org/repos/asf/jclouds-labs/repo
Commit: http://git-wip-us.apache.org/repos/asf/jclouds-labs/commit/867ddef6
Tree: http://git-wip-us.apache.org/repos/asf/jclouds-labs/tree/867ddef6
Diff: http://git-wip-us.apache.org/repos/asf/jclouds-labs/diff/867ddef6

Branch: refs/heads/master
Commit: 867ddef6e9355aac624bf0981e46de45bb42843c
Parents: a91ff3b
Author: Daniel Estevez <co...@danielestevez.com>
Authored: Tue Jan 17 19:13:14 2017 -0500
Committer: Ignasi Barrera <na...@apache.org>
Committed: Thu Jan 26 21:58:23 2017 +0100

----------------------------------------------------------------------
 .../arm/compute/AzureComputeService.java        |  82 +++--
 .../arm/compute/AzureComputeServiceAdapter.java |  20 +-
 .../AzureComputeServiceContextModule.java       | 145 ++++++---
 .../domain/RegionAndIdAndIngressRules.java      |  66 ++++
 .../AzureComputeSecurityGroupExtension.java     | 315 +++++++++++++++++++
 .../NetworkSecurityGroupToSecurityGroup.java    |  72 +++++
 .../NetworkSecurityRuleToIpPermission.java      |  76 +++++
 .../functions/VirtualMachineToNodeMetadata.java |   4 +-
 .../loaders/CreateSecurityGroupIfNeeded.java    |  97 ++++++
 .../CreateResourceGroupThenCreateNodes.java     | 112 +++++--
 .../arm/domain/NetworkSecurityGroup.java        |  17 +-
 .../azurecompute/arm/domain/RegionAndId.java    |   2 +-
 .../arm/functions/CleanupResources.java         |  66 +++-
 .../AzureComputeImageExtensionLiveTest.java     |   1 +
 ...reComputeSecurityGroupExtensionLiveTest.java | 154 +++++++++
 .../NetworkSecurityGroupApiMockTest.java        |   2 +-
 .../internal/BaseAzureComputeApiLiveTest.java   |   2 +-
 .../src/test/resources/logback-test.xml         |   2 +-
 18 files changed, 1100 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeService.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeService.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeService.java
index c215e37..a561375 100644
--- a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeService.java
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeService.java
@@ -16,11 +16,12 @@
  */
 package org.jclouds.azurecompute.arm.compute;
 
-import static com.google.common.base.Preconditions.checkNotNull;
 import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_RUNNING;
 import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_SUSPENDED;
 import static org.jclouds.compute.config.ComputeServiceProperties.TIMEOUT_NODE_TERMINATED;
+
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -30,6 +31,8 @@ import javax.inject.Provider;
 import javax.inject.Singleton;
 
 import org.jclouds.Constants;
+import org.jclouds.azurecompute.arm.compute.functions.LocationToResourceGroupName;
+import org.jclouds.azurecompute.arm.functions.CleanupResources;
 import org.jclouds.collect.Memoized;
 import org.jclouds.compute.ComputeServiceContext;
 import org.jclouds.compute.callables.RunScriptOnNode;
@@ -54,51 +57,74 @@ import org.jclouds.compute.strategy.ResumeNodeStrategy;
 import org.jclouds.compute.strategy.SuspendNodeStrategy;
 import org.jclouds.domain.Credentials;
 import org.jclouds.domain.Location;
-import org.jclouds.azurecompute.arm.functions.CleanupResources;
 import org.jclouds.scriptbuilder.functions.InitAdminAccess;
 
 import com.google.common.base.Optional;
 import com.google.common.base.Predicate;
 import com.google.common.base.Supplier;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.util.concurrent.ListeningExecutorService;
 
 @Singleton
 public class AzureComputeService extends BaseComputeService {
-   protected final CleanupResources cleanupResources;
+   private final CleanupResources cleanupResources;
+   private final LocationToResourceGroupName locationToResourceGroupName;
 
    @Inject
    protected AzureComputeService(ComputeServiceContext context, Map<String, Credentials> credentialStore,
-                                @Memoized Supplier<Set<? extends Image>> images, @Memoized Supplier<Set<? extends Hardware>> sizes,
-                                @Memoized Supplier<Set<? extends Location>> locations, ListNodesStrategy listNodesStrategy,
-                                GetImageStrategy getImageStrategy, GetNodeMetadataStrategy getNodeMetadataStrategy,
-                                CreateNodesInGroupThenAddToSet runNodesAndAddToSetStrategy, RebootNodeStrategy rebootNodeStrategy,
-                                DestroyNodeStrategy destroyNodeStrategy, ResumeNodeStrategy startNodeStrategy,
-                                SuspendNodeStrategy stopNodeStrategy, Provider<TemplateBuilder> templateBuilderProvider,
-                                @Named("DEFAULT") Provider<TemplateOptions> templateOptionsProvider,
-                                @Named(TIMEOUT_NODE_RUNNING) Predicate<AtomicReference<NodeMetadata>> nodeRunning,
-                                @Named(TIMEOUT_NODE_TERMINATED) Predicate<AtomicReference<NodeMetadata>> nodeTerminated,
-                                @Named(TIMEOUT_NODE_SUSPENDED) Predicate<AtomicReference<NodeMetadata>> nodeSuspended,
-                                InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory,
-                                RunScriptOnNode.Factory runScriptOnNodeFactory, InitAdminAccess initAdminAccess,
-                                PersistNodeCredentials persistNodeCredentials, Timeouts timeouts,
-                                @Named(Constants.PROPERTY_USER_THREADS) ListeningExecutorService userExecutor,
-                                 CleanupResources cleanupResources,
-                                Optional<ImageExtension> imageExtension,
-                                Optional<SecurityGroupExtension> securityGroupExtension) {
+         @Memoized Supplier<Set<? extends Image>> images, @Memoized Supplier<Set<? extends Hardware>> sizes,
+         @Memoized Supplier<Set<? extends Location>> locations, ListNodesStrategy listNodesStrategy,
+         GetImageStrategy getImageStrategy, GetNodeMetadataStrategy getNodeMetadataStrategy,
+         CreateNodesInGroupThenAddToSet runNodesAndAddToSetStrategy, RebootNodeStrategy rebootNodeStrategy,
+         DestroyNodeStrategy destroyNodeStrategy, ResumeNodeStrategy startNodeStrategy,
+         SuspendNodeStrategy stopNodeStrategy, Provider<TemplateBuilder> templateBuilderProvider,
+         @Named("DEFAULT") Provider<TemplateOptions> templateOptionsProvider,
+         @Named(TIMEOUT_NODE_RUNNING) Predicate<AtomicReference<NodeMetadata>> nodeRunning,
+         @Named(TIMEOUT_NODE_TERMINATED) Predicate<AtomicReference<NodeMetadata>> nodeTerminated,
+         @Named(TIMEOUT_NODE_SUSPENDED) Predicate<AtomicReference<NodeMetadata>> nodeSuspended,
+         InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory,
+         RunScriptOnNode.Factory runScriptOnNodeFactory, InitAdminAccess initAdminAccess,
+         PersistNodeCredentials persistNodeCredentials, Timeouts timeouts,
+         @Named(Constants.PROPERTY_USER_THREADS) ListeningExecutorService userExecutor,
+         CleanupResources cleanupResources, Optional<ImageExtension> imageExtension,
+         Optional<SecurityGroupExtension> securityGroupExtension,
+         LocationToResourceGroupName locationToResourceGroupName) {
       super(context, credentialStore, images, sizes, locations, listNodesStrategy, getImageStrategy,
-              getNodeMetadataStrategy, runNodesAndAddToSetStrategy, rebootNodeStrategy, destroyNodeStrategy,
-              startNodeStrategy, stopNodeStrategy, templateBuilderProvider, templateOptionsProvider, nodeRunning,
-              nodeTerminated, nodeSuspended, initScriptRunnerFactory, initAdminAccess, runScriptOnNodeFactory,
-              persistNodeCredentials, timeouts, userExecutor, imageExtension, securityGroupExtension);
-      this.cleanupResources = checkNotNull(cleanupResources, "cleanupResources");
-
+            getNodeMetadataStrategy, runNodesAndAddToSetStrategy, rebootNodeStrategy, destroyNodeStrategy,
+            startNodeStrategy, stopNodeStrategy, templateBuilderProvider, templateOptionsProvider, nodeRunning,
+            nodeTerminated, nodeSuspended, initScriptRunnerFactory, initAdminAccess, runScriptOnNodeFactory,
+            persistNodeCredentials, timeouts, userExecutor, imageExtension, securityGroupExtension);
+      this.cleanupResources = cleanupResources;
+      this.locationToResourceGroupName = locationToResourceGroupName;
    }
 
    @Override
    protected void cleanUpIncidentalResourcesOfDeadNodes(Set<? extends NodeMetadata> deadNodes) {
+      ImmutableMultimap.Builder<String, String> regionGroups = ImmutableMultimap.builder();
+      ImmutableSet.Builder<String> resourceGroups = ImmutableSet.builder();
+
       for (NodeMetadata deadNode : deadNodes) {
-         cleanupResources.apply(deadNode.getId());
+         String resourceGroup = locationToResourceGroupName.apply(deadNode.getLocation().getId());
+
+         resourceGroups.add(resourceGroup);
+         if (deadNode.getGroup() != null) {
+            regionGroups.put(resourceGroup, deadNode.getGroup());
+         }
+
+         try {
+            cleanupResources.cleanupNode(deadNode.getId());
+         } catch (Exception ex) {
+            logger.warn(ex, "Error cleaning up resources for node %s", deadNode);
+         }
+      }
+
+      for (Entry<String, String> regionGroup : regionGroups.build().entries()) {
+         cleanupResources.cleanupSecurityGroupIfOrphaned(regionGroup.getKey(), regionGroup.getValue());
       }
-   }
 
+      for (String resourceGroup : resourceGroups.build()) {
+         cleanupResources.deleteResourceGroupIfEmpty(resourceGroup);
+      }
+   }
 }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeServiceAdapter.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeServiceAdapter.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeServiceAdapter.java
index 3b207b7..269f6b0 100644
--- a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeServiceAdapter.java
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/AzureComputeServiceAdapter.java
@@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkState;
 import static com.google.common.collect.Iterables.contains;
 import static com.google.common.collect.Iterables.filter;
 import static com.google.common.collect.Iterables.find;
+import static com.google.common.collect.Iterables.getOnlyElement;
 import static org.jclouds.azurecompute.arm.compute.extensions.AzureComputeImageExtension.CONTAINER_NAME;
 import static org.jclouds.azurecompute.arm.compute.extensions.AzureComputeImageExtension.CUSTOM_IMAGE_OFFER;
 import static org.jclouds.azurecompute.arm.compute.functions.VMImageToImage.decodeFieldsFromUniqueId;
@@ -81,6 +82,7 @@ import org.jclouds.compute.ComputeServiceAdapter;
 import org.jclouds.compute.domain.Image;
 import org.jclouds.compute.domain.OsFamily;
 import org.jclouds.compute.domain.Template;
+import org.jclouds.compute.options.TemplateOptions;
 import org.jclouds.compute.reference.ComputeServiceConstants;
 import org.jclouds.location.Region;
 import org.jclouds.logging.Logger;
@@ -137,11 +139,10 @@ public class AzureComputeServiceAdapter implements ComputeServiceAdapter<Virtual
 
       // TODO ARM specific options
       // TODO network ids => create one nic in each network
-      // TODO inbound ports
 
       String locationName = template.getLocation().getId();
       String subnetId = templateOptions.getSubnetId();
-      NetworkInterfaceCard nic = createNetworkInterfaceCard(subnetId, name, locationName, azureGroup);
+      NetworkInterfaceCard nic = createNetworkInterfaceCard(subnetId, name, locationName, azureGroup, template.getOptions());
       StorageProfile storageProfile = createStorageProfile(name, template.getImage(), templateOptions.getBlob());
       HardwareProfile hardwareProfile = HardwareProfile.builder().vmSize(template.getHardware().getId()).build();
       OSProfile osProfile = createOsProfile(name, template);
@@ -341,7 +342,7 @@ public class AzureComputeServiceAdapter implements ComputeServiceAdapter<Virtual
 
    @Override
    public void destroyNode(final String id) {
-      checkState(cleanupResources.apply(id), "server(%s) and its resources still there after deleting!?", id);
+      checkState(cleanupResources.cleanupNode(id), "server(%s) and its resources still there after deleting!?", id);
    }
 
    @Override
@@ -405,7 +406,7 @@ public class AzureComputeServiceAdapter implements ComputeServiceAdapter<Virtual
    }
 
    private NetworkInterfaceCard createNetworkInterfaceCard(String subnetId, String name, String locationName,
-         String azureGroup) {
+         String azureGroup, TemplateOptions options) {
       final PublicIPAddressApi ipApi = api.getPublicIPAddressApi(azureGroup);
 
       PublicIPAddressProperties properties = PublicIPAddressProperties.builder().publicIPAllocationMethod("Static")
@@ -418,7 +419,7 @@ public class AzureComputeServiceAdapter implements ComputeServiceAdapter<Virtual
       checkState(publicIpAvailable.create(azureGroup).apply(publicIpAddressName),
             "Public IP was not provisioned in the configured timeout");
 
-      final NetworkInterfaceCardProperties networkInterfaceCardProperties = NetworkInterfaceCardProperties
+      final NetworkInterfaceCardProperties.Builder networkInterfaceCardProperties = NetworkInterfaceCardProperties
             .builder()
             .ipConfigurations(
                   ImmutableList.of(IpConfiguration
@@ -427,11 +428,16 @@ public class AzureComputeServiceAdapter implements ComputeServiceAdapter<Virtual
                         .properties(
                               IpConfigurationProperties.builder().privateIPAllocationMethod("Dynamic")
                                     .publicIPAddress(IdReference.create(ip.id())).subnet(IdReference.create(subnetId))
-                                    .build()).build())).build();
+                                    .build()).build()));
+
+      String securityGroup = getOnlyElement(options.getGroups(), null);
+      if (securityGroup != null) {
+         networkInterfaceCardProperties.networkSecurityGroup(IdReference.create(securityGroup));
+      }
 
       String networkInterfaceCardName = "jc-nic-" + name;
       return api.getNetworkInterfaceCardApi(azureGroup).createOrUpdate(networkInterfaceCardName, locationName,
-            networkInterfaceCardProperties, ImmutableMap.of("jclouds", name));
+            networkInterfaceCardProperties.build(), ImmutableMap.of("jclouds", name));
    }
 
    private StorageProfile createStorageProfile(String name, Image image, String blob) {

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/config/AzureComputeServiceContextModule.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/config/AzureComputeServiceContextModule.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/config/AzureComputeServiceContextModule.java
index 34a5160..6a577eb 100644
--- a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/config/AzureComputeServiceContextModule.java
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/config/AzureComputeServiceContextModule.java
@@ -42,15 +42,22 @@ import javax.inject.Singleton;
 import org.jclouds.azurecompute.arm.AzureComputeApi;
 import org.jclouds.azurecompute.arm.compute.AzureComputeService;
 import org.jclouds.azurecompute.arm.compute.AzureComputeServiceAdapter;
+import org.jclouds.azurecompute.arm.compute.domain.RegionAndIdAndIngressRules;
 import org.jclouds.azurecompute.arm.compute.extensions.AzureComputeImageExtension;
+import org.jclouds.azurecompute.arm.compute.extensions.AzureComputeSecurityGroupExtension;
 import org.jclouds.azurecompute.arm.compute.functions.LocationToLocation;
+import org.jclouds.azurecompute.arm.compute.functions.NetworkSecurityGroupToSecurityGroup;
+import org.jclouds.azurecompute.arm.compute.functions.NetworkSecurityRuleToIpPermission;
 import org.jclouds.azurecompute.arm.compute.functions.ResourceDefinitionToCustomImage;
 import org.jclouds.azurecompute.arm.compute.functions.VMHardwareToHardware;
 import org.jclouds.azurecompute.arm.compute.functions.VMImageToImage;
 import org.jclouds.azurecompute.arm.compute.functions.VirtualMachineToNodeMetadata;
+import org.jclouds.azurecompute.arm.compute.loaders.CreateSecurityGroupIfNeeded;
 import org.jclouds.azurecompute.arm.compute.options.AzureTemplateOptions;
 import org.jclouds.azurecompute.arm.compute.strategy.CreateResourceGroupThenCreateNodes;
 import org.jclouds.azurecompute.arm.domain.Location;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityGroup;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRule;
 import org.jclouds.azurecompute.arm.domain.PublicIPAddress;
 import org.jclouds.azurecompute.arm.domain.ResourceDefinition;
 import org.jclouds.azurecompute.arm.domain.VMHardware;
@@ -64,32 +71,38 @@ import org.jclouds.compute.ComputeServiceAdapter;
 import org.jclouds.compute.config.ComputeServiceAdapterContextModule;
 import org.jclouds.compute.domain.Hardware;
 import org.jclouds.compute.domain.NodeMetadata;
+import org.jclouds.compute.domain.SecurityGroup;
 import org.jclouds.compute.extensions.ImageExtension;
+import org.jclouds.compute.extensions.SecurityGroupExtension;
 import org.jclouds.compute.functions.NodeAndTemplateOptionsToStatement;
 import org.jclouds.compute.functions.NodeAndTemplateOptionsToStatementWithoutPublicKey;
 import org.jclouds.compute.options.TemplateOptions;
 import org.jclouds.compute.reference.ComputeServiceConstants.PollPeriod;
 import org.jclouds.compute.reference.ComputeServiceConstants.Timeouts;
 import org.jclouds.compute.strategy.CreateNodesInGroupThenAddToSet;
+import org.jclouds.net.domain.IpPermission;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
 import com.google.inject.Inject;
 import com.google.inject.Provides;
 import com.google.inject.TypeLiteral;
 import com.google.inject.assistedinject.FactoryModuleBuilder;
 
-public class AzureComputeServiceContextModule
-        extends ComputeServiceAdapterContextModule<VirtualMachine, VMHardware, VMImage, Location> {
+public class AzureComputeServiceContextModule extends
+      ComputeServiceAdapterContextModule<VirtualMachine, VMHardware, VMImage, Location> {
 
    @Override
    protected void configure() {
       super.configure();
-      
+
       bind(new TypeLiteral<ComputeServiceAdapter<VirtualMachine, VMHardware, VMImage, Location>>() {
       }).to(AzureComputeServiceAdapter.class);
-      
+
       bind(new TypeLiteral<Function<VMImage, org.jclouds.compute.domain.Image>>() {
       }).to(VMImageToImage.class);
       bind(new TypeLiteral<Function<VMHardware, Hardware>>() {
@@ -98,19 +111,27 @@ public class AzureComputeServiceContextModule
       }).to(VirtualMachineToNodeMetadata.class);
       bind(new TypeLiteral<Function<Location, org.jclouds.domain.Location>>() {
       }).to(LocationToLocation.class);
+      bind(new TypeLiteral<Function<NetworkSecurityGroup, SecurityGroup>>() {
+      }).to(NetworkSecurityGroupToSecurityGroup.class);
+      bind(new TypeLiteral<Function<NetworkSecurityRule, IpPermission>>() {
+      }).to(NetworkSecurityRuleToIpPermission.class);
       bind(ComputeService.class).to(AzureComputeService.class);
-      
+
       install(new LocationsFromComputeServiceAdapterModule<VirtualMachine, VMHardware, VMImage, Location>() {
       });
-      
+
       install(new FactoryModuleBuilder().build(ResourceDefinitionToCustomImage.Factory.class));
 
       bind(TemplateOptions.class).to(AzureTemplateOptions.class);
       bind(NodeAndTemplateOptionsToStatement.class).to(NodeAndTemplateOptionsToStatementWithoutPublicKey.class);
       bind(CreateNodesInGroupThenAddToSet.class).to(CreateResourceGroupThenCreateNodes.class);
-      
+      bind(new TypeLiteral<CacheLoader<RegionAndIdAndIngressRules, String>>() {
+      }).to(CreateSecurityGroupIfNeeded.class);
+
       bind(new TypeLiteral<ImageExtension>() {
       }).to(AzureComputeImageExtension.class);
+      bind(new TypeLiteral<SecurityGroupExtension>() {
+      }).to(AzureComputeSecurityGroupExtension.class);
    }
 
    @Singleton
@@ -190,63 +211,82 @@ public class AzureComputeServiceContextModule
    }
 
    @Provides
+   @Singleton
+   protected final LoadingCache<RegionAndIdAndIngressRules, String> securityGroupMap(
+         CacheLoader<RegionAndIdAndIngressRules, String> in) {
+      return CacheBuilder.newBuilder().build(in);
+   }
+
+   @Provides
    @Named(TIMEOUT_NODE_RUNNING)
    protected VirtualMachineInStatePredicateFactory provideVirtualMachineRunningPredicate(final AzureComputeApi api,
-         Timeouts timeouts, PollPeriod pollPeriod) {
+         final Timeouts timeouts, final PollPeriod pollPeriod) {
       return new VirtualMachineInStatePredicateFactory(api, PowerState.RUNNING, timeouts.nodeRunning,
             pollPeriod.pollInitialPeriod, pollPeriod.pollMaxPeriod);
    }
-   
+
    @Provides
    @Named(TIMEOUT_NODE_TERMINATED)
-   protected Predicate<URI> provideNodeTerminatedPredicate(final AzureComputeApi api, Timeouts timeouts, PollPeriod pollPeriod) {
+   protected Predicate<URI> provideNodeTerminatedPredicate(final AzureComputeApi api, final Timeouts timeouts,
+         final PollPeriod pollPeriod) {
       return retry(new ActionDonePredicate(api), timeouts.nodeTerminated, pollPeriod.pollInitialPeriod,
-              pollPeriod.pollMaxPeriod);
+            pollPeriod.pollMaxPeriod);
    }
 
    @Provides
    @Named(TIMEOUT_IMAGE_AVAILABLE)
-   protected Predicate<URI> provideImageAvailablePredicate(final AzureComputeApi api, Timeouts timeouts, PollPeriod pollPeriod) {
+   protected Predicate<URI> provideImageAvailablePredicate(final AzureComputeApi api, final Timeouts timeouts,
+         final PollPeriod pollPeriod) {
       return retry(new ImageDonePredicate(api), timeouts.imageAvailable, pollPeriod.pollInitialPeriod,
-              pollPeriod.pollMaxPeriod);
+            pollPeriod.pollMaxPeriod);
    }
 
    @Provides
    @Named(TIMEOUT_RESOURCE_DELETED)
-   protected Predicate<URI> provideResourceDeletedPredicate(final AzureComputeApi api, Timeouts timeouts, PollPeriod pollPeriod) {
+   protected Predicate<URI> provideResourceDeletedPredicate(final AzureComputeApi api, final Timeouts timeouts,
+         final PollPeriod pollPeriod) {
       return retry(new ActionDonePredicate(api), timeouts.nodeTerminated, pollPeriod.pollInitialPeriod,
-              pollPeriod.pollMaxPeriod);
+            pollPeriod.pollMaxPeriod);
    }
 
    @Provides
    @Named(TIMEOUT_NODE_SUSPENDED)
    protected VirtualMachineInStatePredicateFactory provideNodeSuspendedPredicate(final AzureComputeApi api,
-         Timeouts timeouts, PollPeriod pollPeriod) {
+         final Timeouts timeouts, final PollPeriod pollPeriod) {
       return new VirtualMachineInStatePredicateFactory(api, PowerState.STOPPED, timeouts.nodeTerminated,
             pollPeriod.pollInitialPeriod, pollPeriod.pollMaxPeriod);
    }
-   
+
    @Provides
    protected PublicIpAvailablePredicateFactory providePublicIpAvailablePredicate(final AzureComputeApi api,
-         final AzureComputeServiceContextModule.AzureComputeConstants azureComputeConstants, Timeouts timeouts,
-         PollPeriod pollPeriod) {
+         final AzureComputeServiceContextModule.AzureComputeConstants azureComputeConstants, final Timeouts timeouts,
+         final PollPeriod pollPeriod) {
       return new PublicIpAvailablePredicateFactory(api, azureComputeConstants.operationTimeout(),
             azureComputeConstants.operationPollInitialPeriod(), azureComputeConstants.operationPollMaxPeriod());
    }
 
+   @Provides
+   protected SecurityGroupAvailablePredicateFactory provideSecurityGroupAvailablePredicate(final AzureComputeApi api,
+         final AzureComputeServiceContextModule.AzureComputeConstants azureComputeConstants, final Timeouts timeouts,
+         final PollPeriod pollPeriod) {
+      return new SecurityGroupAvailablePredicateFactory(api, azureComputeConstants.operationTimeout(),
+            azureComputeConstants.operationPollInitialPeriod(), azureComputeConstants.operationPollMaxPeriod());
+   }
+
    @VisibleForTesting
    static class ActionDonePredicate implements Predicate<URI> {
 
       private final AzureComputeApi api;
 
-      public ActionDonePredicate(AzureComputeApi api) {
+      public ActionDonePredicate(final AzureComputeApi api) {
          this.api = checkNotNull(api, "api must not be null");
       }
 
       @Override
-      public boolean apply(URI uri) {
+      public boolean apply(final URI uri) {
          checkNotNull(uri, "uri cannot be null");
-         return (ParseJobStatus.JobStatus.DONE == api.getJobApi().jobStatus(uri)) || (ParseJobStatus.JobStatus.NO_CONTENT == api.getJobApi().jobStatus(uri));
+         return ParseJobStatus.JobStatus.DONE == api.getJobApi().jobStatus(uri)
+               || ParseJobStatus.JobStatus.NO_CONTENT == api.getJobApi().jobStatus(uri);
       }
 
    }
@@ -256,14 +296,16 @@ public class AzureComputeServiceContextModule
 
       private final AzureComputeApi api;
 
-      public ImageDonePredicate(AzureComputeApi api) {
+      public ImageDonePredicate(final AzureComputeApi api) {
          this.api = checkNotNull(api, "api must not be null");
       }
 
       @Override
-      public boolean apply(URI uri) {
+      public boolean apply(final URI uri) {
          checkNotNull(uri, "uri cannot be null");
-         if (api.getJobApi().jobStatus(uri) != ParseJobStatus.JobStatus.DONE) return false;
+         if (api.getJobApi().jobStatus(uri) != ParseJobStatus.JobStatus.DONE) {
+            return false;
+         }
          List<ResourceDefinition> definitions = api.getJobApi().captureStatus(uri);
          return definitions != null;
       }
@@ -277,8 +319,8 @@ public class AzureComputeServiceContextModule
       private final long period;
       private final long maxPeriod;
 
-      VirtualMachineInStatePredicateFactory(AzureComputeApi api, PowerState powerState, long timeout,
-            long period, long maxPeriod) {
+      VirtualMachineInStatePredicateFactory(final AzureComputeApi api, final PowerState powerState, final long timeout,
+            final long period, final long maxPeriod) {
          this.api = checkNotNull(api, "api cannot be null");
          this.powerState = checkNotNull(powerState, "powerState cannot be null");
          this.timeout = timeout;
@@ -289,17 +331,18 @@ public class AzureComputeServiceContextModule
       public Predicate<String> create(final String azureGroup) {
          return retry(new Predicate<String>() {
             @Override
-            public boolean apply(String name) {
+            public boolean apply(final String name) {
                checkNotNull(name, "name cannot be null");
                VirtualMachineInstance vmInstance = api.getVirtualMachineApi(azureGroup).getInstanceDetails(name);
-               if (vmInstance == null)
+               if (vmInstance == null) {
                   return false;
+               }
                return powerState == vmInstance.powerState();
             }
          }, timeout, period, maxPeriod);
       }
    }
-   
+
    public static class PublicIpAvailablePredicateFactory {
 
       private final AzureComputeApi api;
@@ -307,25 +350,57 @@ public class AzureComputeServiceContextModule
       private final long period;
       private final long maxPeriod;
 
-      PublicIpAvailablePredicateFactory(AzureComputeApi api, long timeout,
-            long period, long maxPeriod) {
+      PublicIpAvailablePredicateFactory(final AzureComputeApi api, final long timeout, final long period,
+            final long maxPeriod) {
          this.api = checkNotNull(api, "api cannot be null");
          this.timeout = timeout;
          this.period = period;
          this.maxPeriod = maxPeriod;
       }
-      
+
       public Predicate<String> create(final String azureGroup) {
          return retry(new Predicate<String>() {
             @Override
-            public boolean apply(String name) {
+            public boolean apply(final String name) {
                checkNotNull(name, "name cannot be null");
                PublicIPAddress publicIp = api.getPublicIPAddressApi(azureGroup).get(name);
-               if (publicIp == null) return false;
+               if (publicIp == null) {
+                  return false;
+               }
                return publicIp.properties().provisioningState().equalsIgnoreCase("Succeeded");
             }
          }, timeout, period, maxPeriod);
       }
    }
 
+   public static class SecurityGroupAvailablePredicateFactory {
+      private final AzureComputeApi api;
+      private final long timeout;
+      private final long period;
+      private final long maxPeriod;
+
+      SecurityGroupAvailablePredicateFactory(final AzureComputeApi api, final long timeout, final long period,
+            final long maxPeriod) {
+         this.api = checkNotNull(api, "api cannot be null");
+         this.timeout = timeout;
+         this.period = period;
+         this.maxPeriod = maxPeriod;
+      }
+
+      public Predicate<String> create(final String resourceGroup) {
+         checkNotNull(resourceGroup, "resourceGroup cannot be null");
+         return retry(new Predicate<String>() {
+            @Override
+            public boolean apply(final String name) {
+               checkNotNull(name, "name cannot be null");
+               NetworkSecurityGroup sg = api.getNetworkSecurityGroupApi(resourceGroup).get(name);
+               if (sg == null) {
+                  return false;
+               }
+               return sg.properties().provisioningState().equalsIgnoreCase("Succeeded");
+            }
+         }, timeout, period, maxPeriod);
+      }
+   }
+
 }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/domain/RegionAndIdAndIngressRules.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/domain/RegionAndIdAndIngressRules.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/domain/RegionAndIdAndIngressRules.java
new file mode 100644
index 0000000..fa9730d
--- /dev/null
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/domain/RegionAndIdAndIngressRules.java
@@ -0,0 +1,66 @@
+/*
+ * 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.jclouds.azurecompute.arm.compute.domain;
+
+import org.jclouds.azurecompute.arm.domain.RegionAndId;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.base.Objects;
+
+@AutoValue
+public abstract class RegionAndIdAndIngressRules {
+
+   abstract RegionAndId regionAndId(); // Intentionally hidden
+   public abstract int[] inboundPorts();
+
+   RegionAndIdAndIngressRules() {
+
+   }
+
+   public static RegionAndIdAndIngressRules create(String region, String id, int[] inboundPorts) {
+      return new AutoValue_RegionAndIdAndIngressRules(RegionAndId.fromRegionAndId(region, id), inboundPorts);
+   }
+
+   public String id() {
+      return regionAndId().id();
+   }
+
+   public String region() {
+      return regionAndId().region();
+   }
+
+   // Intentionally delegate equals and hashcode to the fields in the parent
+   // class so that we can search only by region/id in a map
+
+   @Override
+   public int hashCode() {
+      return Objects.hashCode(region(), id());
+   }
+
+   @Override
+   public boolean equals(Object obj) {
+      if (obj == this) {
+         return true;
+      }
+      if (!(obj instanceof RegionAndId)) {
+         return false;
+      }
+      RegionAndId that = (RegionAndId) obj;
+      return Objects.equal(region(), that.region()) && Objects.equal(id(), that.id());
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/extensions/AzureComputeSecurityGroupExtension.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/extensions/AzureComputeSecurityGroupExtension.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/extensions/AzureComputeSecurityGroupExtension.java
new file mode 100644
index 0000000..12d140b
--- /dev/null
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/extensions/AzureComputeSecurityGroupExtension.java
@@ -0,0 +1,315 @@
+/*
+ * 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.jclouds.azurecompute.arm.compute.extensions;
+
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Predicates.equalTo;
+import static com.google.common.base.Predicates.notNull;
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.filter;
+import static com.google.common.collect.Iterables.transform;
+import static org.jclouds.azurecompute.arm.config.AzureComputeProperties.TIMEOUT_RESOURCE_DELETED;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import javax.annotation.Resource;
+import javax.inject.Inject;
+import javax.inject.Named;
+
+import org.jclouds.azurecompute.arm.AzureComputeApi;
+import org.jclouds.azurecompute.arm.compute.config.AzureComputeServiceContextModule.SecurityGroupAvailablePredicateFactory;
+import org.jclouds.azurecompute.arm.compute.functions.LocationToResourceGroupName;
+import org.jclouds.azurecompute.arm.domain.IdReference;
+import org.jclouds.azurecompute.arm.domain.NetworkInterfaceCard;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityGroup;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityGroupProperties;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRule;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties.Access;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties.Direction;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties.Protocol;
+import org.jclouds.azurecompute.arm.domain.RegionAndId;
+import org.jclouds.azurecompute.arm.domain.VirtualMachine;
+import org.jclouds.azurecompute.arm.features.NetworkSecurityGroupApi;
+import org.jclouds.azurecompute.arm.features.NetworkSecurityRuleApi;
+import org.jclouds.collect.Memoized;
+import org.jclouds.compute.domain.SecurityGroup;
+import org.jclouds.compute.domain.SecurityGroupBuilder;
+import org.jclouds.compute.extensions.SecurityGroupExtension;
+import org.jclouds.compute.reference.ComputeServiceConstants;
+import org.jclouds.domain.Location;
+import org.jclouds.logging.Logger;
+import org.jclouds.net.domain.IpPermission;
+import org.jclouds.net.domain.IpProtocol;
+
+import com.google.common.base.Function;
+import com.google.common.base.Objects;
+import com.google.common.base.Predicate;
+import com.google.common.base.Splitter;
+import com.google.common.base.Supplier;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Ordering;
+
+public class AzureComputeSecurityGroupExtension implements SecurityGroupExtension {
+   @Resource
+   @Named(ComputeServiceConstants.COMPUTE_LOGGER)
+   protected Logger logger = Logger.NULL;
+
+   private final AzureComputeApi api;
+   private final Function<NetworkSecurityGroup, SecurityGroup> securityGroupConverter;
+   private final LocationToResourceGroupName locationToResourceGroupName;
+   private final Supplier<Set<? extends Location>> locations;
+   private final SecurityGroupAvailablePredicateFactory securityGroupAvailable;
+   private final Predicate<URI> resourceDeleted;
+
+   @Inject
+   AzureComputeSecurityGroupExtension(AzureComputeApi api, @Memoized Supplier<Set<? extends Location>> locations,
+         LocationToResourceGroupName locationToResourceGroupName,
+         Function<NetworkSecurityGroup, SecurityGroup> groupConverter,
+         SecurityGroupAvailablePredicateFactory securityRuleAvailable,
+         @Named(TIMEOUT_RESOURCE_DELETED) Predicate<URI> resourceDeleted) {
+      this.api = api;
+      this.locations = locations;
+      this.securityGroupConverter = groupConverter;
+      this.locationToResourceGroupName = locationToResourceGroupName;
+      this.securityGroupAvailable = securityRuleAvailable;
+      this.resourceDeleted = resourceDeleted;
+   }
+
+   @Override
+   public Set<SecurityGroup> listSecurityGroups() {
+      return ImmutableSet.copyOf(concat(transform(locations.get(), new Function<Location, Set<SecurityGroup>>() {
+         @Override
+         public Set<SecurityGroup> apply(Location input) {
+            return listSecurityGroupsInLocation(input);
+         }
+      })));
+   }
+
+   @Override
+   public Set<SecurityGroup> listSecurityGroupsInLocation(Location location) {
+      logger.debug(">> getting security groups for %s...", location);
+      final String resourcegroup = locationToResourceGroupName.apply(location.getId());
+      List<NetworkSecurityGroup> networkGroups = api.getNetworkSecurityGroupApi(resourcegroup).list();
+      return ImmutableSet.copyOf(transform(filter(networkGroups, notNull()), securityGroupConverter));
+   }
+
+   @Override
+   public Set<SecurityGroup> listSecurityGroupsForNode(String nodeId) {
+      logger.debug(">> getting security groups for node %s...", nodeId);
+
+      final RegionAndId regionAndId = RegionAndId.fromSlashEncoded(nodeId);
+      final String resourceGroup = locationToResourceGroupName.apply(regionAndId.region());
+
+      VirtualMachine vm = api.getVirtualMachineApi(resourceGroup).get(regionAndId.id());
+      List<IdReference> networkInterfacesIdReferences = vm.properties().networkProfile().networkInterfaces();
+      List<NetworkSecurityGroup> networkGroups = new ArrayList<NetworkSecurityGroup>();
+
+      for (IdReference networkInterfaceCardIdReference : networkInterfacesIdReferences) {
+         String nicName = Iterables.getLast(Splitter.on("/").split(networkInterfaceCardIdReference.id()));
+         NetworkInterfaceCard card = api.getNetworkInterfaceCardApi(resourceGroup).get(nicName);
+         if (card != null && card.properties().networkSecurityGroup() != null) {
+            String secGroupName = Iterables.getLast(Splitter.on("/").split(
+                  card.properties().networkSecurityGroup().id()));
+            NetworkSecurityGroup group = api.getNetworkSecurityGroupApi(resourceGroup).get(secGroupName);
+            networkGroups.add(group);
+         }
+      }
+
+      return ImmutableSet.copyOf(transform(filter(networkGroups, notNull()), securityGroupConverter));
+   }
+
+   @Override
+   public SecurityGroup getSecurityGroupById(String id) {
+      logger.debug(">> getting security group %s...", id);
+      final RegionAndId regionAndId = RegionAndId.fromSlashEncoded(id);
+      final String resourceGroup = locationToResourceGroupName.apply(regionAndId.region());
+
+      return securityGroupConverter.apply(api.getNetworkSecurityGroupApi(resourceGroup).get(regionAndId.id()));
+   }
+
+   @Override
+   public SecurityGroup createSecurityGroup(String name, Location location) {
+      final String resourceGroup = locationToResourceGroupName.apply(location.getId());
+
+      logger.debug(">> creating security group %s in %s...", name, location);
+
+      SecurityGroupBuilder builder = new SecurityGroupBuilder();
+      builder.name(name);
+      builder.location(location);
+
+      return securityGroupConverter.apply(api.getNetworkSecurityGroupApi(resourceGroup).createOrUpdate(name,
+            location.getId(), null, NetworkSecurityGroupProperties.builder().build()));
+   }
+
+   @Override
+   public boolean removeSecurityGroup(String id) {
+      logger.debug(">> deleting security group %s...", id);
+
+      final RegionAndId regionAndId = RegionAndId.fromSlashEncoded(id);
+      final String resourcegroup = locationToResourceGroupName.apply(regionAndId.region());
+      URI uri = api.getNetworkSecurityGroupApi(resourcegroup).delete(regionAndId.id());
+      return resourceDeleted.apply(uri);
+   }
+
+   @Override
+   public SecurityGroup addIpPermission(IpPermission ipPermission, SecurityGroup group) {
+      return addIpPermission(ipPermission.getIpProtocol(), ipPermission.getFromPort(), ipPermission.getToPort(),
+            ipPermission.getTenantIdGroupNamePairs(), ipPermission.getCidrBlocks(), ipPermission.getGroupIds(), group);
+   }
+
+   @Override
+   public SecurityGroup removeIpPermission(IpPermission ipPermission, SecurityGroup group) {
+      return removeIpPermission(ipPermission.getIpProtocol(), ipPermission.getFromPort(), ipPermission.getToPort(),
+            ipPermission.getTenantIdGroupNamePairs(), ipPermission.getCidrBlocks(), ipPermission.getGroupIds(), group);
+   }
+
+   @Override
+   public SecurityGroup addIpPermission(IpProtocol protocol, int startPort, int endPort,
+         Multimap<String, String> tenantIdGroupNamePairs, Iterable<String> ipRanges, Iterable<String> groupIds,
+         SecurityGroup group) {
+      String portRange = startPort + "-" + endPort;
+      String ruleName = protocol + "-" + portRange;
+
+      logger.debug(">> adding ip permission [%s] to %s...", ruleName, group.getName());
+
+      // TODO: Support Azure network tags somehow?
+
+      final RegionAndId regionAndId = RegionAndId.fromSlashEncoded(group.getId());
+      final String resourceGroup = locationToResourceGroupName.apply(regionAndId.region());
+
+      NetworkSecurityGroupApi groupApi = api.getNetworkSecurityGroupApi(resourceGroup);
+      NetworkSecurityGroup networkSecurityGroup = groupApi.get(regionAndId.id());
+
+      if (networkSecurityGroup == null) {
+         throw new IllegalArgumentException("Security group " + group.getName() + " was not found");
+      }
+
+      NetworkSecurityRuleApi ruleApi = api.getNetworkSecurityRuleApi(resourceGroup, networkSecurityGroup.name());
+      int nextPriority = getRuleStartingPriority(ruleApi);
+
+      for (String ipRange : ipRanges) {
+         NetworkSecurityRuleProperties properties = NetworkSecurityRuleProperties.builder()
+               .protocol(Protocol.fromValue(protocol.name())) //
+               .sourceAddressPrefix(ipRange) //
+               .sourcePortRange("*") //
+               .destinationAddressPrefix("*") //
+               .destinationPortRange(portRange) //
+               .direction(Direction.Inbound) //
+               .access(Access.Allow) //
+               .priority(nextPriority++) //
+               .build();
+
+         logger.debug(">> creating network security rule %s for %s...", ruleName, ipRange);
+
+         ruleApi.createOrUpdate(ruleName, properties);
+
+         checkState(securityGroupAvailable.create(resourceGroup).apply(networkSecurityGroup.name()),
+               "Security group was not updated in the configured timeout");
+      }
+
+      return getSecurityGroupById(group.getId());
+   }
+
+   @Override
+   public SecurityGroup removeIpPermission(final IpProtocol protocol, int startPort, int endPort,
+         Multimap<String, String> tenantIdGroupNamePairs, final Iterable<String> ipRanges, Iterable<String> groupIds,
+         SecurityGroup group) {
+      final String portRange = startPort + "-" + endPort;
+      String ruleName = protocol + "-" + portRange;
+
+      logger.debug(">> deleting ip permissions matching [%s] from %s...", ruleName, group.getName());
+
+      final RegionAndId regionAndId = RegionAndId.fromSlashEncoded(group.getId());
+      final String resourceGroup = locationToResourceGroupName.apply(regionAndId.region());
+
+      NetworkSecurityGroupApi groupApi = api.getNetworkSecurityGroupApi(resourceGroup);
+      NetworkSecurityGroup networkSecurityGroup = groupApi.get(regionAndId.id());
+
+      if (networkSecurityGroup == null) {
+         throw new IllegalArgumentException("Security group " + group.getName() + " was not found");
+      }
+
+      NetworkSecurityRuleApi ruleApi = api.getNetworkSecurityRuleApi(resourceGroup, networkSecurityGroup.name());
+      Iterable<NetworkSecurityRule> rules = filter(ruleApi.list(), new Predicate<NetworkSecurityRule>() {
+         @Override
+         public boolean apply(NetworkSecurityRule input) {
+            NetworkSecurityRuleProperties props = input.properties();
+            return Objects.equal(portRange, props.destinationPortRange())
+                  && Objects.equal(Protocol.fromValue(protocol.name()), props.protocol())
+                  && Objects.equal(Direction.Inbound, props.direction()) //
+                  && Objects.equal(Access.Allow, props.access())
+                  && any(ipRanges, equalTo(props.sourceAddressPrefix().replace("*", "0.0.0.0/0")));
+         }
+      });
+
+      for (NetworkSecurityRule matchingRule : rules) {
+         logger.debug(">> deleting network security rule %s from %s...", matchingRule.name(), group.getName());
+         ruleApi.delete(matchingRule.name());
+         checkState(securityGroupAvailable.create(resourceGroup).apply(networkSecurityGroup.name()),
+               "Security group was not updated in the configured timeout");
+      }
+
+      return getSecurityGroupById(group.getId());
+   }
+
+   @Override
+   public boolean supportsTenantIdGroupNamePairs() {
+      return false;
+   }
+
+   @Override
+   public boolean supportsTenantIdGroupIdPairs() {
+      return false;
+   }
+
+   @Override
+   public boolean supportsGroupIds() {
+      return false;
+   }
+
+   @Override
+   public boolean supportsPortRangesForGroups() {
+      return false;
+   }
+
+   @Override
+   public boolean supportsExclusionCidrBlocks() {
+      return false;
+   }
+
+   private int getRuleStartingPriority(NetworkSecurityRuleApi ruleApi) {
+      List<NetworkSecurityRule> existingRules = ruleApi.list();
+      return existingRules.isEmpty() ? 100 : rulesByPriority().max(existingRules).properties().priority() + 1;
+   }
+
+   private static Ordering<NetworkSecurityRule> rulesByPriority() {
+      return new Ordering<NetworkSecurityRule>() {
+         @Override
+         public int compare(NetworkSecurityRule left, NetworkSecurityRule right) {
+            return left.properties().priority() - right.properties().priority();
+         }
+      };
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/NetworkSecurityGroupToSecurityGroup.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/NetworkSecurityGroupToSecurityGroup.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/NetworkSecurityGroupToSecurityGroup.java
new file mode 100644
index 0000000..65f5b0d
--- /dev/null
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/NetworkSecurityGroupToSecurityGroup.java
@@ -0,0 +1,72 @@
+/*
+ * 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.jclouds.azurecompute.arm.compute.functions;
+
+import static com.google.common.collect.Iterables.filter;
+import static com.google.common.collect.Iterables.transform;
+import static org.jclouds.azurecompute.arm.compute.functions.NetworkSecurityRuleToIpPermission.InboundRule;
+import static org.jclouds.azurecompute.arm.compute.functions.VirtualMachineToNodeMetadata.getLocation;
+
+import java.util.Set;
+
+import javax.inject.Singleton;
+
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityGroup;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRule;
+import org.jclouds.azurecompute.arm.domain.RegionAndId;
+import org.jclouds.collect.Memoized;
+import org.jclouds.compute.domain.SecurityGroup;
+import org.jclouds.compute.domain.SecurityGroupBuilder;
+import org.jclouds.domain.Location;
+import org.jclouds.net.domain.IpPermission;
+
+import com.google.common.base.Function;
+import com.google.common.base.Supplier;
+import com.google.inject.Inject;
+
+@Singleton
+public class NetworkSecurityGroupToSecurityGroup implements Function<NetworkSecurityGroup, SecurityGroup> {
+   private final Function<NetworkSecurityRule, IpPermission> ruleToPermission;
+   private final Supplier<Set<? extends Location>> locations;
+
+   @Inject
+   NetworkSecurityGroupToSecurityGroup(Function<NetworkSecurityRule, IpPermission> ruleToPermission,
+         @Memoized Supplier<Set<? extends Location>> locations) {
+      this.ruleToPermission = ruleToPermission;
+      this.locations = locations;
+   }
+
+   @Override
+   public SecurityGroup apply(NetworkSecurityGroup input) {
+      SecurityGroupBuilder builder = new SecurityGroupBuilder();
+
+      builder.id(RegionAndId.fromRegionAndId(input.location(), input.name()).slashEncode());
+      builder.providerId(input.properties().resourceGuid());
+      builder.name(input.name());
+      builder.location(getLocation(locations, input.location()));
+
+      if (input.properties().securityRules() != null) {
+         // We just supoprt security groups that allow traffic to a set of
+         // targets. We don't support deny rules or origin based rules in the
+         // security group api.
+         builder.ipPermissions(transform(filter(input.properties().securityRules(), InboundRule), ruleToPermission));
+      }
+
+      return builder.build();
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/NetworkSecurityRuleToIpPermission.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/NetworkSecurityRuleToIpPermission.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/NetworkSecurityRuleToIpPermission.java
new file mode 100644
index 0000000..e601d59
--- /dev/null
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/NetworkSecurityRuleToIpPermission.java
@@ -0,0 +1,76 @@
+/*
+ * 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.jclouds.azurecompute.arm.compute.functions;
+
+import javax.annotation.Resource;
+import javax.inject.Named;
+import javax.inject.Singleton;
+
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRule;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties.Access;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties.Direction;
+import org.jclouds.compute.reference.ComputeServiceConstants;
+import org.jclouds.logging.Logger;
+import org.jclouds.net.domain.IpPermission;
+import org.jclouds.net.domain.IpProtocol;
+import org.jclouds.net.util.IpPermissions;
+import org.jclouds.net.util.IpPermissions.PortSelection;
+import org.jclouds.net.util.IpPermissions.ToSourceSelection;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+
+@Singleton
+public class NetworkSecurityRuleToIpPermission implements Function<NetworkSecurityRule, IpPermission> {
+
+   public static final Predicate<NetworkSecurityRule> InboundRule = new Predicate<NetworkSecurityRule>() {
+      @Override
+      public boolean apply(NetworkSecurityRule input) {
+         return Direction.Inbound.equals(input.properties().direction())
+               && Access.Allow.equals(input.properties().access());
+      }
+   };
+
+   @Resource
+   @Named(ComputeServiceConstants.COMPUTE_LOGGER)
+   protected Logger logger = Logger.NULL;
+
+   @Override
+   public IpPermission apply(final NetworkSecurityRule rule) {
+      if (!InboundRule.apply(rule)) {
+         logger.warn(">> ignoring non-inbound networks ecurity rule %s...", rule.name());
+         return null;
+      }
+
+      IpPermission permissions = IpPermissions.permit(IpProtocol.fromValue(rule.properties().protocol().name()));
+
+      String portRange = rule.properties().destinationPortRange();
+      if (!"*".equals(portRange)) {
+         String[] range = portRange.split("-");
+         permissions = PortSelection.class.cast(permissions).fromPort(Integer.parseInt(range[0]))
+               .to(Integer.parseInt(range[1]));
+      }
+
+      if (!"*".equals(rule.properties().sourceAddressPrefix())) {
+         permissions = ToSourceSelection.class.cast(permissions).originatingFromCidrBlock(
+               rule.properties().sourceAddressPrefix());
+      }
+
+      return permissions;
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/VirtualMachineToNodeMetadata.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/VirtualMachineToNodeMetadata.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/VirtualMachineToNodeMetadata.java
index 5e98fbf..22c818c 100644
--- a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/VirtualMachineToNodeMetadata.java
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/functions/VirtualMachineToNodeMetadata.java
@@ -191,7 +191,7 @@ public class VirtualMachineToNodeMetadata implements Function<VirtualMachine, No
             .name()));
       
       String locationName = virtualMachine.location();
-      builder.location(getLocation(locationName));
+      builder.location(getLocation(locations, locationName));
 
       Optional<? extends Image> image = findImage(virtualMachine.properties().storageProfile(), locationName, azureGroup);
       if (image.isPresent()) {
@@ -251,7 +251,7 @@ public class VirtualMachineToNodeMetadata implements Function<VirtualMachine, No
       return publicIpAddresses;
    }
 
-   protected Location getLocation(final String locationName) {
+   protected static Location getLocation(Supplier<Set<? extends Location>> locations, final String locationName) {
       return find(locations.get(), new Predicate<Location>() {
          @Override
          public boolean apply(Location location) {

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/loaders/CreateSecurityGroupIfNeeded.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/loaders/CreateSecurityGroupIfNeeded.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/loaders/CreateSecurityGroupIfNeeded.java
new file mode 100644
index 0000000..efb8abc
--- /dev/null
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/loaders/CreateSecurityGroupIfNeeded.java
@@ -0,0 +1,97 @@
+/*
+ * 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.jclouds.azurecompute.arm.compute.loaders;
+
+import static org.jclouds.compute.util.ComputeServiceUtils.getPortRangesFromList;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Resource;
+import javax.inject.Inject;
+import javax.inject.Named;
+import javax.inject.Singleton;
+
+import org.jclouds.azurecompute.arm.AzureComputeApi;
+import org.jclouds.azurecompute.arm.compute.domain.RegionAndIdAndIngressRules;
+import org.jclouds.azurecompute.arm.compute.functions.LocationToResourceGroupName;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityGroup;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityGroupProperties;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRule;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties.Access;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties.Direction;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityRuleProperties.Protocol;
+import org.jclouds.compute.reference.ComputeServiceConstants;
+import org.jclouds.logging.Logger;
+
+import com.google.common.cache.CacheLoader;
+
+@Singleton
+public class CreateSecurityGroupIfNeeded extends CacheLoader<RegionAndIdAndIngressRules, String> {
+   @Resource
+   @Named(ComputeServiceConstants.COMPUTE_LOGGER)
+   protected Logger logger = Logger.NULL;
+
+   private final AzureComputeApi api;
+   private final LocationToResourceGroupName locationToResourceGroupName;
+
+   @Inject
+   CreateSecurityGroupIfNeeded(AzureComputeApi api, LocationToResourceGroupName locationToResourceGroupName) {
+      this.api = api;
+      this.locationToResourceGroupName = locationToResourceGroupName;
+   }
+
+   @Override
+   public String load(RegionAndIdAndIngressRules key) throws Exception {
+      String resourceGroup = locationToResourceGroupName.apply(key.region());
+      return createSecurityGroup(key.region(), resourceGroup, key.id(), key.inboundPorts());
+   }
+
+   private String createSecurityGroup(String location, String resourceGroup, String name, int[] inboundPorts) {
+      logger.debug(">> creating security group %s in %s...", name, location);
+
+      Map<Integer, Integer> portRanges = getPortRangesFromList(inboundPorts);
+
+      List<NetworkSecurityRule> rules = new ArrayList<NetworkSecurityRule>();
+
+      int startPriority = 100;
+      for (Map.Entry<Integer, Integer> portRange : portRanges.entrySet()) {
+         String range = portRange.getKey() + "-" + portRange.getValue();
+         String ruleName = "tcp-" + range;
+
+         NetworkSecurityRuleProperties properties = NetworkSecurityRuleProperties.builder().protocol(Protocol.Tcp) //
+               .sourceAddressPrefix("*") //
+               .sourcePortRange("*") //
+               .destinationAddressPrefix("*") //
+               .destinationPortRange(range) //
+               .direction(Direction.Inbound) //
+               .access(Access.Allow) //
+               .priority(startPriority++) //
+               .build();
+
+         rules.add(NetworkSecurityRule.create(ruleName, null, null, properties));
+      }
+
+      NetworkSecurityGroup securityGroup = api.getNetworkSecurityGroupApi(resourceGroup).createOrUpdate(name, location,
+            null, NetworkSecurityGroupProperties.builder().securityRules(rules).build());
+
+      return securityGroup.id();
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/strategy/CreateResourceGroupThenCreateNodes.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/strategy/CreateResourceGroupThenCreateNodes.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/strategy/CreateResourceGroupThenCreateNodes.java
index 5ce30b0..524eb69 100644
--- a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/strategy/CreateResourceGroupThenCreateNodes.java
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/compute/strategy/CreateResourceGroupThenCreateNodes.java
@@ -16,8 +16,10 @@
  */
 package org.jclouds.azurecompute.arm.compute.strategy;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.collect.Iterables.getOnlyElement;
 import static org.jclouds.azurecompute.arm.compute.functions.VMImageToImage.decodeFieldsFromUniqueId;
 import static org.jclouds.util.Predicates2.retry;
 
@@ -35,8 +37,11 @@ import javax.inject.Singleton;
 import org.jclouds.Constants;
 import org.jclouds.azurecompute.arm.AzureComputeApi;
 import org.jclouds.azurecompute.arm.compute.config.AzureComputeServiceContextModule;
+import org.jclouds.azurecompute.arm.compute.domain.RegionAndIdAndIngressRules;
 import org.jclouds.azurecompute.arm.compute.functions.LocationToResourceGroupName;
 import org.jclouds.azurecompute.arm.compute.options.AzureTemplateOptions;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityGroup;
+import org.jclouds.azurecompute.arm.domain.RegionAndId;
 import org.jclouds.azurecompute.arm.domain.ResourceGroup;
 import org.jclouds.azurecompute.arm.domain.StorageService;
 import org.jclouds.azurecompute.arm.domain.Subnet;
@@ -51,15 +56,18 @@ import org.jclouds.compute.domain.Image;
 import org.jclouds.compute.domain.NodeMetadata;
 import org.jclouds.compute.domain.Template;
 import org.jclouds.compute.functions.GroupNamingConvention;
+import org.jclouds.compute.options.TemplateOptions;
 import org.jclouds.compute.reference.ComputeServiceConstants;
 import org.jclouds.compute.strategy.CreateNodeWithGroupEncodedIntoName;
 import org.jclouds.compute.strategy.CustomizeNodeAndAddToGoodMapOrPutExceptionIntoBadMap;
 import org.jclouds.compute.strategy.ListNodesStrategy;
 import org.jclouds.compute.strategy.impl.CreateNodesWithGroupEncodedIntoNameThenAddToSet;
+import org.jclouds.domain.Location;
 import org.jclouds.logging.Logger;
 
 import com.google.common.base.Predicate;
 import com.google.common.base.Strings;
+import com.google.common.cache.LoadingCache;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Multimap;
 import com.google.common.util.concurrent.ListenableFuture;
@@ -75,34 +83,37 @@ public class CreateResourceGroupThenCreateNodes extends CreateNodesWithGroupEnco
    private final AzureComputeApi api;
    private final AzureComputeServiceContextModule.AzureComputeConstants azureComputeConstants;
    private final LocationToResourceGroupName locationToResourceGroupName;
+   private final LoadingCache<RegionAndIdAndIngressRules, String> securityGroupMap;
 
    @Inject
    protected CreateResourceGroupThenCreateNodes(
-           CreateNodeWithGroupEncodedIntoName addNodeWithGroupStrategy,
-           ListNodesStrategy listNodesStrategy,
-           GroupNamingConvention.Factory namingConvention,
+         CreateNodeWithGroupEncodedIntoName addNodeWithGroupStrategy,
+         ListNodesStrategy listNodesStrategy,
+         GroupNamingConvention.Factory namingConvention,
          @Named(Constants.PROPERTY_USER_THREADS) ListeningExecutorService userExecutor,
          CustomizeNodeAndAddToGoodMapOrPutExceptionIntoBadMap.Factory customizeNodeAndAddToGoodMapOrPutExceptionIntoBadMapFactory,
          AzureComputeApi api, AzureComputeServiceContextModule.AzureComputeConstants azureComputeConstants,
-         LocationToResourceGroupName locationToResourceGroupName) {
+         LocationToResourceGroupName locationToResourceGroupName,
+         LoadingCache<RegionAndIdAndIngressRules, String> securityGroupMap) {
       super(addNodeWithGroupStrategy, listNodesStrategy, namingConvention, userExecutor,
-              customizeNodeAndAddToGoodMapOrPutExceptionIntoBadMapFactory);
+            customizeNodeAndAddToGoodMapOrPutExceptionIntoBadMapFactory);
       this.api = checkNotNull(api, "api cannot be null");
       checkNotNull(userExecutor, "userExecutor cannot be null");
       this.azureComputeConstants = azureComputeConstants;
       this.locationToResourceGroupName = locationToResourceGroupName;
+      this.securityGroupMap = securityGroupMap;
    }
 
    @Override
    public Map<?, ListenableFuture<Void>> execute(String group, int count, Template template,
-                                                 Set<NodeMetadata> goodNodes, Map<NodeMetadata, Exception> badNodes,
-                                                 Multimap<NodeMetadata, CustomizationResponse> customizationResponses) {
+         Set<NodeMetadata> goodNodes, Map<NodeMetadata, Exception> badNodes,
+         Multimap<NodeMetadata, CustomizationResponse> customizationResponses) {
       // If there is a script to be run on the node and public key
       // authentication has been configured, warn users if the private key
       // is not present
       if (hasRunScriptWithKeyAuthAndNoPrivateKey(template)) {
-         logger.warn(">> a runScript was configured but no SSH key has been provided. " +
-                 "Authentication will delegate to the ssh-agent");
+         logger.warn(">> a runScript was configured but no SSH key has been provided. "
+               + "Authentication will delegate to the ssh-agent");
       }
       String azureGroupName = locationToResourceGroupName.apply(template.getLocation().getId());
 
@@ -112,7 +123,7 @@ public class CreateResourceGroupThenCreateNodes extends CreateNodesWithGroupEnco
       ResourceGroup resourceGroup = resourceGroupApi.get(azureGroupName);
       final String location = template.getLocation().getId();
 
-      if (resourceGroup == null){
+      if (resourceGroup == null) {
          final Map<String, String> tags = ImmutableMap.of("description", "jclouds managed VMs");
          resourceGroupApi.create(azureGroupName, location, tags).name();
       }
@@ -125,33 +136,35 @@ public class CreateResourceGroupThenCreateNodes extends CreateNodesWithGroupEnco
       }
 
       this.getOrCreateVirtualNetworkWithSubnet(vnetName, subnetName, location, options, azureGroupName);
+      configureSecurityGroupForOptions(group, azureGroupName, template.getLocation(), options);
 
       StorageService storageService = getOrCreateStorageService(group, azureGroupName, location, template.getImage());
       String blob = storageService.storageServiceProperties().primaryEndpoints().get("blob");
       options.blob(blob);
-      
+
       Map<?, ListenableFuture<Void>> responses = super.execute(group, count, template, goodNodes, badNodes,
-              customizationResponses);
+            customizationResponses);
 
       return responses;
    }
 
-   protected synchronized void getOrCreateVirtualNetworkWithSubnet(
-           final String virtualNetworkName, final String subnetName, final String location,
-           AzureTemplateOptions options, final String azureGroupName) {
+   protected synchronized void getOrCreateVirtualNetworkWithSubnet(final String virtualNetworkName,
+         final String subnetName, final String location, AzureTemplateOptions options, final String azureGroupName) {
 
-      //Subnets belong to a virtual network so that needs to be created first
+      // Subnets belong to a virtual network so that needs to be created first
       VirtualNetworkApi vnApi = api.getVirtualNetworkApi(azureGroupName);
       VirtualNetwork vn = vnApi.get(virtualNetworkName);
 
       if (vn == null) {
-         VirtualNetwork.VirtualNetworkProperties virtualNetworkProperties = VirtualNetwork.VirtualNetworkProperties.builder()
-                 .addressSpace(VirtualNetwork.AddressSpace.create(Arrays.asList(this.azureComputeConstants.azureDefaultVnetAddressPrefixProperty())))
-                 .subnets(
-                         Arrays.asList(
-                                 Subnet.create(subnetName, null, null,
-                                         Subnet.SubnetProperties.builder().addressPrefix(this.azureComputeConstants.azureDefaultSubnetAddressPrefixProperty()).build())))
-                 .build();
+         VirtualNetwork.VirtualNetworkProperties virtualNetworkProperties = VirtualNetwork.VirtualNetworkProperties
+               .builder()
+               .addressSpace(
+                     VirtualNetwork.AddressSpace.create(Arrays.asList(this.azureComputeConstants
+                           .azureDefaultVnetAddressPrefixProperty())))
+               .subnets(
+                     Arrays.asList(Subnet.create(subnetName, null, null, Subnet.SubnetProperties.builder()
+                           .addressPrefix(this.azureComputeConstants.azureDefaultSubnetAddressPrefixProperty()).build())))
+               .build();
          vn = vnApi.createOrUpdate(virtualNetworkName, location, virtualNetworkProperties);
       }
 
@@ -164,10 +177,11 @@ public class CreateResourceGroupThenCreateNodes extends CreateNodesWithGroupEnco
 
    private static boolean hasRunScriptWithKeyAuthAndNoPrivateKey(Template template) {
       return template.getOptions().getRunScript() != null && template.getOptions().getPublicKey() != null
-              && !template.getOptions().hasLoginPrivateKeyOption();
+            && !template.getOptions().hasLoginPrivateKeyOption();
    }
 
-   public StorageService getOrCreateStorageService(String name, String resourceGroupName, String locationName, Image image) {
+   public StorageService getOrCreateStorageService(String name, String resourceGroupName, String locationName,
+         Image image) {
       String storageAccountName = null;
       VMImage imageRef = decodeFieldsFromUniqueId(image.getId());
       if (imageRef.custom()) {
@@ -179,10 +193,12 @@ public class CreateResourceGroupThenCreateNodes extends CreateNodesWithGroupEnco
       }
 
       StorageService storageService = api.getStorageAccountApi(resourceGroupName).get(storageAccountName);
-      if (storageService != null) return storageService;
+      if (storageService != null)
+         return storageService;
 
-      URI uri = api.getStorageAccountApi(resourceGroupName).create(storageAccountName, locationName, ImmutableMap.of("jclouds",
-              name), ImmutableMap.of("accountType", StorageService.AccountType.Standard_LRS.toString()));
+      URI uri = api.getStorageAccountApi(resourceGroupName).create(storageAccountName, locationName,
+            ImmutableMap.of("jclouds", name),
+            ImmutableMap.of("accountType", StorageService.AccountType.Standard_LRS.toString()));
       boolean starageAccountCreated = retry(new Predicate<URI>() {
          @Override
          public boolean apply(URI uri) {
@@ -195,16 +211,44 @@ public class CreateResourceGroupThenCreateNodes extends CreateNodesWithGroupEnco
       return api.getStorageAccountApi(resourceGroupName).get(storageAccountName);
    }
 
+   private void configureSecurityGroupForOptions(String group, String resourceGroup, Location location,
+         TemplateOptions options) {
+
+      checkArgument(options.getGroups().size() <= 1,
+            "Only one security group can be configured for each network interface");
+
+      if (!options.getGroups().isEmpty()) {
+         String groupName = getOnlyElement(options.getGroups());
+         String groupNameWithourRegion = groupName.indexOf('/') == -1 ? groupName : RegionAndId.fromSlashEncoded(
+               groupName).id();
+         NetworkSecurityGroup securityGroup = api.getNetworkSecurityGroupApi(resourceGroup).get(groupNameWithourRegion);
+         checkArgument(securityGroup != null, "Security group %s was not found", groupName);
+         options.securityGroups(securityGroup.id());
+      } else if (options.getInboundPorts().length > 0) {
+         String name = namingConvention.create().sharedNameForGroup(group);
+         RegionAndIdAndIngressRules regionAndIdAndIngressRules = RegionAndIdAndIngressRules.create(location.getId(),
+               name, options.getInboundPorts());
+         // this will create if not yet exists.
+         String securityGroupId = securityGroupMap.getUnchecked(regionAndIdAndIngressRules);
+         options.securityGroups(securityGroupId);
+      }
+   }
+
    /**
     * Generates a valid storage account
     *
-    * Storage account names must be between 3 and 24 characters in length and may contain numbers and lowercase letters only.
+    * Storage account names must be between 3 and 24 characters in length and
+    * may contain numbers and lowercase letters only.
     *
-    * @param name the node name
-    * @return the storage account name starting from a sanitized name (with only numbers and lowercase letters only ).
-    * If sanitized name is between 3 and 24 characters, storage account name is equals to sanitized name.
-    * If sanitized name is less than 3 characters, storage account is sanitized name plus 4 random chars.
-    * If sanitized name is more than 24 characters, storage account is first 10 chars of sanitized name plus 4 random chars plus last 10 chars of sanitized name.
+    * @param name
+    *           the node name
+    * @return the storage account name starting from a sanitized name (with only
+    *         numbers and lowercase letters only ). If sanitized name is between
+    *         3 and 24 characters, storage account name is equals to sanitized
+    *         name. If sanitized name is less than 3 characters, storage account
+    *         is sanitized name plus 4 random chars. If sanitized name is more
+    *         than 24 characters, storage account is first 10 chars of sanitized
+    *         name plus 4 random chars plus last 10 chars of sanitized name.
     */
    public static String generateStorageAccountName(String name) {
       String random = UUID.randomUUID().toString().substring(0, 4);

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/NetworkSecurityGroup.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/NetworkSecurityGroup.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/NetworkSecurityGroup.java
index ebe842e..e7c75e8 100644
--- a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/NetworkSecurityGroup.java
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/NetworkSecurityGroup.java
@@ -25,7 +25,7 @@ import java.util.Map;
 
 @AutoValue
 public abstract class NetworkSecurityGroup {
-   @Nullable
+   public abstract String id();
    public abstract String name();
 
    @Nullable
@@ -40,15 +40,10 @@ public abstract class NetworkSecurityGroup {
    @Nullable
    public abstract String etag();
 
-   @SerializedNames({"name", "location", "tags", "properties", "etag"})
-   public static NetworkSecurityGroup create(final String name,
-                                             final String location,
-                                             final Map<String, String> tags,
-                                             final NetworkSecurityGroupProperties properties,
-                                             final String etag) {
-      return new AutoValue_NetworkSecurityGroup(name, location,
-                                                (tags == null) ? null : ImmutableMap.copyOf(tags),
-                                                properties, etag);
+   @SerializedNames({ "id", "name", "location", "tags", "properties", "etag" })
+   public static NetworkSecurityGroup create(final String id, final String name, final String location,
+         final Map<String, String> tags, final NetworkSecurityGroupProperties properties, final String etag) {
+      return new AutoValue_NetworkSecurityGroup(id, name, location, (tags == null) ? null : ImmutableMap.copyOf(tags),
+            properties, etag);
    }
 }
-

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/RegionAndId.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/RegionAndId.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/RegionAndId.java
index 233109a..4105ee3 100644
--- a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/RegionAndId.java
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/domain/RegionAndId.java
@@ -29,7 +29,7 @@ public abstract class RegionAndId {
    public abstract String region();
    public abstract String id();
    
-   RegionAndId() {
+   protected RegionAndId() {
       
    }
    

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/functions/CleanupResources.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/functions/CleanupResources.java b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/functions/CleanupResources.java
index ead676a..9d2a1c1 100644
--- a/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/functions/CleanupResources.java
+++ b/azurecompute-arm/src/main/java/org/jclouds/azurecompute/arm/functions/CleanupResources.java
@@ -35,10 +35,13 @@ import org.jclouds.azurecompute.arm.compute.functions.LocationToResourceGroupNam
 import org.jclouds.azurecompute.arm.domain.IdReference;
 import org.jclouds.azurecompute.arm.domain.IpConfiguration;
 import org.jclouds.azurecompute.arm.domain.NetworkInterfaceCard;
+import org.jclouds.azurecompute.arm.domain.NetworkSecurityGroup;
 import org.jclouds.azurecompute.arm.domain.RegionAndId;
 import org.jclouds.azurecompute.arm.domain.StorageServiceKeys;
 import org.jclouds.azurecompute.arm.domain.VirtualMachine;
+import org.jclouds.azurecompute.arm.features.NetworkSecurityGroupApi;
 import org.jclouds.azurecompute.arm.util.BlobHelper;
+import org.jclouds.compute.functions.GroupNamingConvention;
 import org.jclouds.compute.reference.ComputeServiceConstants;
 import org.jclouds.logging.Logger;
 
@@ -49,7 +52,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
 @Singleton
-public class CleanupResources implements Function<String, Boolean> {
+public class CleanupResources {
 
    @Resource
    @Named(ComputeServiceConstants.COMPUTE_LOGGER)
@@ -59,22 +62,23 @@ public class CleanupResources implements Function<String, Boolean> {
    private final Predicate<URI> resourceDeleted;
    private final StorageProfileToStorageAccountName storageProfileToStorageAccountName;
    private final LocationToResourceGroupName locationToResourceGroupName;
+   private final GroupNamingConvention.Factory namingConvention;
 
    @Inject
    CleanupResources(AzureComputeApi azureComputeApi, @Named(TIMEOUT_RESOURCE_DELETED) Predicate<URI> resourceDeleted,
          StorageProfileToStorageAccountName storageProfileToStorageAccountName,
-         LocationToResourceGroupName locationToResourceGroupName) {
+         LocationToResourceGroupName locationToResourceGroupName, GroupNamingConvention.Factory namingConvention) {
       this.api = azureComputeApi;
       this.resourceDeleted = resourceDeleted;
       this.storageProfileToStorageAccountName = storageProfileToStorageAccountName;
       this.locationToResourceGroupName = locationToResourceGroupName;
+      this.namingConvention = namingConvention;
    }
 
-   @Override
-   public Boolean apply(final String id) {
+   public boolean cleanupNode(final String id) {
       RegionAndId regionAndId = RegionAndId.fromSlashEncoded(id);
       String group = locationToResourceGroupName.apply(regionAndId.region());
-      
+
       VirtualMachine virtualMachine = api.getVirtualMachineApi(group).get(regionAndId.id());
       if (virtualMachine == null) {
          return true;
@@ -82,10 +86,17 @@ public class CleanupResources implements Function<String, Boolean> {
 
       logger.debug(">> destroying %s ...", regionAndId.slashEncode());
       boolean vmDeleted = deleteVirtualMachine(group, virtualMachine);
-      
+
       // We don't delete the network here, as it is global to the resource
       // group. It will be deleted when the resource group is deleted
 
+      cleanupVirtualMachineNICs(group, virtualMachine);
+      cleanupVirtualMachineStorage(group, virtualMachine);
+
+      return vmDeleted;
+   }
+
+   public void cleanupVirtualMachineNICs(String group, VirtualMachine virtualMachine) {
       for (String nicName : getNetworkCardInterfaceNames(virtualMachine)) {
          NetworkInterfaceCard nic = api.getNetworkInterfaceCardApi(group).get(nicName);
          Iterable<String> publicIps = getPublicIps(group, nic);
@@ -99,8 +110,11 @@ public class CleanupResources implements Function<String, Boolean> {
             api.getPublicIPAddressApi(group).delete(publicIp);
          }
       }
+   }
 
-      String storageAccountName = storageProfileToStorageAccountName.apply(virtualMachine.properties().storageProfile());
+   public void cleanupVirtualMachineStorage(String group, VirtualMachine virtualMachine) {
+      String storageAccountName = storageProfileToStorageAccountName
+            .apply(virtualMachine.properties().storageProfile());
       StorageServiceKeys keys = api.getStorageAccountApi(group).getKeys(storageAccountName);
 
       // Remove the virtual machine files
@@ -118,20 +132,44 @@ public class CleanupResources implements Function<String, Boolean> {
       } finally {
          closeQuietly(blobHelper);
       }
+   }
 
-      deleteResourceGroupIfEmpty(group);
+   public boolean cleanupSecurityGroupIfOrphaned(String resourceGroup, String group) {
+      String name = namingConvention.create().sharedNameForGroup(group);
+      NetworkSecurityGroupApi sgapi = api.getNetworkSecurityGroupApi(resourceGroup);
 
-      return vmDeleted;
+      boolean deleted = false;
+
+      try {
+         NetworkSecurityGroup securityGroup = sgapi.get(name);
+         if (securityGroup != null) {
+            List<NetworkInterfaceCard> nics = securityGroup.properties().networkInterfaces();
+            if (nics == null || nics.isEmpty()) {
+               logger.debug(">> deleting orphaned security group %s from %s...", name, resourceGroup);
+               try {
+                  deleted = resourceDeleted.apply(sgapi.delete(name));
+               } catch (Exception ex) {
+                  logger.warn(ex, ">> error deleting orphaned security group %s from %s...", name, resourceGroup);
+               }
+            }
+         }
+      } catch (Exception ex) {
+         logger.warn(ex, "Error deleting security groups for %s and group %s", resourceGroup, group);
+      }
+
+      return deleted;
    }
 
-   public void deleteResourceGroupIfEmpty(String group) {
-      if (api.getVirtualMachineApi(group).list().isEmpty() 
-            && api.getStorageAccountApi(group).list().isEmpty()
+   public boolean deleteResourceGroupIfEmpty(String group) {
+      boolean deleted = false;
+      if (api.getVirtualMachineApi(group).list().isEmpty() && api.getStorageAccountApi(group).list().isEmpty()
             && api.getNetworkInterfaceCardApi(group).list().isEmpty()
-            && api.getPublicIPAddressApi(group).list().isEmpty()) {
+            && api.getPublicIPAddressApi(group).list().isEmpty()
+            && api.getNetworkSecurityGroupApi(group).list().isEmpty()) {
          logger.debug(">> the resource group %s is empty. Deleting...", group);
-         resourceDeleted.apply(api.getResourceGroupApi().delete(group));
+         deleted = resourceDeleted.apply(api.getResourceGroupApi().delete(group));
       }
+      return deleted;
    }
 
    private Iterable<String> getPublicIps(String group, NetworkInterfaceCard nic) {

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/867ddef6/azurecompute-arm/src/test/java/org/jclouds/azurecompute/arm/compute/extensions/AzureComputeImageExtensionLiveTest.java
----------------------------------------------------------------------
diff --git a/azurecompute-arm/src/test/java/org/jclouds/azurecompute/arm/compute/extensions/AzureComputeImageExtensionLiveTest.java b/azurecompute-arm/src/test/java/org/jclouds/azurecompute/arm/compute/extensions/AzureComputeImageExtensionLiveTest.java
index c8a9259..8939eb1 100644
--- a/azurecompute-arm/src/test/java/org/jclouds/azurecompute/arm/compute/extensions/AzureComputeImageExtensionLiveTest.java
+++ b/azurecompute-arm/src/test/java/org/jclouds/azurecompute/arm/compute/extensions/AzureComputeImageExtensionLiveTest.java
@@ -71,4 +71,5 @@ public class AzureComputeImageExtensionLiveTest extends BaseImageExtensionLiveTe
             .overrideLoginPrivateKey(keyPair.get("private")));
    }
 
+
 }