Issue 757: refactor elastic ip auto-allocation approach and backfill related tests

This commit is contained in:
Adrian Cole 2011-12-19 01:46:08 -08:00
parent 4225b5b49b
commit bf99fb10a0
15 changed files with 731 additions and 133 deletions

View File

@ -44,8 +44,8 @@ import com.google.common.base.Splitter;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.base.Throwables;
import com.google.common.cache.LoadingCache;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableSet;
import com.google.inject.Injector;
import com.google.inject.Key;
@ -70,7 +70,7 @@ public class EC2ComputeServiceContextModule extends BaseComputeServiceContextMod
protected void installDependencies(){
install(new EC2ComputeServiceDependenciesModule());
}
@Override
protected boolean shouldParseImagesOnDemand(Injector injector) {
// If no owners to query, then will never lookup all images

View File

@ -45,6 +45,7 @@ import org.jclouds.ec2.compute.functions.AddElasticIpsToNodemetadata;
import org.jclouds.ec2.compute.functions.CreateSecurityGroupIfNeeded;
import org.jclouds.ec2.compute.functions.CreateUniqueKeyPair;
import org.jclouds.ec2.compute.functions.CredentialsForInstance;
import org.jclouds.ec2.compute.functions.LoadPublicIpForInstanceOrNull;
import org.jclouds.ec2.compute.functions.RegionAndIdToImage;
import org.jclouds.ec2.compute.functions.RunningInstanceToNodeMetadata;
import org.jclouds.ec2.compute.internal.EC2TemplateBuilderImpl;
@ -62,9 +63,9 @@ import com.google.common.base.Function;
import com.google.common.base.Functions;
import com.google.common.base.Predicate;
import com.google.common.base.Supplier;
import com.google.common.cache.LoadingCache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.inject.AbstractModule;
@ -72,6 +73,7 @@ import com.google.inject.Injector;
import com.google.inject.Provides;
import com.google.inject.Scopes;
import com.google.inject.TypeLiteral;
import com.google.inject.name.Names;
/**
*
@ -80,11 +82,11 @@ import com.google.inject.TypeLiteral;
public class EC2ComputeServiceDependenciesModule extends AbstractModule {
public static final Map<InstanceState, NodeState> instanceToNodeState = ImmutableMap
.<InstanceState, NodeState> builder().put(InstanceState.PENDING, NodeState.PENDING)
.put(InstanceState.RUNNING, NodeState.RUNNING).put(InstanceState.SHUTTING_DOWN, NodeState.PENDING)
.put(InstanceState.TERMINATED, NodeState.TERMINATED).put(InstanceState.STOPPING, NodeState.PENDING)
.put(InstanceState.STOPPED, NodeState.SUSPENDED).put(InstanceState.UNRECOGNIZED, NodeState.UNRECOGNIZED)
.build();
.<InstanceState, NodeState> builder().put(InstanceState.PENDING, NodeState.PENDING).put(
InstanceState.RUNNING, NodeState.RUNNING).put(InstanceState.SHUTTING_DOWN, NodeState.PENDING).put(
InstanceState.TERMINATED, NodeState.TERMINATED).put(InstanceState.STOPPING, NodeState.PENDING)
.put(InstanceState.STOPPED, NodeState.SUSPENDED).put(InstanceState.UNRECOGNIZED, NodeState.UNRECOGNIZED)
.build();
@Singleton
@Provides
@ -99,12 +101,14 @@ public class EC2ComputeServiceDependenciesModule extends AbstractModule {
bind(ComputeService.class).to(EC2ComputeService.class);
bind(new TypeLiteral<CacheLoader<RunningInstance, Credentials>>() {
}).to(CredentialsForInstance.class);
bind(new TypeLiteral<CacheLoader<RegionAndName, String>>() {
}).to(CreateSecurityGroupIfNeeded.class);
bind(new TypeLiteral<Function<RegionAndName, KeyPair>>() {
}).to(CreateUniqueKeyPair.class);
bind(new TypeLiteral<CacheLoader<RegionAndName, Image>>() {
}).to(RegionAndIdToImage.class);
bind(new TypeLiteral<CacheLoader<RegionAndName, String>>() {
}).annotatedWith(Names.named("SECURITY")).to(CreateSecurityGroupIfNeeded.class);
bind(new TypeLiteral<CacheLoader<RegionAndName, String>>() {
}).annotatedWith(Names.named("ELASTICIP")).to(LoadPublicIpForInstanceOrNull.class);
bind(new TypeLiteral<ComputeServiceContext>() {
}).to(new TypeLiteral<ComputeServiceContextImpl<EC2Client, EC2AsyncClient>>() {
}).in(Scopes.SINGLETON);
@ -113,6 +117,9 @@ public class EC2ComputeServiceDependenciesModule extends AbstractModule {
}).in(Scopes.SINGLETON);
}
/**
* only add the overhead of looking up ips when we have enabled the auto-allocate functionality
*/
@Provides
@Singleton
public Function<RunningInstance, NodeMetadata> bindNodeConverter(RunningInstanceToNodeMetadata baseConverter,
@ -148,20 +155,29 @@ public class EC2ComputeServiceDependenciesModule extends AbstractModule {
protected ConcurrentMap<RegionAndName, KeyPair> keypairMap(Injector i) {
return Maps.newConcurrentMap();
}
@Provides
@Singleton
@Named("SECURITY")
protected LoadingCache<RegionAndName, String> securityGroupMap(CacheLoader<RegionAndName, String> in) {
protected LoadingCache<RegionAndName, String> securityGroupMap(
@Named("SECURITY") CacheLoader<RegionAndName, String> in) {
return CacheBuilder.newBuilder().build(in);
}
@Provides
@Singleton
@Named("ELASTICIP")
protected LoadingCache<RegionAndName, String> instanceToElasticIp(
@Named("ELASTICIP") CacheLoader<RegionAndName, String> in) {
return CacheBuilder.newBuilder().build(in);
}
@Provides
@Singleton
@Named("SECURITY")
protected Predicate<RegionAndName> securityGroupEventualConsistencyDelay(SecurityGroupPresent in,
@Named(PROPERTY_EC2_TIMEOUT_SECURITYGROUP_PRESENT) long msDelay) {
@Named(PROPERTY_EC2_TIMEOUT_SECURITYGROUP_PRESENT) long msDelay) {
return new RetryablePredicate<RegionAndName>(in, msDelay, 100l, TimeUnit.MILLISECONDS);
}
}

View File

@ -18,23 +18,24 @@
*/
package org.jclouds.ec2.compute.functions;
import java.util.Set;
import static com.google.common.base.Preconditions.checkNotNull;
import java.util.concurrent.ExecutionException;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Singleton;
import org.jclouds.aws.util.AWSUtils;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeMetadataBuilder;
import org.jclouds.ec2.EC2Client;
import org.jclouds.ec2.domain.PublicIpInstanceIdPair;
import org.jclouds.ec2.compute.domain.RegionAndName;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Splitter;
import com.google.common.base.Throwables;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
/**
* This class searches for elastic ip addresses that are associated with the node, and adds them to
@ -45,61 +46,29 @@ import com.google.common.collect.Iterables;
@Singleton
public class AddElasticIpsToNodemetadata implements Function<NodeMetadata, NodeMetadata> {
private final EC2Client client;
private final LoadingCache<RegionAndName, String> cache;
@Inject
AddElasticIpsToNodemetadata(EC2Client client) {
this.client = client;
protected AddElasticIpsToNodemetadata(@Named("ELASTICIP") LoadingCache<RegionAndName, String> cache) {
this.cache = checkNotNull(cache, "cache");
}
//TODO: can there be multiple elastic ips on one instance?
@Override
public NodeMetadata apply(NodeMetadata arg0) {
String[] parts = AWSUtils.parseHandle(arg0.getId());
String region = parts[0];
String instanceId = parts[1];
Iterable<PublicIpInstanceIdPair> elasticIpsAssociatedWithNode = ipAddressPairsAssignedToInstance(region,
instanceId);
Set<String> publicIps = extractIpAddressFromPairs(elasticIpsAssociatedWithNode);
return addPublicIpsToNode(publicIps, arg0);
}
@VisibleForTesting
NodeMetadata addPublicIpsToNode(Set<String> publicIps, NodeMetadata arg0) {
if (publicIps.size() > 0)
arg0 = NodeMetadataBuilder.fromNodeMetadata(arg0).publicAddresses(
Iterables.concat(publicIps, arg0.getPublicAddresses())).build();
return arg0;
}
@VisibleForTesting
Set<String> extractIpAddressFromPairs(Iterable<PublicIpInstanceIdPair> elasticIpsAssociatedWithNode) {
Set<String> publicIps = ImmutableSet.copyOf(Iterables.transform(elasticIpsAssociatedWithNode,
new Function<PublicIpInstanceIdPair, String>() {
@Override
public String apply(PublicIpInstanceIdPair arg0) {
return arg0.getPublicIp();
}
}));
return publicIps;
}
@VisibleForTesting
Iterable<PublicIpInstanceIdPair> ipAddressPairsAssignedToInstance(String region, final String instanceId) {
Iterable<PublicIpInstanceIdPair> elasticIpsAssociatedWithNode = Iterables.filter(client
.getElasticIPAddressServices().describeAddressesInRegion(region),
new Predicate<PublicIpInstanceIdPair>() {
@Override
public boolean apply(PublicIpInstanceIdPair in) {
return instanceId.equals(in.getInstanceId());
}
});
return elasticIpsAssociatedWithNode;
try {
String publicIp = cache.get(new RegionAndName(region, instanceId));
return NodeMetadataBuilder.fromNodeMetadata(arg0).publicAddresses(
ImmutableSet.<String> builder().addAll(arg0.getPublicAddresses()).add(publicIp).build()).build();
} catch (CacheLoader.InvalidCacheLoadException e) {
// no ip was found
return arg0;
} catch (ExecutionException e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -1,14 +1,38 @@
/**
* Licensed to jclouds, Inc. (jclouds) under one or more
* contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. jclouds 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.ec2.compute.functions;
import static com.google.common.collect.Maps.uniqueIndex;
import java.util.Map;
import javax.inject.Singleton;
import org.jclouds.compute.domain.Image;
import org.jclouds.ec2.compute.domain.RegionAndName;
import com.google.common.base.Function;
/**
* @author Adrian Cole
*/
@Singleton
public class ImagesToRegionAndIdMap implements Function<Iterable<? extends Image>, Map<RegionAndName, ? extends Image>> {
public static Map<RegionAndName, ? extends Image> imagesToMap(Iterable<? extends Image> input) {

View File

@ -0,0 +1,62 @@
/**
* Licensed to jclouds, Inc. (jclouds) under one or more
* contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. jclouds 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.ec2.compute.functions;
import java.util.NoSuchElementException;
import javax.inject.Inject;
import javax.inject.Singleton;
import org.jclouds.ec2.EC2Client;
import org.jclouds.ec2.compute.domain.RegionAndName;
import org.jclouds.ec2.domain.PublicIpInstanceIdPair;
import com.google.common.base.Predicate;
import com.google.common.cache.CacheLoader;
import com.google.common.collect.Iterables;
/**
* @author Adrian Cole
*/
@Singleton
public class LoadPublicIpForInstanceOrNull extends CacheLoader<RegionAndName, String> {
private final EC2Client client;
@Inject
public LoadPublicIpForInstanceOrNull(EC2Client client) {
this.client = client;
}
@Override
public String load(final RegionAndName key) throws Exception {
try {
return Iterables.find(client.getElasticIPAddressServices().describeAddressesInRegion(key.getRegion()),
new Predicate<PublicIpInstanceIdPair>() {
@Override
public boolean apply(PublicIpInstanceIdPair input) {
return key.getName().equals(input.getInstanceId());
}
}).getPublicIp();
} catch (NoSuchElementException e) {
return null;
}
}
}

View File

@ -53,10 +53,12 @@ import org.jclouds.logging.Logger;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Predicate;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Multimap;
import com.google.common.collect.ImmutableSet.Builder;
/**
* creates futures that correlate to
@ -72,11 +74,16 @@ public class EC2CreateNodesInGroupThenAddToSet implements CreateNodesInGroupThen
@Inject
@Named(EC2Constants.PROPERTY_EC2_AUTO_ALLOCATE_ELASTIC_IPS)
@VisibleForTesting
boolean autoAllocateElasticIps = false;
@VisibleForTesting
final EC2Client client;
@VisibleForTesting
final Predicate<NodeMetadata> nodeRunning;
@VisibleForTesting
final LoadingCache<RegionAndName, String> elasticIpCache;
@VisibleForTesting
final CreateKeyPairAndSecurityGroupsAsNeededAndReturnRunOptions createKeyPairAndSecurityGroupsAsNeededAndReturncustomize;
@VisibleForTesting
final Function<RunningInstance, NodeMetadata> runningInstanceToNodeMetadata;
@ -87,15 +94,22 @@ public class EC2CreateNodesInGroupThenAddToSet implements CreateNodesInGroupThen
final Map<String, Credentials> credentialStore;
final Provider<TemplateBuilder> templateBuilderProvider;
@Inject
protected EC2CreateNodesInGroupThenAddToSet(
EC2Client client,
@Named("ELASTICIP")
LoadingCache<RegionAndName, String> elasticIpCache,
@Named("NODE_RUNNING") Predicate<NodeMetadata> nodeRunning,
Provider<TemplateBuilder> templateBuilderProvider,
CreateKeyPairAndSecurityGroupsAsNeededAndReturnRunOptions createKeyPairAndSecurityGroupsAsNeededAndReturncustomize,
InstancePresent instancePresent, Function<RunningInstance, NodeMetadata> runningInstanceToNodeMetadata,
LoadingCache<RunningInstance, Credentials> instanceToCredentials, Map<String, Credentials> credentialStore,
ComputeUtils utils) {
this.client = checkNotNull(client, "client");
this.elasticIpCache = checkNotNull(elasticIpCache, "elasticIpCache");
this.nodeRunning = checkNotNull(nodeRunning, "nodeRunning");
this.templateBuilderProvider = checkNotNull(templateBuilderProvider, "templateBuilderProvider");
this.instancePresent = checkNotNull(instancePresent, "instancePresent");
this.createKeyPairAndSecurityGroupsAsNeededAndReturncustomize = checkNotNull(
@ -135,7 +149,7 @@ public class EC2CreateNodesInGroupThenAddToSet implements CreateNodesInGroupThen
populateCredentials(started);
}
assignElasticIpsToInstances(count, started, ips, template);
assignElasticIpsToInstances(ips, started);
return utils.customizeNodesAndAddToGoodMapOrPutExceptionIntoBadMap(template.getOptions(), transform(started,
runningInstanceToNodeMetadata), goodNodes, badNodes, customizationResponses);
@ -153,39 +167,46 @@ public class EC2CreateNodesInGroupThenAddToSet implements CreateNodesInGroupThen
credentialStore.put("node#" + instance.getRegion() + "/" + instance.getId(), credentials);
}
// TODO write test for this
protected Iterable<String> allocateElasticIpsInRegion(int count, Template template) {
Iterable<String> ips = ImmutableSet.<String> of();
Builder<String> ips = ImmutableSet.<String> builder();
if (!autoAllocateElasticIps)
return ips;
return ips.build();
String region = AWSUtils.getRegionFromLocationOrNull(template.getLocation());
logger.debug("<< allocating elastic IPs for nodes in region (%s)", region);
logger.debug("<< allocating %d elastic IPs for nodes in region (%s)", count, region);
for (int i=0; i<count; ++i) {
ips = Iterables.concat(ips, ImmutableSet.<String> of(
client.getElasticIPAddressServices().allocateAddressInRegion(region)));
for (int i = 0; i < count; ++i) {
ips.add(client.getElasticIPAddressServices().allocateAddressInRegion(region));
}
return ips;
return ips.build();
}
// TODO write test for this
protected void assignElasticIpsToInstances(int count, Iterable<? extends RunningInstance> started,
Iterable<String> ips, Template template) {
protected void assignElasticIpsToInstances(Iterable<String> ips, Iterable<? extends RunningInstance> startedInstances) {
if (!autoAllocateElasticIps)
return;
String region = AWSUtils.getRegionFromLocationOrNull(template.getLocation());
for (int i=0; i<count; ++i) {
// TODO parallel
int i = 0;
for (RunningInstance startedInstance : startedInstances) {
String ip = Iterables.get(ips, i);
String id = Iterables.get(started, i).getId();
String region = startedInstance.getRegion();
String id = startedInstance.getId();
RegionAndName coordinates = new RegionAndName(region, id);
// block until instance is running
logger.debug(">> awaiting status running instance(%s)", coordinates);
nodeRunning.apply(runningInstanceToNodeMetadata.apply(startedInstance));
logger.debug("<< running instance(%s)", coordinates);
logger.debug(">> associating elastic IP %s to instance %s", ip, coordinates);
client.getElasticIPAddressServices().associateAddressInRegion(region, ip, id);
logger.trace("<< associated elastic IP %s to instance %s", ip, coordinates);
// add mapping of instance to ip into the cache
elasticIpCache.put(coordinates, ip);
}
}
// TODO write test for this
protected Iterable<? extends RunningInstance> createKeyPairAndSecurityGroupsAsNeededThenRunInstances(String group,
int count, Template template) {
String region = AWSUtils.getRegionFromLocationOrNull(template.getLocation());

View File

@ -20,8 +20,7 @@ package org.jclouds.ec2.compute.strategy;
import static com.google.common.base.Preconditions.checkNotNull;
import java.util.Iterator;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import javax.annotation.Resource;
import javax.inject.Inject;
@ -34,9 +33,14 @@ import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.strategy.DestroyNodeStrategy;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.ec2.EC2Client;
import org.jclouds.ec2.compute.domain.RegionAndName;
import org.jclouds.ec2.reference.EC2Constants;
import org.jclouds.logging.Logger;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
/**
*
* @author Adrian Cole
@ -48,14 +52,19 @@ public class EC2DestroyNodeStrategy implements DestroyNodeStrategy {
protected Logger logger = Logger.NULL;
protected final EC2Client client;
protected final GetNodeMetadataStrategy getNode;
protected final LoadingCache<RegionAndName, String> elasticIpCache;
@Inject
@Named(EC2Constants.PROPERTY_EC2_AUTO_ALLOCATE_ELASTIC_IPS)
@VisibleForTesting
boolean autoAllocateElasticIps = false;
@Inject
protected EC2DestroyNodeStrategy(EC2Client client, GetNodeMetadataStrategy getNode) {
protected EC2DestroyNodeStrategy(EC2Client client, GetNodeMetadataStrategy getNode,
@Named("ELASTICIP") LoadingCache<RegionAndName, String> elasticIpCache) {
this.client = checkNotNull(client, "client");
this.getNode = checkNotNull(getNode, "getNode");
this.elasticIpCache = checkNotNull(elasticIpCache, "elasticIpCache");
}
@Override
@ -63,25 +72,36 @@ public class EC2DestroyNodeStrategy implements DestroyNodeStrategy {
String[] parts = AWSUtils.parseHandle(id);
String region = parts[0];
String instanceId = parts[1];
Set<String> publicIps = getNode.getNode(id).getPublicAddresses();
releaseElasticIpInRegion(region, instanceId, publicIps);
destroyInstanceInRegion(region, instanceId);
// TODO: can there be multiple?
releaseAnyPublicIpForInstanceInRegion(instanceId, region);
destroyInstanceInRegion(instanceId, region);
return getNode.getNode(id);
}
protected void releaseElasticIpInRegion(String region, String instanceId, Set<String> publicIps) {
if (!autoAllocateElasticIps)
return;
protected void releaseAnyPublicIpForInstanceInRegion(String instanceId, String region) {
if (!autoAllocateElasticIps)
return;
try {
String ip = elasticIpCache.get(new RegionAndName(region, instanceId));
logger.debug(">> disassociating elastic IP %s", ip);
client.getElasticIPAddressServices().disassociateAddressInRegion(region, ip);
logger.trace("<< disassociated elastic IP %s", ip);
elasticIpCache.invalidate(new RegionAndName(region, instanceId));
logger.debug(">> releasing elastic IP %s", ip);
client.getElasticIPAddressServices().releaseAddressInRegion(region, ip);
logger.trace("<< released elastic IP %s", ip);
} catch (CacheLoader.InvalidCacheLoadException e) {
// no ip was found
return;
} catch (ExecutionException e) {
// don't propagate as we need to clean up the node regardless
logger.warn(e, "error cleaning up elastic ip for instance %s/%s", region, instanceId);
}
Iterator<String> it = publicIps.iterator();
while (it.hasNext()) {
String publicIp = (String)it.next();
client.getElasticIPAddressServices().disassociateAddressInRegion(region, publicIp);
client.getElasticIPAddressServices().releaseAddressInRegion(region, publicIp);
}
}
protected void destroyInstanceInRegion(String region, String instanceId) {
protected void destroyInstanceInRegion(String instanceId, String region) {
client.getInstanceServices().terminateInstancesInRegion(region, instanceId);
}
}

View File

@ -19,10 +19,9 @@
package org.jclouds.ec2.compute;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;
import java.io.IOException;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
@ -30,7 +29,6 @@ import java.util.Set;
import org.jclouds.compute.BaseComputeServiceLiveTest;
import org.jclouds.compute.ComputeServiceContext;
import org.jclouds.compute.ComputeServiceContextFactory;
import org.jclouds.compute.RunNodesException;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.OsFamily;
import org.jclouds.compute.domain.Template;
@ -181,7 +179,6 @@ public class EC2ComputeServiceLiveTest extends BaseComputeServiceLiveTest {
public void testAutoIpAllocation() throws Exception {
ComputeServiceContext context = null;
String group = this.group + "aip";
try {
Properties overrides = setupProperties();
overrides.setProperty(EC2Constants.PROPERTY_EC2_AUTO_ALLOCATE_ELASTIC_IPS, "true");
@ -211,7 +208,7 @@ public class EC2ComputeServiceLiveTest extends BaseComputeServiceLiveTest {
EC2Client ec2 = EC2Client.class.cast(context.getProviderSpecificContext().getApi());
Set<PublicIpInstanceIdPair> ipidpairs =
ec2.getElasticIPAddressServices().describeAddressesInRegion(region, publicIps.toArray(new String[0]));
assertTrue(ipidpairs.size() == 1);
assert (ipidpairs.size() == 1) : ipidpairs;
// check that the elastic ip is in node.publicAddresses
PublicIpInstanceIdPair ipidpair = Iterables.get(ipidpairs, 0);

View File

@ -0,0 +1,92 @@
/**
* Licensed to jclouds, Inc. (jclouds) under one or more
* contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. jclouds 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.ec2.compute.functions;
import static org.testng.Assert.assertEquals;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeMetadataBuilder;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.ec2.compute.domain.RegionAndName;
import org.testng.annotations.Test;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
/**
* @author Adrian Cole
*/
@Test(groups = "unit", singleThreaded = true, testName = "AddElasticIpsToNodemetadataTest")
public class AddElasticIpsToNodemetadataTest {
NodeMetadata node = new NodeMetadataBuilder().state(NodeState.RUNNING).group("zkclustertest").name("foo").hostname(
"ip-10-212-81-7").privateAddresses(ImmutableSet.of("10.212.81.7")).publicAddresses(
ImmutableSet.of("174.129.173.155")).imageId("us-east-1/ami-63be790a").id("us-east-1/i-911444f0")
.providerId("i-911444f0").tags(ImmutableSet.of("Empty")).userMetadata(ImmutableMap.of("Name", "foo"))
.build();
@Test
public void testReturnsNodeWithExtraIpWhenFoundInCacheAndNodeHadAPublicIp() throws Exception {
RegionAndName key = new RegionAndName("us-east-1", node.getProviderId());
String val = "1.1.1.1";
LoadingCache<RegionAndName, String> cache = cacheOf(key, val);
AddElasticIpsToNodemetadata fn = new AddElasticIpsToNodemetadata(cache);
assertEquals(fn.apply(node).getPublicAddresses(), ImmutableSet.of("174.129.173.155", "1.1.1.1"));
}
@Test
public void testReturnsNodeWithIpWhenFoundInCacheAndNodeHadNoPublicIp() throws Exception {
RegionAndName key = new RegionAndName("us-east-1", node.getProviderId());
String val = "1.1.1.1";
LoadingCache<RegionAndName, String> cache = cacheOf(key, val);
AddElasticIpsToNodemetadata fn = new AddElasticIpsToNodemetadata(cache);
assertEquals(fn.apply(
NodeMetadataBuilder.fromNodeMetadata(node).publicAddresses(ImmutableSet.<String> of()).build())
.getPublicAddresses(), ImmutableSet.of("1.1.1.1"));
}
@Test
public void testReturnsSameNodeWhenNotFoundInCache() throws Exception {
RegionAndName key = new RegionAndName("us-east-1", node.getProviderId());
String val = null;
LoadingCache<RegionAndName, String> cache = cacheOf(key, val);
AddElasticIpsToNodemetadata fn = new AddElasticIpsToNodemetadata(cache);
assertEquals(fn.apply(node).getPublicAddresses(), ImmutableSet.of("174.129.173.155"));
}
private LoadingCache<RegionAndName, String> cacheOf(final RegionAndName key, final String val) {
return CacheBuilder.newBuilder().build(new CacheLoader<RegionAndName, String>() {
@Override
public String load(RegionAndName in) throws Exception {
return key.equals(in) ? val : null;
}
});
}
}

View File

@ -0,0 +1,107 @@
/**
* Licensed to jclouds, Inc. (jclouds) under one or more
* contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. jclouds 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.ec2.compute.functions;
import static org.easymock.EasyMock.expect;
import static org.easymock.classextension.EasyMock.createMock;
import static org.easymock.classextension.EasyMock.replay;
import static org.easymock.classextension.EasyMock.verify;
import static org.testng.Assert.assertEquals;
import org.jclouds.ec2.EC2Client;
import org.jclouds.ec2.compute.domain.RegionAndName;
import org.jclouds.ec2.domain.PublicIpInstanceIdPair;
import org.jclouds.ec2.services.ElasticIPAddressClient;
import org.testng.annotations.Test;
import com.google.common.collect.ImmutableSet;
/**
* @author Adrian Cole
*/
@Test(groups = "unit", singleThreaded = true, testName = "LoadPublicIpForInstanceOrNullTest")
public class LoadPublicIpForInstanceOrNullTest {
@Test
public void testReturnsPublicIpOnMatch() throws Exception {
EC2Client client = createMock(EC2Client.class);
ElasticIPAddressClient ipClient = createMock(ElasticIPAddressClient.class);
expect(client.getElasticIPAddressServices()).andReturn(ipClient).atLeastOnce();
expect(ipClient.describeAddressesInRegion("region")).andReturn(
ImmutableSet.<PublicIpInstanceIdPair> of(new PublicIpInstanceIdPair("region", "1.1.1.1", "i-blah")))
.atLeastOnce();
replay(client);
replay(ipClient);
LoadPublicIpForInstanceOrNull parser = new LoadPublicIpForInstanceOrNull(client);
assertEquals(parser.load(new RegionAndName("region", "i-blah")), "1.1.1.1");
verify(client);
verify(ipClient);
}
@Test
public void testReturnsNullWhenNotFound() throws Exception {
EC2Client client = createMock(EC2Client.class);
ElasticIPAddressClient ipClient = createMock(ElasticIPAddressClient.class);
expect(client.getElasticIPAddressServices()).andReturn(ipClient).atLeastOnce();
expect(ipClient.describeAddressesInRegion("region")).andReturn(ImmutableSet.<PublicIpInstanceIdPair> of())
.atLeastOnce();
replay(client);
replay(ipClient);
LoadPublicIpForInstanceOrNull parser = new LoadPublicIpForInstanceOrNull(client);
assertEquals(parser.load(new RegionAndName("region", "i-blah")), null);
verify(client);
verify(ipClient);
}
@Test
public void testReturnsNullWhenNotAssigned() throws Exception {
EC2Client client = createMock(EC2Client.class);
ElasticIPAddressClient ipClient = createMock(ElasticIPAddressClient.class);
expect(client.getElasticIPAddressServices()).andReturn(ipClient).atLeastOnce();
expect(ipClient.describeAddressesInRegion("region")).andReturn(
ImmutableSet.<PublicIpInstanceIdPair> of(new PublicIpInstanceIdPair("region", "1.1.1.1", null)))
.atLeastOnce();
replay(client);
replay(ipClient);
LoadPublicIpForInstanceOrNull parser = new LoadPublicIpForInstanceOrNull(client);
assertEquals(parser.load(new RegionAndName("region", "i-blah")), null);
verify(client);
verify(ipClient);
}
}

View File

@ -48,9 +48,11 @@ import org.jclouds.ec2.compute.predicates.InstancePresent;
import org.jclouds.ec2.domain.Reservation;
import org.jclouds.ec2.domain.RunningInstance;
import org.jclouds.ec2.options.RunInstancesOptions;
import org.jclouds.ec2.services.ElasticIPAddressClient;
import org.jclouds.ec2.services.InstanceClient;
import org.testng.annotations.Test;
import com.google.common.base.Predicate;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
@ -60,9 +62,93 @@ import com.google.inject.util.Providers;
/**
* @author Adrian Cole
*/
@Test(groups = "unit")
@Test(groups = "unit", singleThreaded = true, testName = "EC2CreateNodesInGroupThenAddToSetTest")
public class EC2CreateNodesInGroupThenAddToSetTest {
@SuppressWarnings("unchecked")
public void testIpAllocationThenAfterNodeRunningAssignThenUpdateCache() {
Location location = ZONE_AP_SOUTHEAST_1A;
String region = "ap-southeast-1";
String zone = "ap-southeast-1a";
String imageId = "ami1";
String instanceCreatedId = "instance1";
// setup mocks
TemplateBuilder templateBuilder = createMock(TemplateBuilder.class);
EC2CreateNodesInGroupThenAddToSet strategy = setupStrategy(templateBuilder);
InputParams input = new InputParams(location);
InstanceClient instanceClient = createMock(InstanceClient.class);
ElasticIPAddressClient ipClient = createMock(ElasticIPAddressClient.class);
RunInstancesOptions ec2Options = createMock(RunInstancesOptions.class);
RunningInstance instance = createMock(RunningInstance.class);
Reservation<? extends RunningInstance> reservation = new Reservation<RunningInstance>(region, ImmutableSet
.<String> of(), ImmutableSet.<RunningInstance> of(instance), "ownerId", "requesterId", "reservationId");
NodeMetadata nodeMetadata = createMock(NodeMetadata.class);
// enable auto-allocation
strategy.autoAllocateElasticIps = true;
// setup expectations
expect(templateBuilder.fromTemplate(input.template)).andReturn(templateBuilder);
expect(templateBuilder.build()).andReturn(input.template);
expect(strategy.client.getInstanceServices()).andReturn(instanceClient).atLeastOnce();
expect(
strategy.createKeyPairAndSecurityGroupsAsNeededAndReturncustomize.execute(region, input.tag,
input.template)).andReturn(ec2Options);
expect(strategy.client.getElasticIPAddressServices()).andReturn(ipClient).atLeastOnce();
expect(input.template.getLocation()).andReturn(input.location).atLeastOnce();
expect(input.template.getImage()).andReturn(input.image).atLeastOnce();
expect(input.image.getProviderId()).andReturn(imageId).atLeastOnce();
// differences when ip allocation
expect(ipClient.allocateAddressInRegion(region)).andReturn("1.1.1.1");
expect(strategy.runningInstanceToNodeMetadata.apply(instance)).andReturn(nodeMetadata).atLeastOnce();
expect(strategy.nodeRunning.apply(nodeMetadata)).andReturn(true);
ipClient.associateAddressInRegion(region, "1.1.1.1", instanceCreatedId);
strategy.elasticIpCache.put(new RegionAndName(region, instanceCreatedId), "1.1.1.1");
expect(instanceClient.runInstancesInRegion(region, zone, imageId, 1, input.count, ec2Options)).andReturn(
Reservation.class.cast(reservation));
expect(instance.getId()).andReturn(instanceCreatedId).atLeastOnce();
// simulate a lazy credentials fetch
Credentials creds = new Credentials("foo", "bar");
expect(strategy.instanceToCredentials.apply(instance)).andReturn(creds);
expect(instance.getRegion()).andReturn(region).atLeastOnce();
expect(strategy.credentialStore.put("node#" + region + "/" + instanceCreatedId, creds)).andReturn(null);
expect(strategy.instancePresent.apply(new RegionAndName(region, instanceCreatedId))).andReturn(true);
expect(input.template.getOptions()).andReturn(input.options).atLeastOnce();
expect(
strategy.utils.customizeNodesAndAddToGoodMapOrPutExceptionIntoBadMap(eq(input.options),
containsNodeMetadata(nodeMetadata), eq(input.nodes), eq(input.badNodes),
eq(input.customization))).andReturn(null);
// replay mocks
replay(templateBuilder);
replay(instanceClient);
replay(ipClient);
replay(ec2Options);
replay(instance);
replay(nodeMetadata);
input.replayMe();
replayStrategy(strategy);
// run
strategy.execute(input.tag, input.count, input.template, input.nodes, input.badNodes, input.customization);
// verify mocks
verify(templateBuilder);
verify(instanceClient);
verify(ipClient);
verify(ec2Options);
verify(instance);
verify(nodeMetadata);
input.verifyMe();
verifyStrategy(strategy);
}
@Test
public void testZoneAsALocation() {
assertRegionAndZoneForLocation(ZONE_AP_SOUTHEAST_1A, "ap-southeast-1", "ap-southeast-1a");
@ -105,9 +191,8 @@ public class EC2CreateNodesInGroupThenAddToSetTest {
InstanceClient instanceClient = createMock(InstanceClient.class);
RunInstancesOptions ec2Options = createMock(RunInstancesOptions.class);
RunningInstance instance = createMock(RunningInstance.class);
Reservation<? extends RunningInstance> reservation = new Reservation<RunningInstance>(region,
ImmutableSet.<String> of(), ImmutableSet.<RunningInstance> of(instance), "ownerId", "requesterId",
"reservationId");
Reservation<? extends RunningInstance> reservation = new Reservation<RunningInstance>(region, ImmutableSet
.<String> of(), ImmutableSet.<RunningInstance> of(instance), "ownerId", "requesterId", "reservationId");
NodeMetadata nodeMetadata = createMock(NodeMetadata.class);
// setup expectations
@ -115,13 +200,13 @@ public class EC2CreateNodesInGroupThenAddToSetTest {
expect(templateBuilder.build()).andReturn(input.template);
expect(strategy.client.getInstanceServices()).andReturn(instanceClient).atLeastOnce();
expect(
strategy.createKeyPairAndSecurityGroupsAsNeededAndReturncustomize
.execute(region, input.tag, input.template)).andReturn(ec2Options);
strategy.createKeyPairAndSecurityGroupsAsNeededAndReturncustomize.execute(region, input.tag,
input.template)).andReturn(ec2Options);
expect(input.template.getLocation()).andReturn(input.location).atLeastOnce();
expect(input.template.getImage()).andReturn(input.image).atLeastOnce();
expect(input.image.getProviderId()).andReturn(imageId).atLeastOnce();
expect(instanceClient.runInstancesInRegion(region, zone, imageId, 1, input.count, ec2Options)).andReturn(
Reservation.class.cast(reservation));
Reservation.class.cast(reservation));
expect(instance.getId()).andReturn(instanceCreatedId).atLeastOnce();
// simulate a lazy credentials fetch
Credentials creds = new Credentials("foo", "bar");
@ -134,9 +219,9 @@ public class EC2CreateNodesInGroupThenAddToSetTest {
expect(strategy.runningInstanceToNodeMetadata.apply(instance)).andReturn(nodeMetadata);
expect(
strategy.utils.customizeNodesAndAddToGoodMapOrPutExceptionIntoBadMap(eq(input.options),
containsNodeMetadata(nodeMetadata), eq(input.nodes), eq(input.badNodes), eq(input.customization)))
.andReturn(null);
strategy.utils.customizeNodesAndAddToGoodMapOrPutExceptionIntoBadMap(eq(input.options),
containsNodeMetadata(nodeMetadata), eq(input.nodes), eq(input.badNodes),
eq(input.customization))).andReturn(null);
// replay mocks
replay(templateBuilder);
@ -160,12 +245,11 @@ public class EC2CreateNodesInGroupThenAddToSetTest {
verifyStrategy(strategy);
}
private static final Location REGION_AP_SOUTHEAST_1 = new LocationBuilder().scope(LocationScope.REGION)
.id("ap-southeast-1").description("ap-southeast-1")
.parent(new LocationBuilder().scope(LocationScope.PROVIDER).id("aws-ec2").description("aws-ec2").build())
.build();
private static final Location ZONE_AP_SOUTHEAST_1A = new LocationBuilder().scope(LocationScope.ZONE)
.id("ap-southeast-1a").description("ap-southeast-1a").parent(REGION_AP_SOUTHEAST_1).build();
private static final Location REGION_AP_SOUTHEAST_1 = new LocationBuilder().scope(LocationScope.REGION).id(
"ap-southeast-1").description("ap-southeast-1").parent(
new LocationBuilder().scope(LocationScope.PROVIDER).id("aws-ec2").description("aws-ec2").build()).build();
private static final Location ZONE_AP_SOUTHEAST_1A = new LocationBuilder().scope(LocationScope.ZONE).id(
"ap-southeast-1a").description("ap-southeast-1a").parent(REGION_AP_SOUTHEAST_1).build();
// /////////////////////////////////////////////////////////////////////
@SuppressWarnings("unchecked")
@ -209,6 +293,8 @@ public class EC2CreateNodesInGroupThenAddToSetTest {
private void verifyStrategy(EC2CreateNodesInGroupThenAddToSet strategy) {
verify(strategy.createKeyPairAndSecurityGroupsAsNeededAndReturncustomize);
verify(strategy.client);
verify(strategy.nodeRunning);
verify(strategy.elasticIpCache);
verify(strategy.instancePresent);
verify(strategy.runningInstanceToNodeMetadata);
verify(strategy.instanceToCredentials);
@ -223,16 +309,20 @@ public class EC2CreateNodesInGroupThenAddToSetTest {
InstancePresent instancePresent = createMock(InstancePresent.class);
RunningInstanceToNodeMetadata runningInstanceToNodeMetadata = createMock(RunningInstanceToNodeMetadata.class);
LoadingCache<RunningInstance, Credentials> instanceToCredentials = createMock(LoadingCache.class);
LoadingCache<RegionAndName, String> elasticIpCache = createMock(LoadingCache.class);
Predicate<NodeMetadata> nodeRunning = createMock(Predicate.class);
Map<String, Credentials> credentialStore = createMock(Map.class);
ComputeUtils utils = createMock(ComputeUtils.class);
return new EC2CreateNodesInGroupThenAddToSet(client, Providers.<TemplateBuilder> of(template),
createKeyPairAndSecurityGroupsAsNeededAndReturncustomize, instancePresent,
runningInstanceToNodeMetadata, instanceToCredentials, credentialStore, utils);
return new EC2CreateNodesInGroupThenAddToSet(client, elasticIpCache, nodeRunning, Providers
.<TemplateBuilder> of(template), createKeyPairAndSecurityGroupsAsNeededAndReturncustomize,
instancePresent, runningInstanceToNodeMetadata, instanceToCredentials, credentialStore, utils);
}
private void replayStrategy(EC2CreateNodesInGroupThenAddToSet strategy) {
replay(strategy.createKeyPairAndSecurityGroupsAsNeededAndReturncustomize);
replay(strategy.client);
replay(strategy.elasticIpCache);
replay(strategy.nodeRunning);
replay(strategy.instancePresent);
replay(strategy.runningInstanceToNodeMetadata);
replay(strategy.instanceToCredentials);

View File

@ -0,0 +1,190 @@
/**
* Licensed to jclouds, Inc. (jclouds) under one or more
* contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. jclouds 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.ec2.compute.strategy;
import static org.easymock.EasyMock.expect;
import static org.easymock.classextension.EasyMock.createMock;
import static org.easymock.classextension.EasyMock.replay;
import static org.easymock.classextension.EasyMock.verify;
import static org.testng.Assert.assertEquals;
import java.util.concurrent.ExecutionException;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.ec2.EC2Client;
import org.jclouds.ec2.compute.domain.RegionAndName;
import org.jclouds.ec2.services.ElasticIPAddressClient;
import org.jclouds.ec2.services.InstanceClient;
import org.testng.annotations.Test;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
/**
* @author Adrian Cole
*/
@Test(groups = "unit", singleThreaded = true, testName = "EC2DestroyNodeStrategyTest")
public class EC2DestroyNodeStrategyTest {
@SuppressWarnings("unchecked")
@Test
public void testDestroyNodeTerminatesInstanceAndReturnsRefreshedNode() throws Exception {
EC2Client client = createMock(EC2Client.class);
InstanceClient instanceClient = createMock(InstanceClient.class);
GetNodeMetadataStrategy getNode = createMock(GetNodeMetadataStrategy.class);
LoadingCache<RegionAndName, String> elasticIpCache = createMock(LoadingCache.class);
NodeMetadata node = createMock(NodeMetadata.class);
expect(client.getInstanceServices()).andReturn(instanceClient).atLeastOnce();
expect(instanceClient.terminateInstancesInRegion("region", "i-blah")).andReturn(null);
expect(getNode.getNode("region/i-blah")).andReturn(node);
replay(client);
replay(getNode);
replay(instanceClient);
replay(elasticIpCache);
EC2DestroyNodeStrategy destroyer = new EC2DestroyNodeStrategy(client, getNode, elasticIpCache);
assertEquals(destroyer.destroyNode("region/i-blah"), node);
verify(client);
verify(getNode);
verify(instanceClient);
verify(elasticIpCache);
}
@SuppressWarnings("unchecked")
@Test
public void testDestroyNodeDisassociatesAndReleasesIpThenTerminatesInstanceAndReturnsRefreshedNode()
throws Exception {
EC2Client client = createMock(EC2Client.class);
GetNodeMetadataStrategy getNode = createMock(GetNodeMetadataStrategy.class);
LoadingCache<RegionAndName, String> elasticIpCache = createMock(LoadingCache.class);
ElasticIPAddressClient ipClient = createMock(ElasticIPAddressClient.class);
InstanceClient instanceClient = createMock(InstanceClient.class);
NodeMetadata node = createMock(NodeMetadata.class);
expect(elasticIpCache.get(new RegionAndName("region", "i-blah"))).andReturn("1.1.1.1");
expect(client.getElasticIPAddressServices()).andReturn(ipClient).atLeastOnce();
ipClient.disassociateAddressInRegion("region", "1.1.1.1");
ipClient.releaseAddressInRegion("region", "1.1.1.1");
elasticIpCache.invalidate(new RegionAndName("region", "i-blah"));
expect(client.getInstanceServices()).andReturn(instanceClient).atLeastOnce();
expect(instanceClient.terminateInstancesInRegion("region", "i-blah")).andReturn(null);
expect(getNode.getNode("region/i-blah")).andReturn(node);
replay(client);
replay(getNode);
replay(elasticIpCache);
replay(instanceClient);
replay(ipClient);
EC2DestroyNodeStrategy destroyer = new EC2DestroyNodeStrategy(client, getNode, elasticIpCache);
destroyer.autoAllocateElasticIps = true;
assertEquals(destroyer.destroyNode("region/i-blah"), node);
verify(client);
verify(getNode);
verify(elasticIpCache);
verify(instanceClient);
verify(ipClient);
}
@SuppressWarnings("unchecked")
@Test
public void testDestroyNodeSafeOnCacheMissThenTerminatesInstanceAndReturnsRefreshedNode()
throws Exception {
EC2Client client = createMock(EC2Client.class);
GetNodeMetadataStrategy getNode = createMock(GetNodeMetadataStrategy.class);
LoadingCache<RegionAndName, String> elasticIpCache = createMock(LoadingCache.class);
ElasticIPAddressClient ipClient = createMock(ElasticIPAddressClient.class);
InstanceClient instanceClient = createMock(InstanceClient.class);
NodeMetadata node = createMock(NodeMetadata.class);
expect(elasticIpCache.get(new RegionAndName("region", "i-blah"))).andThrow(new CacheLoader.InvalidCacheLoadException(null));
expect(client.getInstanceServices()).andReturn(instanceClient).atLeastOnce();
expect(instanceClient.terminateInstancesInRegion("region", "i-blah")).andReturn(null);
expect(getNode.getNode("region/i-blah")).andReturn(node);
replay(client);
replay(getNode);
replay(elasticIpCache);
replay(instanceClient);
replay(ipClient);
EC2DestroyNodeStrategy destroyer = new EC2DestroyNodeStrategy(client, getNode, elasticIpCache);
destroyer.autoAllocateElasticIps = true;
assertEquals(destroyer.destroyNode("region/i-blah"), node);
verify(client);
verify(getNode);
verify(elasticIpCache);
verify(instanceClient);
verify(ipClient);
}
@SuppressWarnings("unchecked")
@Test
public void testDestroyNodeSafeOnCacheExecutionExceptionThenTerminatesInstanceAndReturnsRefreshedNode()
throws Exception {
EC2Client client = createMock(EC2Client.class);
GetNodeMetadataStrategy getNode = createMock(GetNodeMetadataStrategy.class);
LoadingCache<RegionAndName, String> elasticIpCache = createMock(LoadingCache.class);
ElasticIPAddressClient ipClient = createMock(ElasticIPAddressClient.class);
InstanceClient instanceClient = createMock(InstanceClient.class);
NodeMetadata node = createMock(NodeMetadata.class);
expect(elasticIpCache.get(new RegionAndName("region", "i-blah"))).andThrow(new ExecutionException(null));
expect(client.getInstanceServices()).andReturn(instanceClient).atLeastOnce();
expect(instanceClient.terminateInstancesInRegion("region", "i-blah")).andReturn(null);
expect(getNode.getNode("region/i-blah")).andReturn(node);
replay(client);
replay(getNode);
replay(elasticIpCache);
replay(instanceClient);
replay(ipClient);
EC2DestroyNodeStrategy destroyer = new EC2DestroyNodeStrategy(client, getNode, elasticIpCache);
destroyer.autoAllocateElasticIps = true;
assertEquals(destroyer.destroyNode("region/i-blah"), node);
verify(client);
verify(getNode);
verify(elasticIpCache);
verify(instanceClient);
verify(ipClient);
}
}

View File

@ -38,7 +38,6 @@ import org.jclouds.aws.ec2.predicates.PlacementGroupDeleted;
import org.jclouds.compute.ComputeService;
import org.jclouds.compute.ComputeServiceContext;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.internal.ComputeServiceContextImpl;
import org.jclouds.compute.options.TemplateOptions;
@ -48,8 +47,8 @@ import org.jclouds.ec2.compute.domain.RegionAndName;
import org.jclouds.ec2.compute.functions.CreateSecurityGroupIfNeeded;
import org.jclouds.ec2.compute.functions.CreateUniqueKeyPair;
import org.jclouds.ec2.compute.functions.CredentialsForInstance;
import org.jclouds.ec2.compute.functions.LoadPublicIpForInstanceOrNull;
import org.jclouds.ec2.compute.functions.RegionAndIdToImage;
import org.jclouds.ec2.compute.functions.RunningInstanceToNodeMetadata;
import org.jclouds.ec2.compute.internal.EC2TemplateBuilderImpl;
import org.jclouds.ec2.domain.KeyPair;
import org.jclouds.ec2.domain.RunningInstance;
@ -59,14 +58,15 @@ import org.jclouds.rest.internal.RestContextImpl;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.cache.LoadingCache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.Sets;
import com.google.inject.Provides;
import com.google.inject.Scopes;
import com.google.inject.TypeLiteral;
import com.google.inject.assistedinject.FactoryModuleBuilder;
import com.google.inject.name.Names;
/**
*
@ -81,7 +81,9 @@ public class AWSEC2ComputeServiceDependenciesModule extends EC2ComputeServiceDep
bind(new TypeLiteral<CacheLoader<RunningInstance, Credentials>>() {
}).to(CredentialsForInstance.class);
bind(new TypeLiteral<CacheLoader<RegionAndName, String>>() {
}).to(CreateSecurityGroupIfNeeded.class);
}).annotatedWith(Names.named("SECURITY")).to(CreateSecurityGroupIfNeeded.class);
bind(new TypeLiteral<CacheLoader<RegionAndName, String>>() {
}).annotatedWith(Names.named("ELASTICIP")).to(LoadPublicIpForInstanceOrNull.class);
bind(new TypeLiteral<Function<RegionAndName, KeyPair>>() {
}).to(CreateUniqueKeyPair.class);
bind(new TypeLiteral<Function<RegionNameAndPublicKeyMaterial, KeyPair>>() {

View File

@ -45,6 +45,7 @@ import org.jclouds.compute.options.TemplateOptions;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.domain.Credentials;
import org.jclouds.ec2.compute.domain.RegionAndName;
import org.jclouds.ec2.compute.strategy.EC2CreateNodesInGroupThenAddToSet;
import org.jclouds.ec2.domain.RunningInstance;
import org.jclouds.ec2.options.RunInstancesOptions;
@ -52,6 +53,7 @@ import org.jclouds.logging.Logger;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
@ -76,6 +78,8 @@ public class AWSEC2CreateNodesInGroupThenAddToSet extends EC2CreateNodesInGroupT
@Inject
protected AWSEC2CreateNodesInGroupThenAddToSet(
AWSEC2Client client,
@Named("ELASTICIP") LoadingCache<RegionAndName, String> elasticIpCache,
@Named("NODE_RUNNING") Predicate<NodeMetadata> nodeRunning,
AWSEC2AsyncClient aclient,
@Named(PROPERTY_EC2_GENERATE_INSTANCE_NAMES) boolean generateInstanceNames,
Provider<TemplateBuilder> templateBuilderProvider,
@ -84,8 +88,8 @@ public class AWSEC2CreateNodesInGroupThenAddToSet extends EC2CreateNodesInGroupT
Function<RunningInstance, NodeMetadata> runningInstanceToNodeMetadata,
LoadingCache<RunningInstance, Credentials> instanceToCredentials, Map<String, Credentials> credentialStore,
ComputeUtils utils, SpotInstanceRequestToAWSRunningInstance spotConverter) {
super(client, templateBuilderProvider, createKeyPairAndSecurityGroupsAsNeededAndReturncustomize, instancePresent,
runningInstanceToNodeMetadata, instanceToCredentials, credentialStore, utils);
super(client, elasticIpCache, nodeRunning, templateBuilderProvider, createKeyPairAndSecurityGroupsAsNeededAndReturncustomize,
instancePresent, runningInstanceToNodeMetadata, instanceToCredentials, credentialStore, utils);
this.client = checkNotNull(client, "client");
this.aclient = checkNotNull(aclient, "aclient");
this.spotConverter = checkNotNull(spotConverter, "spotConverter");

View File

@ -25,13 +25,16 @@ import java.util.Map;
import java.util.NoSuchElementException;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Singleton;
import org.jclouds.aws.ec2.AWSEC2Client;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.domain.Credentials;
import org.jclouds.ec2.compute.domain.RegionAndName;
import org.jclouds.ec2.compute.strategy.EC2DestroyNodeStrategy;
import com.google.common.cache.LoadingCache;
import com.google.common.collect.Iterables;
/**
@ -46,14 +49,15 @@ public class AWSEC2DestroyNodeStrategy extends EC2DestroyNodeStrategy {
@Inject
protected AWSEC2DestroyNodeStrategy(AWSEC2Client client, GetNodeMetadataStrategy getNode,
@Named("ELASTICIP") LoadingCache<RegionAndName, String> elasticIpCache,
Map<String, Credentials> credentialStore) {
super(client, getNode);
super(client, getNode, elasticIpCache);
this.client = checkNotNull(client, "client");
this.credentialStore = checkNotNull(credentialStore, "credentialStore");
}
@Override
protected void destroyInstanceInRegion(String region, String id) {
protected void destroyInstanceInRegion(String id, String region) {
String spotId = id;
if (id.indexOf("sir-") != 0) {
try {