Issue 156: refactored compute api classes and added reboot support

This commit is contained in:
Adrian Cole 2010-02-08 22:09:07 -08:00
parent 52e40c4651
commit df86359e65
38 changed files with 1620 additions and 1443 deletions

View File

@ -18,15 +18,7 @@
*/
package org.jclouds.aws.ec2.compute;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.jclouds.aws.ec2.options.RunInstancesOptions.Builder.withKeyName;
import static org.jclouds.concurrent.ConcurrentUtils.awaitCompletion;
import static org.jclouds.concurrent.ConcurrentUtils.makeListenable;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import javax.inject.Inject;
@ -37,221 +29,60 @@ import javax.inject.Singleton;
import org.jclouds.Constants;
import org.jclouds.aws.domain.Region;
import org.jclouds.aws.ec2.EC2Client;
import org.jclouds.aws.ec2.compute.domain.EC2Size;
import org.jclouds.aws.ec2.compute.config.EC2ComputeServiceContextModule.GetRegionFromNodeOrDefault;
import org.jclouds.aws.ec2.compute.domain.KeyPairCredentials;
import org.jclouds.aws.ec2.compute.domain.PortsRegionTag;
import org.jclouds.aws.ec2.compute.domain.RegionTag;
import org.jclouds.aws.ec2.compute.functions.CreateKeyPairIfNeeded;
import org.jclouds.aws.ec2.compute.functions.CreateSecurityGroupIfNeeded;
import org.jclouds.aws.ec2.compute.functions.RunningInstanceToNodeMetadata;
import org.jclouds.aws.ec2.domain.AvailabilityZone;
import org.jclouds.aws.ec2.domain.InstanceState;
import org.jclouds.aws.ec2.domain.Reservation;
import org.jclouds.aws.ec2.domain.RunningInstance;
import org.jclouds.aws.ec2.options.RunInstancesOptions;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.ComputeType;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.internal.BaseComputeService;
import org.jclouds.compute.options.TemplateOptions;
import org.jclouds.compute.strategy.DestroyNodeStrategy;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.compute.strategy.ListNodesStrategy;
import org.jclouds.compute.strategy.RebootNodeStrategy;
import org.jclouds.compute.strategy.RunNodesAndAddToSetStrategy;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.domain.Location;
import org.jclouds.domain.LocationScope;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.inject.internal.ImmutableSet;
/**
* @author Adrian Cole
*/
@Singleton
public class EC2ComputeService extends BaseComputeService {
private static Function<RunningInstance, String> instanceToId = new Function<RunningInstance, String>() {
@Override
public String apply(RunningInstance from) {
return from.getId();
}
};
protected final EC2Client ec2Client;
protected final GetRegionFromNodeOrDefault getRegionFromNodeOrDefault;
protected final Map<RegionTag, KeyPairCredentials> credentialsMap;
protected final Map<PortsRegionTag, String> securityGroupMap;
protected final CreateKeyPairIfNeeded createKeyPairIfNeeded;
protected final CreateSecurityGroupIfNeeded createSecurityGroupIfNeeded;
protected final Predicate<RunningInstance> instanceStateRunning;
protected final Predicate<RunningInstance> instanceStateTerminated;
protected final RunningInstanceToNodeMetadata runningInstanceToNodeMetadata;
@Inject
EC2ComputeService(Provider<Map<String, ? extends Image>> images,
protected EC2ComputeService(Provider<Map<String, ? extends Image>> images,
Provider<Map<String, ? extends Size>> sizes,
Provider<Map<String, ? extends Location>> locations,
Provider<TemplateBuilder> templateBuilderProvider, String nodeNamingConvention,
ComputeUtils utils, @Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor,
EC2Client ec2Client, Map<RegionTag, KeyPairCredentials> credentialsMap,
ListNodesStrategy listNodesStrategy, GetNodeMetadataStrategy getNodeMetadataStrategy,
RunNodesAndAddToSetStrategy runNodesAndAddToSetStrategy,
RebootNodeStrategy rebootNodeStrategy, DestroyNodeStrategy destroyNodeStrategy,
Provider<TemplateBuilder> templateBuilderProvider, ComputeUtils utils,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor, EC2Client ec2Client,
GetRegionFromNodeOrDefault getRegionFromNodeOrDefault,
Map<RegionTag, KeyPairCredentials> credentialsMap,
Map<PortsRegionTag, String> securityGroupMap,
CreateKeyPairIfNeeded createKeyPairIfNeeded,
CreateSecurityGroupIfNeeded createSecurityGroupIfNeeded,
@Named("RUNNING") Predicate<RunningInstance> instanceStateRunning,
@Named("TERMINATED") Predicate<RunningInstance> instanceStateTerminated,
RunningInstanceToNodeMetadata runningInstanceToNodeMetadata) {
super(images, sizes, locations, templateBuilderProvider, nodeNamingConvention, utils,
executor);
@Named("TERMINATED") Predicate<RunningInstance> instanceStateTerminated) {
super(images, sizes, locations, listNodesStrategy, getNodeMetadataStrategy,
runNodesAndAddToSetStrategy, rebootNodeStrategy, destroyNodeStrategy,
templateBuilderProvider, utils, executor);
this.ec2Client = ec2Client;
this.getRegionFromNodeOrDefault = getRegionFromNodeOrDefault;
this.credentialsMap = credentialsMap;
this.securityGroupMap = securityGroupMap;
this.createKeyPairIfNeeded = createKeyPairIfNeeded;
this.createSecurityGroupIfNeeded = createSecurityGroupIfNeeded;
this.instanceStateRunning = instanceStateRunning;
this.instanceStateTerminated = instanceStateTerminated;
this.runningInstanceToNodeMetadata = runningInstanceToNodeMetadata;
}
@Override
public Map<String, NodeMetadata> runNodesWithTag(String tag, int count, final Template template) {
checkArgument(tag.indexOf('-') == -1, "tag cannot contain hyphens");
checkArgument(template.getSize() instanceof EC2Size,
"unexpected image type. should be EC2Size, was: " + template.getSize().getClass());
EC2Size ec2Size = EC2Size.class.cast(template.getSize());
// parse the availability zone of the request
AvailabilityZone zone = template.getLocation().getScope() == LocationScope.ZONE ? AvailabilityZone
.fromValue(template.getLocation().getId())
: null;
// if the location has a parent, it must be an availability zone.
Region region = zone == null ? Region.fromValue(template.getLocation().getId()) : Region
.fromValue(template.getLocation().getParent());
// get or create incidental resources
// TODO race condition. we were using MapMaker, but it doesn't seem to refresh properly when
// another thread
// deletes a key
RegionTag regionTag = new RegionTag(region, tag);
if (!credentialsMap.containsKey(regionTag)) {
credentialsMap.put(regionTag, createKeyPairIfNeeded.apply(regionTag));
}
TemplateOptions options = template.getOptions();
PortsRegionTag portsRegionTag = new PortsRegionTag(region, tag, options.getInboundPorts());
if (!securityGroupMap.containsKey(portsRegionTag)) {
securityGroupMap.put(portsRegionTag, createSecurityGroupIfNeeded.apply(portsRegionTag));
}
logger
.debug(
">> running %d instance region(%s) zone(%s) ami(%s) type(%s) keyPair(%s) securityGroup(%s)",
count, region, zone, template.getImage().getId(),
ec2Size.getInstanceType(), tag, tag);
RunInstancesOptions instanceOptions = withKeyName(tag)// key
.asType(ec2Size.getInstanceType())// instance size
.withSecurityGroup(tag)// group I created above
.withAdditionalInfo(tag);
final Set<NodeMetadata> nodes = Sets.newHashSet();
int nodesToStart = count;
while (nodesToStart > 0) {
Reservation reservation = ec2Client.getInstanceServices().runInstancesInRegion(region,
zone, template.getImage().getId(), 1, nodesToStart, instanceOptions);
Iterable<String> ids = Iterables.transform(reservation, instanceToId);
String idsString = Joiner.on(',').join(ids);
logger.debug("<< started instances(%s)", idsString);
Iterables.all(reservation, instanceStateRunning);
logger.debug("<< running instances(%s)", idsString);
Map<NodeMetadata, ListenableFuture<Void>> responses = Maps.newHashMap();
for (final NodeMetadata node : Iterables.transform(Iterables.concat(ec2Client
.getInstanceServices().describeInstancesInRegion(region,
Iterables.toArray(ids, String.class))), runningInstanceToNodeMetadata)) {
responses.put(node, makeListenable(executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
try {
utils.runOptionsOnNode(node, template.getOptions());
logger.debug("<< options applied instance(%s)", node.getId());
nodes.add(node);
} catch (Exception e) {
logger.error(e, "<< error applying instance(%s) [%s] destroying ", node
.getId(), e.getMessage());
destroyNode(node);
}
return null;
}
}), executor));
}
nodesToStart = awaitCompletion(responses, executor, null, logger, "nodes").size();
}
return Maps.uniqueIndex(nodes, METADATA_TO_ID);
}
@Override
public NodeMetadata getNodeMetadata(ComputeMetadata node) {
checkArgument(node.getType() == ComputeType.NODE, "this is only valid for nodes, not "
+ node.getType());
checkNotNull(node.getId(), "node.id");
Region region = getRegionFromNodeOrDefault(node);
RunningInstance runningInstance = Iterables.getOnlyElement(getAllRunningInstancesInRegion(
region, node.getId()));
return runningInstanceToNodeMetadata.apply(runningInstance);
}
private Iterable<RunningInstance> getAllRunningInstancesInRegion(Region region, String id) {
return Iterables
.concat(ec2Client.getInstanceServices().describeInstancesInRegion(region, id));
}
protected Iterable<NodeMetadata> doGetNodes() {
Set<NodeMetadata> nodes = Sets.newHashSet();
for (Region region : ImmutableSet.of(Region.US_EAST_1, Region.US_WEST_1, Region.EU_WEST_1)) {
Iterables.addAll(nodes, Iterables.transform(Iterables.concat(ec2Client
.getInstanceServices().describeInstancesInRegion(region)),
runningInstanceToNodeMetadata));
}
return nodes;
}
@Override
protected boolean doDestroyNode(ComputeMetadata metadata) {
NodeMetadata node = metadata instanceof NodeMetadata ? NodeMetadata.class.cast(metadata)
: getNodeMetadata(metadata);
String tag = checkNotNull(node.getTag(), "node.tag");
Region region = getRegionFromNodeOrDefault(node);
RunningInstance instance = getInstance(node, region);
if (instance.getInstanceState() != InstanceState.TERMINATED) {
logger.debug(">> terminating instance(%s)", node.getId());
boolean success = false;
while (!success) {
ec2Client.getInstanceServices().terminateInstancesInRegion(region, node.getId());
success = instanceStateTerminated.apply(getInstance(node, region));
}
logger.debug("<< terminated instance(%s) success(%s)", node.getId(), success);
}
if (Iterables.all(doGetNodes(tag).values(), new Predicate<NodeMetadata>() {
@Override
public boolean apply(NodeMetadata input) {
return input.getState() == NodeState.TERMINATED;
}
})) {
deleteKeyPair(region, tag);
deleteSecurityGroup(region, tag);
}
return true;
}
private RunningInstance getInstance(NodeMetadata node, Region region) {
return Iterables.getOnlyElement(getAllRunningInstancesInRegion(region, node.getId()));
}
private void deleteSecurityGroup(Region region, String tag) {
@ -273,16 +104,20 @@ public class EC2ComputeService extends BaseComputeService {
}
}
private Region getRegionFromNodeOrDefault(ComputeMetadata node) {
Location location = getLocations().get(node.getLocationId());
Region region = location.getScope() == LocationScope.REGION ? Region.fromValue(location
.getId()) : Region.fromValue(location.getParent());
return region;
}
@Override
protected NodeMetadata startNode(String tag, String name, Template template) {
throw new UnsupportedOperationException();
public void destroyNodesWithTag(String tag) {
super.destroyNodesWithTag(tag);
Iterable<? extends NodeMetadata> nodes = Iterables.filter(getNodesWithTag(tag).values(),
new Predicate<NodeMetadata>() {
@Override
public boolean apply(NodeMetadata input) {
return input.getState() == NodeState.TERMINATED;
}
});
if (Iterables.size(nodes) > 0) {
Region region = getRegionFromNodeOrDefault.apply(Iterables.get(nodes, 0));
deleteKeyPair(region, tag);
deleteSecurityGroup(region, tag);
}
}
}

View File

@ -31,6 +31,7 @@ import java.util.regex.Matcher;
import java.util.regex.Pattern;
import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Singleton;
@ -45,19 +46,30 @@ import org.jclouds.aws.ec2.compute.domain.PortsRegionTag;
import org.jclouds.aws.ec2.compute.domain.RegionTag;
import org.jclouds.aws.ec2.compute.functions.CreateKeyPairIfNeeded;
import org.jclouds.aws.ec2.compute.functions.CreateSecurityGroupIfNeeded;
import org.jclouds.aws.ec2.compute.functions.RunningInstanceToNodeMetadata;
import org.jclouds.aws.ec2.compute.strategy.EC2DestroyNodeStrategy;
import org.jclouds.aws.ec2.compute.strategy.EC2RunNodesAndAddToSetStrategy;
import org.jclouds.aws.ec2.config.EC2ContextModule;
import org.jclouds.aws.ec2.domain.AvailabilityZone;
import org.jclouds.aws.ec2.domain.RunningInstance;
import org.jclouds.aws.ec2.services.InstanceClient;
import org.jclouds.compute.ComputeService;
import org.jclouds.compute.ComputeServiceContext;
import org.jclouds.compute.domain.Architecture;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.OsFamily;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.internal.ImageImpl;
import org.jclouds.compute.internal.ComputeServiceContextImpl;
import org.jclouds.compute.predicates.RunScriptRunning;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.strategy.DestroyNodeStrategy;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.compute.strategy.ListNodesStrategy;
import org.jclouds.compute.strategy.RebootNodeStrategy;
import org.jclouds.compute.strategy.RunNodesAndAddToSetStrategy;
import org.jclouds.domain.Location;
import org.jclouds.domain.LocationScope;
import org.jclouds.domain.internal.LocationImpl;
@ -83,6 +95,111 @@ import com.google.inject.Provides;
* @author Adrian Cole
*/
public class EC2ComputeServiceContextModule extends EC2ContextModule {
@Override
protected void configure() {
super.configure();
bind(ComputeService.class).to(EC2ComputeService.class);
bind(RunNodesAndAddToSetStrategy.class).to(EC2RunNodesAndAddToSetStrategy.class);
bind(ListNodesStrategy.class).to(EC2ListNodesStrategy.class);
bind(GetNodeMetadataStrategy.class).to(EC2GetNodeMetadataStrategy.class);
bind(RebootNodeStrategy.class).to(EC2RebootNodeStrategy.class);
bind(DestroyNodeStrategy.class).to(EC2DestroyNodeStrategy.class);
}
@Singleton
public static class EC2ListNodesStrategy implements ListNodesStrategy {
private final InstanceClient client;
private final RunningInstanceToNodeMetadata runningInstanceToNodeMetadata;
@Inject
protected EC2ListNodesStrategy(InstanceClient client,
RunningInstanceToNodeMetadata runningInstanceToNodeMetadata) {
this.client = client;
this.runningInstanceToNodeMetadata = runningInstanceToNodeMetadata;
}
@Override
public Iterable<? extends ComputeMetadata> execute() {
Set<NodeMetadata> nodes = Sets.newHashSet();
for (Region region : ImmutableSet.of(Region.US_EAST_1, Region.US_WEST_1, Region.EU_WEST_1)) {
Iterables.addAll(nodes, Iterables.transform(Iterables.concat(client
.describeInstancesInRegion(region)), runningInstanceToNodeMetadata));
}
return nodes;
}
}
@Singleton
public static class GetRegionFromNodeOrDefault implements Function<ComputeMetadata, Region> {
private final Map<String, ? extends Location> locations;
@Inject
protected GetRegionFromNodeOrDefault(Map<String, ? extends Location> locations) {
this.locations = locations;
}
public Region apply(ComputeMetadata node) {
Location location = locations.get(node.getLocationId());
Region region = location.getScope() == LocationScope.REGION ? Region.fromValue(location
.getId()) : Region.fromValue(location.getParent());
return region;
}
}
@Singleton
public static class EC2GetNodeMetadataStrategy implements GetNodeMetadataStrategy {
private final InstanceClient client;
private final RunningInstanceToNodeMetadata runningInstanceToNodeMetadata;
private final GetRegionFromNodeOrDefault getRegionFromNodeOrDefault;
@Inject
protected EC2GetNodeMetadataStrategy(InstanceClient client,
GetRegionFromNodeOrDefault getRegionFromNodeOrDefault,
RunningInstanceToNodeMetadata runningInstanceToNodeMetadata) {
this.client = client;
this.getRegionFromNodeOrDefault = getRegionFromNodeOrDefault;
this.runningInstanceToNodeMetadata = runningInstanceToNodeMetadata;
}
@Override
public NodeMetadata execute(ComputeMetadata node) {
Region region = getRegionFromNodeOrDefault.apply(node);
RunningInstance runningInstance = Iterables.getOnlyElement(getAllRunningInstancesInRegion(
client, region, node.getId()));
return runningInstanceToNodeMetadata.apply(runningInstance);
}
}
public static Iterable<RunningInstance> getAllRunningInstancesInRegion(InstanceClient client,
Region region, String id) {
return Iterables.concat(client.describeInstancesInRegion(region, id));
}
@Singleton
public static class EC2RebootNodeStrategy implements RebootNodeStrategy {
private final InstanceClient client;
private final GetRegionFromNodeOrDefault getRegionFromNodeOrDefault;
@Inject
protected EC2RebootNodeStrategy(InstanceClient client,
GetRegionFromNodeOrDefault getRegionFromNodeOrDefault) {
this.client = client;
this.getRegionFromNodeOrDefault = getRegionFromNodeOrDefault;
}
@Override
public boolean execute(ComputeMetadata node) {
Region region = getRegionFromNodeOrDefault.apply(node);
client.rebootInstancesInRegion(region, node.getId());
return true;
}
}
@Provides
@Singleton
@Named("NOT_RUNNING")
@ -91,12 +208,6 @@ public class EC2ComputeServiceContextModule extends EC2ContextModule {
TimeUnit.SECONDS);
}
@Override
protected void configure() {
super.configure();
bind(ComputeService.class).to(EC2ComputeService.class).asEagerSingleton();
}
@Provides
@Singleton
protected final Map<RegionTag, KeyPairCredentials> credentialsMap(CreateKeyPairIfNeeded in) {

View File

@ -0,0 +1,77 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.aws.ec2.compute.strategy;
import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Singleton;
import org.jclouds.aws.domain.Region;
import org.jclouds.aws.ec2.EC2Client;
import org.jclouds.aws.ec2.compute.config.EC2ComputeServiceContextModule.GetRegionFromNodeOrDefault;
import org.jclouds.aws.ec2.domain.RunningInstance;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.strategy.DestroyNodeStrategy;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.logging.Logger;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
/**
*
* @author Adrian Cole
*/
@Singleton
public class EC2DestroyNodeStrategy implements DestroyNodeStrategy {
@Resource
@Named(ComputeServiceConstants.COMPUTE_LOGGER)
protected Logger logger = Logger.NULL;
protected final EC2Client ec2Client;
protected final Predicate<RunningInstance> instanceStateTerminated;
protected final GetRegionFromNodeOrDefault getRegionFromNodeOrDefault;
protected final GetNodeMetadataStrategy getNodeMetadataStrategy;
@Inject
protected EC2DestroyNodeStrategy(EC2Client ec2Client,
@Named("TERMINATED") Predicate<RunningInstance> instanceStateTerminated,
GetRegionFromNodeOrDefault getRegionFromNodeOrDefault,
GetNodeMetadataStrategy getNodeMetadataStrategy) {
this.ec2Client = ec2Client;
this.instanceStateTerminated = instanceStateTerminated;
this.getRegionFromNodeOrDefault = getRegionFromNodeOrDefault;
this.getNodeMetadataStrategy = getNodeMetadataStrategy;
}
@Override
public boolean execute(ComputeMetadata metadata) {
NodeMetadata node = metadata instanceof NodeMetadata ? NodeMetadata.class.cast(metadata)
: getNodeMetadataStrategy.execute(metadata);
Region region = getRegionFromNodeOrDefault.apply(node);
ec2Client.getInstanceServices().terminateInstancesInRegion(region, node.getId());
return instanceStateTerminated.apply(Iterables.getOnlyElement(Iterables.concat(ec2Client
.getInstanceServices().describeInstancesInRegion(region, node.getId()))));
}
}

View File

@ -0,0 +1,192 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.aws.ec2.compute.strategy;
import static com.google.common.base.Preconditions.checkArgument;
import static org.jclouds.aws.ec2.options.RunInstancesOptions.Builder.withKeyName;
import static org.jclouds.concurrent.ConcurrentUtils.makeListenable;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Singleton;
import org.jclouds.Constants;
import org.jclouds.aws.domain.Region;
import org.jclouds.aws.ec2.EC2Client;
import org.jclouds.aws.ec2.compute.domain.EC2Size;
import org.jclouds.aws.ec2.compute.domain.KeyPairCredentials;
import org.jclouds.aws.ec2.compute.domain.PortsRegionTag;
import org.jclouds.aws.ec2.compute.domain.RegionTag;
import org.jclouds.aws.ec2.compute.functions.CreateKeyPairIfNeeded;
import org.jclouds.aws.ec2.compute.functions.CreateSecurityGroupIfNeeded;
import org.jclouds.aws.ec2.compute.functions.RunningInstanceToNodeMetadata;
import org.jclouds.aws.ec2.domain.AvailabilityZone;
import org.jclouds.aws.ec2.domain.Reservation;
import org.jclouds.aws.ec2.domain.RunningInstance;
import org.jclouds.aws.ec2.options.RunInstancesOptions;
import org.jclouds.compute.ComputeService;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.options.TemplateOptions;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.strategy.RunNodesAndAddToSetStrategy;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.domain.LocationScope;
import org.jclouds.logging.Logger;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.ListenableFuture;
/**
* creates futures that correlate to
*
* @author Adrian Cole
*/
@Singleton
public class EC2RunNodesAndAddToSetStrategy implements RunNodesAndAddToSetStrategy {
@Resource
@Named(ComputeServiceConstants.COMPUTE_LOGGER)
protected Logger logger = Logger.NULL;
private static Function<RunningInstance, String> instanceToId = new Function<RunningInstance, String>() {
@Override
public String apply(RunningInstance from) {
return from.getId();
}
};
protected final ComputeService computeService;
protected final EC2Client ec2Client;
protected final Map<RegionTag, KeyPairCredentials> credentialsMap;
protected final Map<PortsRegionTag, String> securityGroupMap;
protected final CreateKeyPairIfNeeded createKeyPairIfNeeded;
protected final CreateSecurityGroupIfNeeded createSecurityGroupIfNeeded;
protected final Predicate<RunningInstance> instanceStateRunning;
protected final RunningInstanceToNodeMetadata runningInstanceToNodeMetadata;
protected final ComputeUtils utils;
@Inject
protected EC2RunNodesAndAddToSetStrategy(ComputeService computeService, EC2Client ec2Client,
Map<RegionTag, KeyPairCredentials> credentialsMap,
Map<PortsRegionTag, String> securityGroupMap,
CreateKeyPairIfNeeded createKeyPairIfNeeded,
CreateSecurityGroupIfNeeded createSecurityGroupIfNeeded,
@Named("RUNNING") Predicate<RunningInstance> instanceStateRunning,
RunningInstanceToNodeMetadata runningInstanceToNodeMetadata, ComputeUtils utils,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
this.computeService = computeService;
this.ec2Client = ec2Client;
this.credentialsMap = credentialsMap;
this.securityGroupMap = securityGroupMap;
this.createKeyPairIfNeeded = createKeyPairIfNeeded;
this.createSecurityGroupIfNeeded = createSecurityGroupIfNeeded;
this.instanceStateRunning = instanceStateRunning;
this.runningInstanceToNodeMetadata = runningInstanceToNodeMetadata;
this.utils = utils;
this.executor = executor;
}
protected final ExecutorService executor;
@Override
public Map<?, ListenableFuture<Void>> execute(final String tag, final int count,
final Template template, final Set<NodeMetadata> nodes) {
checkArgument(template.getSize() instanceof EC2Size,
"unexpected image type. should be EC2Size, was: " + template.getSize().getClass());
EC2Size ec2Size = EC2Size.class.cast(template.getSize());
// parse the availability zone of the request
AvailabilityZone zone = template.getLocation().getScope() == LocationScope.ZONE ? AvailabilityZone
.fromValue(template.getLocation().getId())
: null;
// if the location has a parent, it must be an availability zone.
Region region = zone == null ? Region.fromValue(template.getLocation().getId()) : Region
.fromValue(template.getLocation().getParent());
// get or create incidental resources
// TODO race condition. we were using MapMaker, but it doesn't seem to refresh properly when
// another thread
// deletes a key
RegionTag regionTag = new RegionTag(region, tag);
if (!credentialsMap.containsKey(regionTag)) {
credentialsMap.put(regionTag, createKeyPairIfNeeded.apply(regionTag));
}
TemplateOptions options = template.getOptions();
PortsRegionTag portsRegionTag = new PortsRegionTag(region, tag, options.getInboundPorts());
if (!securityGroupMap.containsKey(portsRegionTag)) {
securityGroupMap.put(portsRegionTag, createSecurityGroupIfNeeded.apply(portsRegionTag));
}
logger
.debug(
">> running %d instance region(%s) zone(%s) ami(%s) type(%s) keyPair(%s) securityGroup(%s)",
count, region, zone, template.getImage().getId(),
ec2Size.getInstanceType(), tag, tag);
RunInstancesOptions instanceOptions = withKeyName(tag)// key
.asType(ec2Size.getInstanceType())// instance size
.withSecurityGroup(tag)// group I created above
.withAdditionalInfo(tag);
Reservation reservation = ec2Client.getInstanceServices().runInstancesInRegion(region, zone,
template.getImage().getId(), 1, count, instanceOptions);
Iterable<String> ids = Iterables.transform(reservation, instanceToId);
String idsString = Joiner.on(',').join(ids);
logger.debug("<< started instances(%s)", idsString);
Iterables.all(reservation, instanceStateRunning);
logger.debug("<< running instances(%s)", idsString);
Map<NodeMetadata, ListenableFuture<Void>> responses = Maps.newHashMap();
for (final NodeMetadata node : Iterables.transform(getInstances(region, ids),
runningInstanceToNodeMetadata)) {
responses.put(node, makeListenable(executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
try {
utils.runOptionsOnNode(node, template.getOptions());
logger.debug("<< options applied node(%s)", node.getId());
nodes.add(computeService.getNodeMetadata(node));
} catch (Exception e) {
if (!template.getOptions().shouldDestroyOnError())
nodes.add(computeService.getNodeMetadata(node));
}
return null;
}
}), executor));
}
return responses;
}
private Iterable<RunningInstance> getInstances(Region region, Iterable<String> ids) {
return Iterables.concat(ec2Client.getInstanceServices().describeInstancesInRegion(region,
Iterables.toArray(ids, String.class)));
}
}

View File

@ -96,7 +96,7 @@ public class EC2RestClientModule extends AbstractModule {
@Named("TERMINATED")
protected Predicate<RunningInstance> instanceStateTerminated(
InstanceStateTerminated stateTerminated) {
return new RetryablePredicate<RunningInstance>(stateTerminated, 600, 50,
return new RetryablePredicate<RunningInstance>(stateTerminated, 20000, 500,
TimeUnit.MILLISECONDS);
}

View File

@ -21,6 +21,7 @@ package org.jclouds.aws.ec2.compute;
import static org.jclouds.compute.domain.OsFamily.UBUNTU;
import org.jclouds.compute.BaseComputeServiceLiveTest;
import org.jclouds.compute.domain.Architecture;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.ssh.jsch.config.JschSshClientModule;
@ -40,7 +41,7 @@ public class EC2ComputeServiceLiveTest extends BaseComputeServiceLiveTest {
}
protected Template buildTemplate(TemplateBuilder templateBuilder) {
return templateBuilder.osFamily(UBUNTU).smallest().build();
return templateBuilder.osFamily(UBUNTU).smallest().architecture(Architecture.X86_32).build();
}
@Override

View File

@ -58,6 +58,11 @@ public class EC2ContextModuleTest {
});
}
@Test
void testM1SMALLIsSmallest() {
// TODO
}
@Test
void testContextImpl() {
RestContext<EC2AsyncClient, EC2Client> handler = createInjector().getInstance(

View File

@ -26,14 +26,18 @@ import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.internal.BaseComputeService;
import org.jclouds.domain.Location;
import com.google.inject.ImplementedBy;
/**
* Provides portable access to launching compute instances.
*
* @author Adrian Cole
* @author Ivan Meredith
*/
@ImplementedBy(BaseComputeService.class)
public interface ComputeService {
/**
* Makes a new template builder for this service
@ -119,6 +123,17 @@ public interface ComputeService {
*/
void destroyNodesWithTag(String tag);
/**
* reboot the node.
*/
void rebootNode(ComputeMetadata node);
/**
* nodes which are tagged are treated as a logical set. Using this command, you can save time by
* rebooting the nodes in parallel.
*/
void rebootNodesWithTag(String tag);
/**
* Find a node by its id
*/

View File

@ -28,7 +28,6 @@ import static com.google.common.base.Preconditions.checkNotNull;
import static org.jclouds.concurrent.ConcurrentUtils.awaitCompletion;
import static org.jclouds.concurrent.ConcurrentUtils.makeListenable;
import java.security.SecureRandom;
import java.util.Map;
import java.util.Set;
import java.util.Map.Entry;
@ -36,8 +35,10 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Provider;
import javax.inject.Singleton;
import org.jclouds.Constants;
import org.jclouds.compute.ComputeService;
@ -50,6 +51,11 @@ import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.strategy.DestroyNodeStrategy;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.compute.strategy.ListNodesStrategy;
import org.jclouds.compute.strategy.RebootNodeStrategy;
import org.jclouds.compute.strategy.RunNodesAndAddToSetStrategy;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.domain.Location;
import org.jclouds.logging.Logger;
@ -66,7 +72,8 @@ import com.google.common.util.concurrent.ListenableFuture;
*
* @author Adrian Cole
*/
public abstract class BaseComputeService implements ComputeService {
@Singleton
public class BaseComputeService implements ComputeService {
@Resource
@Named(ComputeServiceConstants.COMPUTE_LOGGER)
@ -74,8 +81,12 @@ public abstract class BaseComputeService implements ComputeService {
protected final Provider<Map<String, ? extends Image>> images;
protected final Provider<Map<String, ? extends Size>> sizes;
protected final Provider<Map<String, ? extends Location>> locations;
protected final ListNodesStrategy listNodesStrategy;
protected final GetNodeMetadataStrategy getNodeMetadataStrategy;
protected final RunNodesAndAddToSetStrategy runNodesAndAddToSetStrategy;
protected final RebootNodeStrategy rebootNodeStrategy;
protected final DestroyNodeStrategy destroyNodeStrategy;
protected final Provider<TemplateBuilder> templateBuilderProvider;
protected final String nodeNamingConvention;
protected final ComputeUtils utils;
protected final ExecutorService executor;
@ -107,16 +118,24 @@ public abstract class BaseComputeService implements ComputeService {
}
};
@Inject
protected BaseComputeService(Provider<Map<String, ? extends Image>> images,
Provider<Map<String, ? extends Size>> sizes,
Provider<Map<String, ? extends Location>> locations,
Provider<TemplateBuilder> templateBuilderProvider, String nodeNamingConvention,
ComputeUtils utils, @Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
ListNodesStrategy listNodesStrategy, GetNodeMetadataStrategy getNodeMetadataStrategy,
RunNodesAndAddToSetStrategy runNodesAndAddToSetStrategy,
RebootNodeStrategy rebootNodeStrategy, DestroyNodeStrategy destroyNodeStrategy,
Provider<TemplateBuilder> templateBuilderProvider, ComputeUtils utils,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
this.images = images;
this.sizes = sizes;
this.locations = locations;
this.listNodesStrategy = listNodesStrategy;
this.getNodeMetadataStrategy = getNodeMetadataStrategy;
this.runNodesAndAddToSetStrategy = runNodesAndAddToSetStrategy;
this.rebootNodeStrategy = rebootNodeStrategy;
this.destroyNodeStrategy = destroyNodeStrategy;
this.templateBuilderProvider = templateBuilderProvider;
this.nodeNamingConvention = nodeNamingConvention;
this.utils = utils;
this.executor = executor;
}
@ -126,38 +145,18 @@ public abstract class BaseComputeService implements ComputeService {
final Template template) {
checkArgument(tag.indexOf('-') == -1, "tag cannot contain hyphens");
checkNotNull(template.getLocation(), "location");
final Set<NodeMetadata> nodes = Sets.newHashSet();
logger.debug(">> running %d node%s tag(%s) location(%s) image(%s) size(%s) options(%s)",
count, count > 1 ? "s" : "", tag, template.getLocation().getId(), template
.getImage().getId(), template.getSize().getId(), template.getOptions());
Map<String, ListenableFuture<Void>> responses = Maps.newHashMap();
for (final String name : getNextNames(tag, count)) {
responses.put(name, makeListenable(executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
NodeMetadata node = null;
logger.debug(">> starting node(%s) tag(%s)", name, tag);
node = startNode(tag, name, template);
logger.debug("<< running node(%s)", node.getId());
try {
utils.runOptionsOnNode(node, template.getOptions());
logger.debug("<< options applied node(%s)", node.getId());
nodes.add(node);
} catch (Exception e) {
if (!template.getOptions().shouldDestroyOnError())
nodes.add(node);
}
return null;
}
}), executor));
}
Map<String, Exception> exceptions = awaitCompletion(responses, executor, null, logger,
final Set<NodeMetadata> nodes = Sets.newHashSet();
Map<?, ListenableFuture<Void>> responses = runNodesAndAddToSetStrategy.execute(tag, count,
template, nodes);
Map<?, Exception> exceptions = awaitCompletion(responses, executor, null, logger,
"starting nodes");
if (exceptions.size() > 0 && template.getOptions().shouldDestroyOnError()) {
ImmutableMap<String, ? extends ComputeMetadata> currentNodes = Maps.uniqueIndex(doGetNodes(),
METADATA_TO_ID);
for (Entry<String, Exception> entry : exceptions.entrySet()) {
ImmutableMap<?, ? extends ComputeMetadata> currentNodes = Maps.uniqueIndex(
listNodesStrategy.execute(), METADATA_TO_ID);
for (Entry<?, Exception> entry : exceptions.entrySet()) {
logger.error(entry.getValue(), "<< error applying nodes(%s) [%s] destroying ", entry
.getKey(), entry.getValue().getMessage());
destroyNode(currentNodes.get(entry.getKey()));
@ -166,37 +165,27 @@ public abstract class BaseComputeService implements ComputeService {
return Maps.uniqueIndex(nodes, METADATA_TO_ID);
}
protected Set<String> getNextNames(final String tag, int count) {
Set<String> names = Sets.newHashSet();
int nodeIndex = new SecureRandom().nextInt(8096);
Iterable<? extends ComputeMetadata> allNodes = doGetNodes();
Map<String, ? extends ComputeMetadata> currentNodes = Maps.uniqueIndex(allNodes, METADATA_TO_NAME);
while (names.size() < count) {
String name = String.format(nodeNamingConvention, tag, nodeIndex++);
if (!currentNodes.containsKey(name)) {
names.add(name);
}
}
return names;
@Override
public void destroyNode(ComputeMetadata node) {
checkArgument(node.getType() == ComputeType.NODE, "this is only valid for nodes, not "
+ node.getType());
checkNotNull(node.getId(), "node.id");
logger.debug(">> destroying node(%s)", node.getId());
boolean successful = destroyNodeStrategy.execute(node);
logger.debug("<< destroyed node(%s) success(%s)", node.getId(), successful);
}
protected abstract NodeMetadata startNode(String tag, String name, Template template);
protected abstract boolean doDestroyNode(ComputeMetadata node);
protected abstract Iterable<? extends ComputeMetadata> doGetNodes();
@Override
public void destroyNodesWithTag(String tag) { // TODO parallel
logger.debug(">> terminating nodes by tag(%s)", tag);
Iterable<? extends NodeMetadata> nodesToDestroy = Iterables.filter(doGetNodes(tag).values(),
new Predicate<NodeMetadata>() {
@Override
public boolean apply(NodeMetadata input) {
return input.getState() != NodeState.TERMINATED;
logger.debug(">> destroying nodes by tag(%s)", tag);
Iterable<? extends NodeMetadata> nodesToDestroy = Iterables.filter(doGetNodesWithTag(tag)
.values(), new Predicate<NodeMetadata>() {
@Override
public boolean apply(NodeMetadata input) {
return input.getState() != NodeState.TERMINATED;
}
});
}
});
Map<NodeMetadata, ListenableFuture<Void>> responses = Maps.newHashMap();
for (final NodeMetadata node : nodesToDestroy) {
responses.put(node, makeListenable(executor.submit(new Callable<Void>() {
@ -211,36 +200,27 @@ public abstract class BaseComputeService implements ComputeService {
logger.debug("<< destroyed");
}
@Override
public Map<String, ? extends Location> getLocations() {
return locations.get();
}
@Override
public Map<String, ? extends ComputeMetadata> getNodes() {
logger.debug(">> listing servers");
ImmutableMap<String, ? extends ComputeMetadata> map = Maps.uniqueIndex(doGetNodes(), METADATA_TO_ID);
ImmutableMap<String, ? extends ComputeMetadata> map = Maps.uniqueIndex(listNodesStrategy
.execute(), METADATA_TO_ID);
logger.debug("<< list(%d)", map.size());
return map;
}
@Override
public void destroyNode(ComputeMetadata node) {
checkArgument(node.getType() == ComputeType.NODE, "this is only valid for nodes, not "
+ node.getType());
checkNotNull(node.getId(), "node.id");
logger.debug(">> deleting node(%s)", node.getId());
boolean successful = doDestroyNode(node);
logger.debug("<< deleted node(%s) success(%s)", node.getId(), successful);
}
protected Map<String, ? extends NodeMetadata> doGetNodes(final String tag) {
Iterable<? extends NodeMetadata> nodes = Iterables.filter(Iterables.transform(doGetNodes(),
new Function<ComputeMetadata, NodeMetadata>() {
/**
* If the result of {@link ListNodesStrategy#execute} is a set of nodes, then return them.
* Otherwise iteratively call {@link #getNodeMetadata}
*/
protected Map<String, ? extends NodeMetadata> doGetNodesWithTag(final String tag) {
Iterable<? extends NodeMetadata> nodes = Iterables.filter(Iterables.transform(
listNodesStrategy.execute(), new Function<ComputeMetadata, NodeMetadata>() {
@Override
public NodeMetadata apply(ComputeMetadata from) {
return getNodeMetadata(from);
return from instanceof NodeMetadata ? NodeMetadata.class.cast(from)
: getNodeMetadata(from);
}
}), new Predicate<NodeMetadata>() {
@ -257,7 +237,7 @@ public abstract class BaseComputeService implements ComputeService {
@Override
public Map<String, ? extends NodeMetadata> getNodesWithTag(String tag) {
logger.debug(">> listing nodes by tag(%s)", tag);
Map<String, ? extends NodeMetadata> nodes = doGetNodes(tag);
Map<String, ? extends NodeMetadata> nodes = doGetNodesWithTag(tag);
logger.debug("<< list(%d)", nodes.size());
return nodes;
}
@ -272,9 +252,55 @@ public abstract class BaseComputeService implements ComputeService {
return images.get();
}
@Override
public Map<String, ? extends Location> getLocations() {
return locations.get();
}
@Override
public TemplateBuilder templateBuilder() {
return templateBuilderProvider.get();
}
@Override
public NodeMetadata getNodeMetadata(ComputeMetadata node) {
checkArgument(node.getType() == ComputeType.NODE, "this is only valid for nodes, not "
+ node.getType());
return getNodeMetadataStrategy.execute(node);
}
@Override
public void rebootNode(ComputeMetadata node) {
checkArgument(node.getType() == ComputeType.NODE, "this is only valid for nodes, not "
+ node.getType());
checkNotNull(node.getId(), "node.id");
logger.debug(">> rebooting node(%s)", node.getId());
boolean successful = rebootNodeStrategy.execute(node);
logger.debug("<< rebooted node(%s) success(%s)", node.getId(), successful);
}
@Override
public void rebootNodesWithTag(String tag) { // TODO parallel
logger.debug(">> rebooting nodes by tag(%s)", tag);
Iterable<? extends NodeMetadata> nodesToReboot = Iterables.filter(doGetNodesWithTag(tag)
.values(), new Predicate<NodeMetadata>() {
@Override
public boolean apply(NodeMetadata input) {
return input.getState() != NodeState.TERMINATED;
}
});
Map<NodeMetadata, ListenableFuture<Void>> responses = Maps.newHashMap();
for (final NodeMetadata node : nodesToReboot) {
responses.put(node, makeListenable(executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
rebootNode(node);
return null;
}
}), executor));
}
awaitCompletion(responses, executor, null, logger, "rebooting nodes");
logger.debug("<< rebooted");
}
}

View File

@ -217,10 +217,12 @@ public class TemplateBuilderImpl implements TemplateBuilder {
};
static final Ordering<Image> DEFAULT_IMAGE_ORDERING = new Ordering<Image>() {
public int compare(Image left, Image right) {
return ComparisonChain.start().compare(left.getArchitecture(), right.getArchitecture()).compare(left.getName(), right.getName(),
return ComparisonChain.start().compare(left.getName(), right.getName(),
Ordering.<String> natural().nullsLast()).compare(left.getVersion(),
right.getVersion(), Ordering.<String> natural().nullsLast()).compare(left.getOsDescription(),
right.getOsDescription(), Ordering.<String> natural().nullsLast()).result();
right.getVersion(), Ordering.<String> natural().nullsLast()).compare(
left.getOsDescription(), right.getOsDescription(),
Ordering.<String> natural().nullsLast()).compare(left.getArchitecture(),
right.getArchitecture()).result();
}
};

View File

@ -0,0 +1,34 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.compute.strategy;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.Template;
/**
* Adds a node into an existing tag set, or creates one new.
*
* @author Adrian Cole
*/
public interface AddNodeWithTagStrategy {
NodeMetadata execute(String tag, String name, Template template);
}

View File

@ -0,0 +1,34 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.compute.strategy;
import org.jclouds.compute.domain.ComputeMetadata;
/**
* terminates the node and blocks until it is no longer visible or in the state TERMINATED. If this
* is the last node in a tagset, incidental resources are also destroyed.
*
* @author Adrian Cole
*/
public interface DestroyNodeStrategy {
boolean execute(ComputeMetadata node);
}

View File

@ -0,0 +1,34 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.compute.strategy;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.NodeMetadata;
/**
* returns all details associated to the node below.
*
* @author Adrian Cole
*/
public interface GetNodeMetadataStrategy {
NodeMetadata execute(ComputeMetadata node);
}

View File

@ -0,0 +1,32 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.compute.strategy;
import org.jclouds.compute.domain.ComputeMetadata;
/**
*
* @author Adrian Cole
*/
public interface ListNodesStrategy {
Iterable<? extends ComputeMetadata> execute();
}

View File

@ -0,0 +1,33 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.compute.strategy;
import org.jclouds.compute.domain.ComputeMetadata;
/**
* Reboots a node unless it is in the state TERMINATED.
*
* @author Adrian Cole
*/
public interface RebootNodeStrategy {
boolean execute(ComputeMetadata node);
}

View File

@ -0,0 +1,42 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.compute.strategy;
import java.util.Map;
import java.util.Set;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.strategy.impl.OneByOneRunNodesAndAddToSetStrategy;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.inject.ImplementedBy;
/**
* creates futures that correlate to
*
* @author Adrian Cole
*/
@ImplementedBy(OneByOneRunNodesAndAddToSetStrategy.class)
public interface RunNodesAndAddToSetStrategy {
Map<?, ListenableFuture<Void>> execute(String tag, int count, Template template,
Set<NodeMetadata> nodes);
}

View File

@ -0,0 +1,120 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.compute.strategy.impl;
import static org.jclouds.concurrent.ConcurrentUtils.makeListenable;
import java.security.SecureRandom;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Singleton;
import org.jclouds.Constants;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.internal.BaseComputeService;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.strategy.AddNodeWithTagStrategy;
import org.jclouds.compute.strategy.ListNodesStrategy;
import org.jclouds.compute.strategy.RunNodesAndAddToSetStrategy;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.logging.Logger;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
/**
* creates futures that correlate to
*
* @author Adrian Cole
*/
@Singleton
public class OneByOneRunNodesAndAddToSetStrategy implements RunNodesAndAddToSetStrategy {
@Resource
@Named(ComputeServiceConstants.COMPUTE_LOGGER)
protected Logger logger = Logger.NULL;
protected final AddNodeWithTagStrategy addNodeWithTagStrategy;
protected final ListNodesStrategy listNodesStrategy;
protected final String nodeNamingConvention;
protected final ComputeUtils utils;
protected final ExecutorService executor;
@Inject
protected OneByOneRunNodesAndAddToSetStrategy(AddNodeWithTagStrategy addNodeWithTagStrategy,
ListNodesStrategy listNodesStrategy,
@Named("NAMING_CONVENTION") String nodeNamingConvention, ComputeUtils utils,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
this.addNodeWithTagStrategy = addNodeWithTagStrategy;
this.listNodesStrategy = listNodesStrategy;
this.nodeNamingConvention = nodeNamingConvention;
this.utils = utils;
this.executor = executor;
}
@Override
public Map<?, ListenableFuture<Void>> execute(final String tag, final int count,
final Template template, final Set<NodeMetadata> nodes) {
Map<String, ListenableFuture<Void>> responses = Maps.newHashMap();
for (final String name : getNextNames(tag, count)) {
responses.put(name, makeListenable(executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
NodeMetadata node = null;
logger.debug(">> starting node(%s) tag(%s)", name, tag);
node = addNodeWithTagStrategy.execute(tag, name, template);
logger.debug("<< running node(%s)", node.getId());
try {
utils.runOptionsOnNode(node, template.getOptions());
logger.debug("<< options applied node(%s)", node.getId());
nodes.add(node);
} catch (Exception e) {
if (!template.getOptions().shouldDestroyOnError())
nodes.add(node);
}
return null;
}
}), executor));
}
return responses;
}
protected Set<String> getNextNames(final String tag, int count) {
Set<String> names = Sets.newHashSet();
int nodeIndex = new SecureRandom().nextInt(8096);
Map<String, ? extends ComputeMetadata> currentNodes = Maps.uniqueIndex(listNodesStrategy
.execute(), BaseComputeService.METADATA_TO_NAME);
while (names.size() < count) {
String name = String.format(nodeNamingConvention, tag, nodeIndex++);
if (!currentNodes.containsKey(name)) {
names.add(name);
}
}
return names;
}
}

View File

@ -165,6 +165,7 @@ public abstract class BaseComputeServiceLiveTest {
assertNotNull(node.getId());
assertNotNull(node.getTag());
assertEquals(node.getTag(), tag);
assertEquals(node.getState(), NodeState.RUNNING);
assert node.getPublicAddresses().size() >= 1 || node.getPrivateAddresses().size() >= 1 : "no ips in"
+ node;
assertNotNull(node.getCredentials());
@ -187,6 +188,7 @@ public abstract class BaseComputeServiceLiveTest {
NodeMetadata metadata = client.getNodeMetadata(node);
assertEquals(metadata.getId(), node.getId());
assertEquals(metadata.getName(), node.getName());
assertEquals(metadata.getState(), NodeState.RUNNING);
assertEquals(metadata.getPrivateAddresses(), node.getPrivateAddresses());
assertEquals(metadata.getPublicAddresses(), node.getPublicAddresses());
}
@ -199,6 +201,12 @@ public abstract class BaseComputeServiceLiveTest {
}
@Test(enabled = true, dependsOnMethods = "testGet")
public void testReboot() throws Exception {
client.rebootNodesWithTag(tag);
testGet();
}
public void testListNodes() throws Exception {
for (Entry<String, ? extends ComputeMetadata> node : client.getNodes().entrySet()) {
assertEquals(node.getKey(), node.getValue().getId());

View File

@ -81,23 +81,26 @@ public class JschSshClient implements SshClient {
private Session session;
private final byte[] privateKey;
final byte[] emptyPassPhrase = new byte[0];
private final int timeout;
@Inject
public JschSshClient(InetSocketAddress socket, String username, String password) {
public JschSshClient(InetSocketAddress socket, int timeout, String username, String password) {
this.host = checkNotNull(socket, "socket").getAddress();
checkArgument(socket.getPort() > 0, "ssh port must be greater then zero" + socket.getPort());
this.port = socket.getPort();
this.username = checkNotNull(username, "username");
this.password = checkNotNull(password, "password");
this.timeout = timeout;
this.privateKey = null;
}
@Inject
public JschSshClient(InetSocketAddress socket, String username, byte[] privateKey) {
public JschSshClient(InetSocketAddress socket, int timeout, String username, byte[] privateKey) {
this.host = checkNotNull(socket, "socket").getAddress();
checkArgument(socket.getPort() > 0, "ssh port must be greater then zero" + socket.getPort());
this.port = socket.getPort();
this.username = checkNotNull(username, "username");
this.timeout = timeout;
this.password = null;
this.privateKey = checkNotNull(privateKey, "privateKey");
}
@ -159,7 +162,8 @@ public class JschSshClient implements SshClient {
session = null;
try {
session = jsch.getSession(username, host.getHostAddress(), port);
session.setTimeout(120 * 1000);
if (timeout != 0)
session.setTimeout(timeout);
logger.debug("%s@%s:%d: Session created.", username, host.getHostAddress(), port);
if (password != null) {
session.setPassword(password);

View File

@ -22,6 +22,9 @@ import java.io.ByteArrayOutputStream;
import java.net.InetSocketAddress;
import java.util.Map;
import javax.inject.Named;
import org.jclouds.Constants;
import org.jclouds.ssh.ConfiguresSshClient;
import org.jclouds.ssh.SshClient;
import org.jclouds.ssh.jsch.JschSshClient;
@ -29,6 +32,7 @@ import org.jclouds.ssh.jsch.JschSshClient;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.inject.AbstractModule;
import com.google.inject.Inject;
import com.google.inject.Scopes;
import com.jcraft.jsch.JSch;
import com.jcraft.jsch.JSchException;
@ -46,13 +50,16 @@ public class JschSshClientModule extends AbstractModule {
}
private static class Factory implements SshClient.Factory {
@Named(Constants.PROPERTY_CONNECTION_TIMEOUT)
@Inject(optional=true)
int timeout = 60000;
public SshClient create(InetSocketAddress socket, String username, String password) {
return new JschSshClient(socket, username, password);
return new JschSshClient(socket, timeout, username, password);
}
public SshClient create(InetSocketAddress socket, String username, byte[] privateKey) {
return new JschSshClient(socket, username, privateKey);
return new JschSshClient(socket, timeout, username, privateKey);
}
@Override

View File

@ -1,121 +0,0 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.rackspace.cloudservers.compute;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Provider;
import javax.inject.Singleton;
import org.jclouds.Constants;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.ComputeType;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.domain.internal.NodeMetadataImpl;
import org.jclouds.compute.internal.BaseComputeService;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.domain.Credentials;
import org.jclouds.domain.Location;
import org.jclouds.rackspace.cloudservers.CloudServersClient;
import org.jclouds.rackspace.cloudservers.domain.Server;
import org.jclouds.rackspace.cloudservers.options.ListOptions;
import org.jclouds.rackspace.reference.RackspaceConstants;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
/**
* @author Adrian Cole
*/
@Singleton
public class CloudServersComputeService extends BaseComputeService {
private final CloudServersClient client;
private final Predicate<Server> serverActive;
private final Predicate<Server> serverDeleted;
private final Function<Server, NodeMetadata> serverToNodeMetadata;
@Inject
public CloudServersComputeService(Provider<TemplateBuilder> templateBuilderProvider,
Provider<Map<String, ? extends Image>> images,
Provider<Map<String, ? extends Size>> sizes,
Provider<Map<String, ? extends Location>> locations, ComputeUtils utils,
CloudServersClient client, Function<Server, NodeMetadata> serverToNodeMetadata,
@Named("ACTIVE") Predicate<Server> serverActive,
@Named("DELETED") Predicate<Server> serverDeleted,
@Named(RackspaceConstants.PROPERTY_RACKSPACE_USER) String account,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
super(images, sizes, locations, templateBuilderProvider, account + "-%s-%d", utils, executor);
this.client = client;
this.serverActive = serverActive;
this.serverDeleted = serverDeleted;
this.serverToNodeMetadata = serverToNodeMetadata;
}
@Override
protected NodeMetadata startNode(final String tag, final String name, final Template template) {
Server server = client.createServer(name, Integer.parseInt(template.getImage().getId()),
Integer.parseInt(template.getSize().getId()));
serverActive.apply(server);
return new NodeMetadataImpl(server.getId() + "", name, template.getLocation().getId(), null, server.getMetadata(), tag,
NodeState.RUNNING, server.getAddresses().getPublicAddresses(), server.getAddresses()
.getPrivateAddresses(), ImmutableMap.<String, String> of(),
new Credentials("root", server.getAdminPass()));
}
@Override
public NodeMetadata getNodeMetadata(ComputeMetadata node) {
checkArgument(node.getType() == ComputeType.NODE, "this is only valid for nodes, not "
+ node.getType());
checkNotNull(node.getId(), "node.id");
int serverId = Integer.parseInt(node.getId());
Server server = client.getServer(serverId);
return server == null ? null : serverToNodeMetadata.apply(server);
}
@Override
protected Iterable<NodeMetadata> doGetNodes() {
return Iterables.transform(client.listServers(ListOptions.Builder.withDetails()),
serverToNodeMetadata);
}
@Override
protected boolean doDestroyNode(ComputeMetadata node) {
int serverId = Integer.parseInt(node.getId());
// if false server wasn't around in the first place
if (!client.deleteServer(serverId))
return false;
Server server = client.getServer(serverId);
return server == null ? false : serverDeleted.apply(server);
}
}

View File

@ -42,12 +42,20 @@ import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.compute.domain.OsFamily;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.internal.ImageImpl;
import org.jclouds.compute.domain.internal.NodeMetadataImpl;
import org.jclouds.compute.domain.internal.SizeImpl;
import org.jclouds.compute.internal.BaseComputeService;
import org.jclouds.compute.internal.ComputeServiceContextImpl;
import org.jclouds.compute.predicates.RunScriptRunning;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.strategy.AddNodeWithTagStrategy;
import org.jclouds.compute.strategy.DestroyNodeStrategy;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.compute.strategy.ListNodesStrategy;
import org.jclouds.compute.strategy.RebootNodeStrategy;
import org.jclouds.domain.Credentials;
import org.jclouds.domain.Location;
import org.jclouds.domain.LocationScope;
import org.jclouds.domain.internal.LocationImpl;
@ -55,12 +63,13 @@ import org.jclouds.logging.Logger;
import org.jclouds.predicates.RetryablePredicate;
import org.jclouds.rackspace.cloudservers.CloudServersAsyncClient;
import org.jclouds.rackspace.cloudservers.CloudServersClient;
import org.jclouds.rackspace.cloudservers.compute.CloudServersComputeService;
import org.jclouds.rackspace.cloudservers.config.CloudServersContextModule;
import org.jclouds.rackspace.cloudservers.domain.Flavor;
import org.jclouds.rackspace.cloudservers.domain.RebootType;
import org.jclouds.rackspace.cloudservers.domain.Server;
import org.jclouds.rackspace.cloudservers.domain.ServerStatus;
import org.jclouds.rackspace.cloudservers.options.ListOptions;
import org.jclouds.rackspace.reference.RackspaceConstants;
import org.jclouds.rest.RestContext;
import org.jclouds.ssh.SshClient;
@ -69,14 +78,15 @@ import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.inject.Provides;
import com.google.inject.TypeLiteral;
/**
* Configures the {@link CloudServersComputeServiceContext}; requires
* {@link CloudServersComputeService} bound.
* Configures the {@link CloudServersComputeServiceContext}; requires {@link BaseComputeService}
* bound.
*
* @author Adrian Cole
*/
@ -85,9 +95,134 @@ public class CloudServersComputeServiceContextModule extends CloudServersContext
@Override
protected void configure() {
super.configure();
bind(ComputeService.class).to(CloudServersComputeService.class).asEagerSingleton();
bind(new TypeLiteral<Function<Server, NodeMetadata>>() {
}).to(ServerToNodeMetadata.class);
bind(AddNodeWithTagStrategy.class).to(CloudServersAddNodeWithTagStrategy.class);
bind(ListNodesStrategy.class).to(CloudServersListNodesStrategy.class);
bind(GetNodeMetadataStrategy.class).to(CloudServersGetNodeMetadataStrategy.class);
bind(RebootNodeStrategy.class).to(CloudServersRebootNodeStrategy.class);
bind(DestroyNodeStrategy.class).to(CloudServersDestroyNodeStrategy.class);
}
@Provides
@Named("NAMING_CONVENTION")
@Singleton
String provideNamingConvention(@Named(RackspaceConstants.PROPERTY_RACKSPACE_USER) String account) {
return account + "-%s-%d";
}
@Singleton
public static class CloudServersRebootNodeStrategy implements RebootNodeStrategy {
private final CloudServersClient client;
private final Predicate<Server> serverActive;
@Inject
protected CloudServersRebootNodeStrategy(CloudServersClient client,
@Named("ACTIVE") Predicate<Server> serverActive) {
this.client = client;
this.serverActive = serverActive;
}
@Override
public boolean execute(ComputeMetadata node) {
int serverId = Integer.parseInt(node.getId());
// if false server wasn't around in the first place
client.rebootServer(serverId, RebootType.HARD);
Server server = client.getServer(serverId);
return server == null ? false : serverActive.apply(server);
}
}
@Singleton
public static class CloudServersDestroyNodeStrategy implements DestroyNodeStrategy {
private final CloudServersClient client;
private final Predicate<Server> serverDeleted;
@Inject
protected CloudServersDestroyNodeStrategy(CloudServersClient client,
@Named("DELETED") Predicate<Server> serverDeleted) {
this.client = client;
this.serverDeleted = serverDeleted;
}
@Override
public boolean execute(ComputeMetadata node) {
int serverId = Integer.parseInt(node.getId());
// if false server wasn't around in the first place
if (!client.deleteServer(serverId))
return false;
Server server = client.getServer(serverId);
return server == null ? false : serverDeleted.apply(server);
}
}
@Singleton
public static class CloudServersAddNodeWithTagStrategy implements AddNodeWithTagStrategy {
private final CloudServersClient client;
private final Predicate<Server> serverActive;
@Inject
protected CloudServersAddNodeWithTagStrategy(CloudServersClient client,
@Named("ACTIVE") Predicate<Server> serverActive) {
this.client = client;
this.serverActive = serverActive;
}
@Override
public NodeMetadata execute(String tag, String name, Template template) {
Server server = client.createServer(name, Integer.parseInt(template.getImage().getId()),
Integer.parseInt(template.getSize().getId()));
serverActive.apply(server);
return new NodeMetadataImpl(server.getId() + "", name, template.getLocation().getId(),
null, server.getMetadata(), tag, NodeState.RUNNING, server.getAddresses()
.getPublicAddresses(), server.getAddresses().getPrivateAddresses(),
ImmutableMap.<String, String> of(),
new Credentials("root", server.getAdminPass()));
}
}
@Singleton
public static class CloudServersListNodesStrategy implements ListNodesStrategy {
private final CloudServersClient client;
private final Function<Server, NodeMetadata> serverToNodeMetadata;
@Inject
protected CloudServersListNodesStrategy(CloudServersClient client,
Function<Server, NodeMetadata> serverToNodeMetadata) {
this.client = client;
this.serverToNodeMetadata = serverToNodeMetadata;
}
@Override
public Iterable<? extends ComputeMetadata> execute() {
return Iterables.transform(client.listServers(ListOptions.Builder.withDetails()),
serverToNodeMetadata);
}
}
@Singleton
public static class CloudServersGetNodeMetadataStrategy implements GetNodeMetadataStrategy {
private final CloudServersClient client;
private final Function<Server, NodeMetadata> serverToNodeMetadata;
@Inject
protected CloudServersGetNodeMetadataStrategy(CloudServersClient client,
Function<Server, NodeMetadata> serverToNodeMetadata) {
this.client = client;
this.serverToNodeMetadata = serverToNodeMetadata;
}
@Override
public NodeMetadata execute(ComputeMetadata node) {
int serverId = Integer.parseInt(node.getId());
Server server = client.getServer(serverId);
return server == null ? null : serverToNodeMetadata.apply(server);
}
}
@Singleton

View File

@ -60,6 +60,5 @@ public class CloudServersComputeServiceLiveTest extends BaseComputeServiceLiveTe
@SuppressWarnings("unused")
RestContext<CloudServersAsyncClient, CloudServersClient> tmContext = new ComputeServiceContextFactory()
.createContext(service, user, password).getProviderSpecificContext();
CloudServersComputeService.class.cast(client);
}
}

View File

@ -1,129 +0,0 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.rimuhosting.miro.compute;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import java.net.InetAddress;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Provider;
import javax.inject.Singleton;
import org.jclouds.Constants;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.ComputeType;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.domain.internal.NodeMetadataImpl;
import org.jclouds.compute.internal.BaseComputeService;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.domain.Credentials;
import org.jclouds.domain.Location;
import org.jclouds.rimuhosting.miro.RimuHostingClient;
import org.jclouds.rimuhosting.miro.domain.NewServerResponse;
import org.jclouds.rimuhosting.miro.domain.Server;
import org.jclouds.rimuhosting.miro.domain.internal.RunningState;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
/**
* @author Ivan Meredith
*/
@Singleton
public class RimuHostingComputeService extends BaseComputeService {
private final RimuHostingClient client;
private final Function<Server, NodeMetadata> serverToNodeMetadata;
private final Map<RunningState, NodeState> runningStateToNodeState;
private final Predicate<Server> serverRunning;
private final Predicate<Server> serverDestroyed;
private final Function<Server, Iterable<InetAddress>> getPublicAddresses;
@Inject
public RimuHostingComputeService(Provider<TemplateBuilder> templateBuilderProvider,
Provider<Map<String, ? extends Image>> images,
Provider<Map<String, ? extends Size>> sizes,
Provider<Map<String, ? extends Location>> locations, ComputeUtils utils,
RimuHostingClient client, Map<RunningState, NodeState> runningStateToNodeState,
@Named("RUNNING") Predicate<Server> serverRunning,
@Named("DESTROYED") Predicate<Server> serverDestroyed,
Function<Server, NodeMetadata> serverToNodeMetadata,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor,
Function<Server, Iterable<InetAddress>> getPublicAddresses) {
super(images, sizes, locations, templateBuilderProvider, "%s-%d", utils, executor);
this.client = client;
this.runningStateToNodeState = runningStateToNodeState;
this.serverRunning = serverRunning;
this.serverDestroyed = serverDestroyed;
this.serverToNodeMetadata = serverToNodeMetadata;
this.getPublicAddresses = getPublicAddresses;
}
@Override
protected NodeMetadata startNode(final String tag, final String name, final Template template) {
NewServerResponse serverResponse = client.createServer(name, checkNotNull(template.getImage()
.getId(), "imageId"), checkNotNull(template.getSize().getId(), "sizeId"));
serverRunning.apply(serverResponse.getServer());
Server server = client.getServer(serverResponse.getServer().getId());
// we have to lookup the new details in order to retrieve the currently assigned ip address.
NodeMetadata node = new NodeMetadataImpl(server.getId().toString(), name, template
.getLocation().getId(), null, ImmutableMap.<String, String> of(), tag,
runningStateToNodeState.get(server.getState()), getPublicAddresses.apply(server),
ImmutableList.<InetAddress> of(), ImmutableMap.<String, String> of(),
new Credentials("root", serverResponse.getNewInstanceRequest().getCreateOptions()
.getPassword()));
return node;
}
@Override
public NodeMetadata getNodeMetadata(ComputeMetadata node) {
checkArgument(node.getType() == ComputeType.NODE, "this is only valid for nodes, not "
+ node.getType());
checkNotNull(node.getId(), "node.id");
long serverId = Long.parseLong(node.getId());
Server server = client.getServer(serverId);
return server == null ? null : serverToNodeMetadata.apply(server);
}
@Override
protected Iterable<NodeMetadata> doGetNodes() {
return Iterables.transform(client.getServerList(), serverToNodeMetadata);
}
@Override
protected boolean doDestroyNode(ComputeMetadata node) {
long serverId = Long.parseLong(node.getId());
client.destroyServer(serverId);
return serverDestroyed.apply(client.getServer(serverId));
}
}

View File

@ -18,6 +18,8 @@
*/
package org.jclouds.rimuhosting.miro.compute.config;
import static com.google.common.base.Preconditions.checkNotNull;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.Map;
@ -44,12 +46,18 @@ import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.compute.domain.OsFamily;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.internal.ImageImpl;
import org.jclouds.compute.domain.internal.NodeMetadataImpl;
import org.jclouds.compute.domain.internal.SizeImpl;
import org.jclouds.compute.internal.ComputeServiceContextImpl;
import org.jclouds.compute.predicates.RunScriptRunning;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.strategy.AddNodeWithTagStrategy;
import org.jclouds.compute.strategy.DestroyNodeStrategy;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.compute.strategy.ListNodesStrategy;
import org.jclouds.compute.strategy.RebootNodeStrategy;
import org.jclouds.domain.Credentials;
import org.jclouds.domain.Location;
import org.jclouds.domain.LocationScope;
@ -59,8 +67,8 @@ import org.jclouds.predicates.RetryablePredicate;
import org.jclouds.rest.RestContext;
import org.jclouds.rimuhosting.miro.RimuHostingAsyncClient;
import org.jclouds.rimuhosting.miro.RimuHostingClient;
import org.jclouds.rimuhosting.miro.compute.RimuHostingComputeService;
import org.jclouds.rimuhosting.miro.config.RimuHostingContextModule;
import org.jclouds.rimuhosting.miro.domain.NewServerResponse;
import org.jclouds.rimuhosting.miro.domain.PricingPlan;
import org.jclouds.rimuhosting.miro.domain.Server;
import org.jclouds.rimuhosting.miro.domain.internal.RunningState;
@ -89,11 +97,139 @@ public class RimuHostingComputeServiceContextModule extends RimuHostingContextMo
@Override
protected void configure() {
super.configure();
bind(ComputeService.class).to(RimuHostingComputeService.class).asEagerSingleton();
bind(new TypeLiteral<Function<Server, NodeMetadata>>() {
}).to(ServerToNodeMetadata.class);
bind(new TypeLiteral<Function<Server, Iterable<InetAddress>>>() {
}).to(ServerToPublicAddresses.class);
bind(AddNodeWithTagStrategy.class).to(RimuHostingAddNodeWithTagStrategy.class);
bind(ListNodesStrategy.class).to(RimuHostingListNodesStrategy.class);
bind(GetNodeMetadataStrategy.class).to(RimuHostingGetNodeMetadataStrategy.class);
bind(RebootNodeStrategy.class).to(RimuHostingRebootNodeStrategy.class);
bind(DestroyNodeStrategy.class).to(RimuHostingDestroyNodeStrategy.class);
}
@Provides
@Named("NAMING_CONVENTION")
@Singleton
String provideNamingConvention() {
return "%s-%d";
}
@Singleton
public static class RimuHostingRebootNodeStrategy implements RebootNodeStrategy {
private final RimuHostingClient client;
@Inject
protected RimuHostingRebootNodeStrategy(RimuHostingClient client) {
this.client = client;
}
@Override
public boolean execute(ComputeMetadata node) {
Long serverId = Long.parseLong(node.getId());
// if false server wasn't around in the first place
return client.restartServer(serverId).getState() == RunningState.RUNNING;
}
}
@Singleton
public static class RimuHostingDestroyNodeStrategy implements DestroyNodeStrategy {
private final RimuHostingClient client;
private final Predicate<Server> serverDestroyed;
@Inject
protected RimuHostingDestroyNodeStrategy(RimuHostingClient client,
@Named("DESTROYED") Predicate<Server> serverDestroyed) {
this.client = client;
this.serverDestroyed = serverDestroyed;
}
@Override
public boolean execute(ComputeMetadata node) {
long serverId = Long.parseLong(node.getId());
client.destroyServer(serverId);
return serverDestroyed.apply(client.getServer(serverId));
}
}
@Singleton
public static class RimuHostingAddNodeWithTagStrategy implements AddNodeWithTagStrategy {
private final RimuHostingClient client;
private final Predicate<Server> serverRunning;
private final Function<Server, Iterable<InetAddress>> getPublicAddresses;
private final Map<RunningState, NodeState> runningStateToNodeState;
@Inject
protected RimuHostingAddNodeWithTagStrategy(RimuHostingClient client,
@Named("RUNNING") Predicate<Server> serverRunning,
Function<Server, Iterable<InetAddress>> getPublicAddresses,
Map<RunningState, NodeState> runningStateToNodeState) {
this.client = client;
this.serverRunning = serverRunning;
this.getPublicAddresses = getPublicAddresses;
this.runningStateToNodeState = runningStateToNodeState;
}
@Override
public NodeMetadata execute(String tag, String name, Template template) {
NewServerResponse serverResponse = client.createServer(name, checkNotNull(template
.getImage().getId(), "imageId"), checkNotNull(template.getSize().getId(),
"sizeId"));
serverRunning.apply(serverResponse.getServer());
Server server = client.getServer(serverResponse.getServer().getId());
// we have to lookup the new details in order to retrieve the currently assigned ip
// address.
NodeMetadata node = new NodeMetadataImpl(server.getId().toString(), name, template
.getLocation().getId(), null, ImmutableMap.<String, String> of(), tag,
runningStateToNodeState.get(server.getState()), getPublicAddresses.apply(server),
ImmutableList.<InetAddress> of(), ImmutableMap.<String, String> of(),
new Credentials("root", serverResponse.getNewInstanceRequest().getCreateOptions()
.getPassword()));
return node;
}
}
@Singleton
public static class RimuHostingListNodesStrategy implements ListNodesStrategy {
private final RimuHostingClient client;
private final Function<Server, NodeMetadata> serverToNodeMetadata;
@Inject
protected RimuHostingListNodesStrategy(RimuHostingClient client,
Function<Server, NodeMetadata> serverToNodeMetadata) {
this.client = client;
this.serverToNodeMetadata = serverToNodeMetadata;
}
@Override
public Iterable<? extends ComputeMetadata> execute() {
return Iterables.transform(client.getServerList(), serverToNodeMetadata);
}
}
@Singleton
public static class RimuHostingGetNodeMetadataStrategy implements GetNodeMetadataStrategy {
private final RimuHostingClient client;
private final Function<Server, NodeMetadata> serverToNodeMetadata;
@Inject
protected RimuHostingGetNodeMetadataStrategy(RimuHostingClient client,
Function<Server, NodeMetadata> serverToNodeMetadata) {
this.client = client;
this.serverToNodeMetadata = serverToNodeMetadata;
}
@Override
public NodeMetadata execute(ComputeMetadata node) {
long serverId = Long.parseLong(node.getId());
Server server = client.getServer(serverId);
return server == null ? null : serverToNodeMetadata.apply(server);
}
}
@Singleton

View File

@ -18,45 +18,27 @@
*/
package org.jclouds.vcloud.compute;
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 org.jclouds.vcloud.options.InstantiateVAppTemplateOptions.Builder.processorCount;
import java.net.InetAddress;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import javax.annotation.Resource;
import javax.inject.Named;
import javax.inject.Provider;
import javax.inject.Singleton;
import org.jclouds.Constants;
import org.jclouds.compute.ComputeService;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.ComputeType;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.domain.internal.NodeMetadataImpl;
import org.jclouds.compute.internal.BaseComputeService;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.domain.Credentials;
import org.jclouds.domain.Location;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.logging.Logger;
import org.jclouds.vcloud.VCloudClient;
import org.jclouds.vcloud.VCloudMediaType;
import org.jclouds.vcloud.domain.NamedResource;
import org.jclouds.vcloud.domain.Task;
import org.jclouds.vcloud.domain.VApp;
import org.jclouds.vcloud.domain.VAppStatus;
import org.jclouds.vcloud.options.InstantiateVAppTemplateOptions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
@ -66,8 +48,10 @@ import com.google.inject.Inject;
* @author Adrian Cole
*/
@Singleton
public class VCloudComputeService extends BaseComputeService implements ComputeService,
VCloudComputeClient {
public class BaseVCloudComputeClient implements VCloudComputeClient {
@Resource
@Named(ComputeServiceConstants.COMPUTE_LOGGER)
protected Logger logger = Logger.NULL;
protected final VCloudClient client;
protected final Predicate<String> taskTester;
@ -75,76 +59,15 @@ public class VCloudComputeService extends BaseComputeService implements ComputeS
protected final Map<VAppStatus, NodeState> vAppStatusToNodeState;
@Inject
public VCloudComputeService(Provider<TemplateBuilder> templateBuilderProvider,
Provider<Map<String, ? extends Image>> images,
Provider<Map<String, ? extends Size>> sizes,
Provider<Map<String, ? extends Location>> locations, ComputeUtils utils,
VCloudClient client, Predicate<String> successTester,
public BaseVCloudComputeClient(VCloudClient client, Predicate<String> successTester,
@Named("NOT_FOUND") Predicate<VApp> notFoundTester,
Map<VAppStatus, NodeState> vAppStatusToNodeState,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
super(images, sizes, locations, templateBuilderProvider, "%s-%d", utils, executor);
Map<VAppStatus, NodeState> vAppStatusToNodeState) {
this.client = client;
this.taskTester = successTester;
this.notFoundTester = notFoundTester;
this.vAppStatusToNodeState = vAppStatusToNodeState;
}
@Override
protected NodeMetadata startNode(final String tag, final String name, final Template template) {
Map<String, String> metaMap = start(template.getLocation().getId(), name, template.getImage()
.getId(), template.getSize().getCores(), template.getSize().getRam(), template
.getSize().getDisk() * 1024 * 1024l, ImmutableMap.<String, String> of(), template
.getOptions().getInboundPorts());
VApp vApp = client.getVApp(metaMap.get("id"));
return newCreateNodeResponse(tag, template, metaMap, vApp);
}
protected NodeMetadata newCreateNodeResponse(String tag, Template template,
Map<String, String> metaMap, VApp vApp) {
return new NodeMetadataImpl(vApp.getId(), vApp.getName(), template.getLocation().getId(),
vApp.getLocation(), ImmutableMap.<String, String> of(), tag, vAppStatusToNodeState
.get(vApp.getStatus()), getPublicAddresses(vApp.getId()), vApp
.getNetworkToAddresses().values(), ImmutableMap.<String, String> of(),
new Credentials(metaMap.get("username"), metaMap.get("password")));
}
@Override
public NodeMetadata getNodeMetadata(ComputeMetadata node) {
checkArgument(node.getType() == ComputeType.NODE, "this is only valid for nodes, not "
+ node.getType());
return getNodeMetadataByIdInVDC(checkNotNull(node.getLocationId(), "location"), checkNotNull(
node.getId(), "node.id"));
}
protected NodeMetadata getNodeMetadataByIdInVDC(String vDCId, String id) {
VApp vApp = client.getVApp(id);
String tag = vApp.getName().replaceAll("-[0-9]+", "");
return new NodeMetadataImpl(vApp.getId(), vApp.getName(), vDCId, vApp.getLocation(),
ImmutableMap.<String, String> of(), tag,
vAppStatusToNodeState.get(vApp.getStatus()), vApp.getNetworkToAddresses().values(),
ImmutableSet.<InetAddress> of(), ImmutableMap.<String, String> of(), null);
}
@Override
protected Iterable<? extends ComputeMetadata> doGetNodes() {
Set<ComputeMetadata> nodes = Sets.newHashSet();
for (NamedResource vdc : client.getDefaultOrganization().getVDCs().values()) {
for (NamedResource resource : client.getVDC(vdc.getId()).getResourceEntities().values()) {
if (resource.getType().equals(VCloudMediaType.VAPP_XML)) {
nodes.add(getNodeMetadataByIdInVDC(vdc.getId(), resource.getId()));
}
}
}
return nodes;
}
@Override
protected boolean doDestroyNode(ComputeMetadata node) {
stop(checkNotNull(node.getId(), "node.id"));
return true;
}
public Map<String, String> start(String vDCId, String name, String templateId, int minCores,
int minMegs, Long diskSize, Map<String, String> properties, int... portsToOpen) {
logger

View File

@ -24,10 +24,13 @@ import java.util.Set;
import javax.annotation.Nullable;
import com.google.inject.ImplementedBy;
/**
*
* @author Adrian Cole
*/
@ImplementedBy(BaseVCloudComputeClient.class)
public interface VCloudComputeClient {
/**
* Runs through all commands necessary to startup a vApp, opening at least one ip address to the

View File

@ -18,6 +18,9 @@
*/
package org.jclouds.vcloud.compute.config;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
@ -27,6 +30,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Singleton;
@ -35,16 +39,26 @@ import org.jclouds.compute.ComputeService;
import org.jclouds.compute.ComputeServiceContext;
import org.jclouds.compute.domain.Architecture;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.ComputeType;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.compute.domain.OsFamily;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.internal.ImageImpl;
import org.jclouds.compute.domain.internal.NodeMetadataImpl;
import org.jclouds.compute.domain.internal.SizeImpl;
import org.jclouds.compute.internal.ComputeServiceContextImpl;
import org.jclouds.compute.predicates.RunScriptRunning;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.strategy.AddNodeWithTagStrategy;
import org.jclouds.compute.strategy.DestroyNodeStrategy;
import org.jclouds.compute.strategy.GetNodeMetadataStrategy;
import org.jclouds.compute.strategy.ListNodesStrategy;
import org.jclouds.compute.strategy.RebootNodeStrategy;
import org.jclouds.concurrent.ConcurrentUtils;
import org.jclouds.domain.Credentials;
import org.jclouds.domain.Location;
import org.jclouds.domain.LocationScope;
import org.jclouds.domain.internal.LocationImpl;
@ -55,12 +69,14 @@ import org.jclouds.ssh.SshClient;
import org.jclouds.vcloud.VCloudAsyncClient;
import org.jclouds.vcloud.VCloudClient;
import org.jclouds.vcloud.VCloudMediaType;
import org.jclouds.vcloud.compute.BaseVCloudComputeClient;
import org.jclouds.vcloud.compute.VCloudComputeClient;
import org.jclouds.vcloud.compute.VCloudComputeService;
import org.jclouds.vcloud.config.VCloudContextModule;
import org.jclouds.vcloud.domain.Catalog;
import org.jclouds.vcloud.domain.CatalogItem;
import org.jclouds.vcloud.domain.NamedResource;
import org.jclouds.vcloud.domain.Task;
import org.jclouds.vcloud.domain.VApp;
import org.jclouds.vcloud.domain.VAppStatus;
import org.jclouds.vcloud.domain.VAppTemplate;
import org.jclouds.vcloud.domain.VDC;
@ -74,11 +90,11 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.inject.Injector;
import com.google.inject.Provides;
/**
* Configures the {@link VCloudComputeServiceContext}; requires {@link VCloudComputeService} bound.
* Configures the {@link VCloudComputeServiceContext}; requires {@link BaseVCloudComputeClient}
* bound.
*
* @author Adrian Cole
*/
@ -87,28 +103,172 @@ public class VCloudComputeServiceContextModule extends VCloudContextModule {
@Singleton
@Provides
Map<VAppStatus, NodeState> provideVAppStatusToNodeState() {
return ImmutableMap
.<VAppStatus, NodeState> builder().put(VAppStatus.OFF, NodeState.TERMINATED).put(
VAppStatus.ON, NodeState.RUNNING).put(VAppStatus.RESOLVED, NodeState.PENDING).put(
VAppStatus.SUSPENDED, NodeState.SUSPENDED).put(VAppStatus.UNRESOLVED,
NodeState.PENDING).build();
return ImmutableMap.<VAppStatus, NodeState> builder().put(VAppStatus.OFF,
NodeState.TERMINATED).put(VAppStatus.ON, NodeState.RUNNING).put(VAppStatus.RESOLVED,
NodeState.PENDING).put(VAppStatus.SUSPENDED, NodeState.SUSPENDED).put(
VAppStatus.UNRESOLVED, NodeState.PENDING).build();
}
@Override
protected void configure() {
super.configure();
bind(AddNodeWithTagStrategy.class).to(VCloudAddNodeWithTagStrategy.class);
bind(ListNodesStrategy.class).to(VCloudListNodesStrategy.class);
bind(GetNodeMetadataStrategy.class).to(VCloudGetNodeMetadataStrategy.class);
bind(RebootNodeStrategy.class).to(VCloudRebootNodeStrategy.class);
bind(DestroyNodeStrategy.class).to(VCloudDestroyNodeStrategy.class);
}
@Provides
@Named("NAMING_CONVENTION")
@Singleton
protected ComputeService provideComputeService(Injector injector) {
return injector.getInstance(VCloudComputeService.class);
String provideNamingConvention() {
return "%s-%d";
}
@Provides
@Singleton
protected VCloudComputeClient provideComputeClient(VCloudComputeService in) {
return in;
public static class VCloudRebootNodeStrategy implements RebootNodeStrategy {
private final VCloudClient client;
protected final Predicate<String> taskTester;
@Inject
protected VCloudRebootNodeStrategy(VCloudClient client, Predicate<String> taskTester) {
this.client = client;
this.taskTester = taskTester;
}
@Override
public boolean execute(ComputeMetadata node) {
Task task = client.resetVApp(node.getId());
return taskTester.apply(task.getId());
}
}
@Singleton
public static class VCloudDestroyNodeStrategy implements DestroyNodeStrategy {
protected final VCloudComputeClient computeClient;
@Inject
protected VCloudDestroyNodeStrategy(VCloudComputeClient computeClient) {
this.computeClient = computeClient;
}
@Override
public boolean execute(ComputeMetadata node) {
computeClient.stop(checkNotNull(node.getId(), "node.id"));
return true;
}
}
@Singleton
public static class VCloudAddNodeWithTagStrategy implements AddNodeWithTagStrategy {
protected final VCloudClient client;
protected final VCloudComputeClient computeClient;
protected final Map<VAppStatus, NodeState> vAppStatusToNodeState;
@Inject
protected VCloudAddNodeWithTagStrategy(VCloudClient client,
VCloudComputeClient computeClient, Map<VAppStatus, NodeState> vAppStatusToNodeState) {
super();
this.client = client;
this.computeClient = computeClient;
this.vAppStatusToNodeState = vAppStatusToNodeState;
}
@Override
public NodeMetadata execute(String tag, String name, Template template) {
Map<String, String> metaMap = computeClient.start(template.getLocation().getId(), name,
template.getImage().getId(), template.getSize().getCores(), template.getSize()
.getRam(), template.getSize().getDisk() * 1024 * 1024l, ImmutableMap
.<String, String> of(), template.getOptions().getInboundPorts());
VApp vApp = client.getVApp(metaMap.get("id"));
return newCreateNodeResponse(tag, template, metaMap, vApp);
}
protected NodeMetadata newCreateNodeResponse(String tag, Template template,
Map<String, String> metaMap, VApp vApp) {
return new NodeMetadataImpl(vApp.getId(), vApp.getName(), template.getLocation().getId(),
vApp.getLocation(), ImmutableMap.<String, String> of(), tag,
vAppStatusToNodeState.get(vApp.getStatus()), computeClient
.getPublicAddresses(vApp.getId()), computeClient
.getPrivateAddresses(vApp.getId()), ImmutableMap.<String, String> of(),
new Credentials(metaMap.get("username"), metaMap.get("password")));
}
}
@Singleton
public static class VCloudListNodesStrategy extends VCloudGetNodeMetadata implements
ListNodesStrategy {
@Inject
protected VCloudListNodesStrategy(VCloudClient client, VCloudComputeClient computeClient,
Map<VAppStatus, NodeState> vAppStatusToNodeState) {
super(client, computeClient, vAppStatusToNodeState);
}
@Override
public Iterable<? extends ComputeMetadata> execute() {
Set<ComputeMetadata> nodes = Sets.newHashSet();
for (NamedResource vdc : client.getDefaultOrganization().getVDCs().values()) {
for (NamedResource resource : client.getVDC(vdc.getId()).getResourceEntities().values()) {
if (resource.getType().equals(VCloudMediaType.VAPP_XML)) {
nodes.add(getNodeMetadataByIdInVDC(vdc.getId(), resource.getId()));
}
}
}
return nodes;
}
}
@Singleton
public static class VCloudGetNodeMetadataStrategy extends VCloudGetNodeMetadata implements
GetNodeMetadataStrategy {
@Inject
protected VCloudGetNodeMetadataStrategy(VCloudClient client,
VCloudComputeClient computeClient, Map<VAppStatus, NodeState> vAppStatusToNodeState) {
super(client, computeClient, vAppStatusToNodeState);
}
@Override
public NodeMetadata execute(ComputeMetadata node) {
checkArgument(node.getType() == ComputeType.NODE, "this is only valid for nodes, not "
+ node.getType());
return getNodeMetadataByIdInVDC(checkNotNull(node.getLocationId(), "location"),
checkNotNull(node.getId(), "node.id"));
}
}
@Singleton
public static class VCloudGetNodeMetadata {
protected final VCloudClient client;
protected final VCloudComputeClient computeClient;
protected final Map<VAppStatus, NodeState> vAppStatusToNodeState;
@Inject
protected VCloudGetNodeMetadata(VCloudClient client, VCloudComputeClient computeClient,
Map<VAppStatus, NodeState> vAppStatusToNodeState) {
this.client = client;
this.computeClient = computeClient;
this.vAppStatusToNodeState = vAppStatusToNodeState;
}
protected NodeMetadata getNodeMetadataByIdInVDC(String vDCId, String id) {
VApp vApp = client.getVApp(id);
String tag = vApp.getName().replaceAll("-[0-9]+", "");
return new NodeMetadataImpl(vApp.getId(), vApp.getName(), vDCId, vApp.getLocation(),
ImmutableMap.<String, String> of(), tag, vAppStatusToNodeState.get(vApp
.getStatus()), computeClient.getPublicAddresses(id), computeClient
.getPrivateAddresses(id), ImmutableMap.<String, String> of(), null);
}
}
@Provides

View File

@ -46,7 +46,7 @@ public class VCloudComputeServiceLiveTest extends BaseComputeServiceLiveTest {
RestContext<VCloudAsyncClient, VCloudClient> tmContext = new ComputeServiceContextFactory()
.createContext(service, user, password).getProviderSpecificContext();
VCloudComputeService computeService = VCloudComputeService.class.cast(client);
BaseVCloudComputeClient computeService = BaseVCloudComputeClient.class.cast(client);
@SuppressWarnings("unused")
VCloudComputeClient computeClient = VCloudComputeClient.class.cast(computeService);

View File

@ -0,0 +1,44 @@
package org.jclouds.vcloud.hostingdotcom.compute;
import static com.google.common.base.Preconditions.checkState;
import java.util.Map;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Singleton;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.vcloud.VCloudClient;
import org.jclouds.vcloud.compute.BaseVCloudComputeClient;
import org.jclouds.vcloud.domain.VApp;
import org.jclouds.vcloud.domain.VAppStatus;
import org.jclouds.vcloud.hostingdotcom.domain.HostingDotComVApp;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
/**
* @author Adrian Cole
*/
@Singleton
public class HostingDotComVCloudComputeClient extends BaseVCloudComputeClient {
@Inject
protected HostingDotComVCloudComputeClient(VCloudClient client,
Predicate<String> successTester, @Named("NOT_FOUND") Predicate<VApp> notFoundTester,
Map<VAppStatus, NodeState> vAppStatusToNodeState) {
super(client, successTester, notFoundTester, vAppStatusToNodeState);
}
@Override
protected Map<String, String> parseResponse(VApp vAppResponse) {
checkState(vAppResponse instanceof HostingDotComVApp,
"bad configuration, vApp should be an instance of "
+ HostingDotComVApp.class.getName());
HostingDotComVApp hVApp = HostingDotComVApp.class.cast(vAppResponse);
return ImmutableMap.<String, String> of("id", vAppResponse.getId(), "username", hVApp
.getUsername(), "password", hVApp.getPassword());
}
}

View File

@ -1,57 +0,0 @@
package org.jclouds.vcloud.hostingdotcom.compute;
import static com.google.common.base.Preconditions.checkState;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Provider;
import javax.inject.Singleton;
import org.jclouds.Constants;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.domain.Location;
import org.jclouds.vcloud.VCloudClient;
import org.jclouds.vcloud.compute.VCloudComputeService;
import org.jclouds.vcloud.domain.VApp;
import org.jclouds.vcloud.domain.VAppStatus;
import org.jclouds.vcloud.hostingdotcom.domain.HostingDotComVApp;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
/**
* @author Adrian Cole
*/
@Singleton
public class HostingDotComVCloudComputeService extends VCloudComputeService {
@Inject
HostingDotComVCloudComputeService(Provider<TemplateBuilder> templateBuilderProvider,
Provider<Map<String, ? extends Image>> images,
Provider<Map<String, ? extends Size>> sizes,
Provider<Map<String, ? extends Location>> locations, ComputeUtils utils,
VCloudClient client, Predicate<String> successTester,
@Named("NOT_FOUND") Predicate<VApp> notFoundTester,
Map<VAppStatus, NodeState> vAppStatusToNodeState,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
super(templateBuilderProvider, images, sizes, locations, utils, client, successTester,
notFoundTester, vAppStatusToNodeState, executor);
}
@Override
protected Map<String, String> parseResponse(VApp vAppResponse) {
checkState(vAppResponse instanceof HostingDotComVApp,
"bad configuration, vApp should be an instance of "
+ HostingDotComVApp.class.getName());
HostingDotComVApp hVApp = HostingDotComVApp.class.cast(vAppResponse);
return ImmutableMap.<String, String> of("id", vAppResponse.getId(), "username", hVApp
.getUsername(), "password", hVApp.getPassword());
}
}

View File

@ -18,15 +18,13 @@
*/
package org.jclouds.vcloud.hostingdotcom.compute.config;
import org.jclouds.compute.ComputeService;
import org.jclouds.vcloud.compute.VCloudComputeClient;
import org.jclouds.vcloud.compute.config.VCloudComputeServiceContextModule;
import org.jclouds.vcloud.hostingdotcom.compute.HostingDotComVCloudComputeService;
import com.google.inject.Injector;
import org.jclouds.vcloud.hostingdotcom.compute.HostingDotComVCloudComputeClient;
/**
* Configures the {@link HostingDotComVCloudComputeServiceContext}; requires
* {@link HostingDotComVCloudComputeService} bound.
* {@link HostingDotComVCloudComputeClient} bound.
*
* @author Adrian Cole
*/
@ -36,11 +34,7 @@ public class HostingDotComVCloudComputeServiceContextModule extends
@Override
protected void configure() {
super.configure();
}
@Override
protected ComputeService provideComputeService(Injector injector) {
return injector.getInstance(HostingDotComVCloudComputeService.class);
bind(VCloudComputeClient.class).to(HostingDotComVCloudComputeClient.class);
}
}

View File

@ -54,7 +54,7 @@ public class HostingDotComVCloudComputeClientLiveTest extends VCloudComputeClien
Injector injector = new HostingDotComVCloudContextBuilder(
new HostingDotComVCloudPropertiesBuilder(account, key).build()).withModules(
new Log4JLoggingModule(), new JschSshClientModule()).buildInjector();
computeClient = injector.getInstance(HostingDotComVCloudComputeService.class);
computeClient = injector.getInstance(HostingDotComVCloudComputeClient.class);
client = injector.getInstance(HostingDotComVCloudClient.class);
addressTester = injector.getInstance(Key.get(new TypeLiteral<Predicate<InetAddress>>() {
}));

View File

@ -18,23 +18,19 @@
*/
package org.jclouds.vcloud.terremark.compute;
import static com.google.common.base.Preconditions.checkArgument;
import static org.jclouds.vcloud.terremark.options.AddInternetServiceOptions.Builder.withDescription;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Singleton;
import org.jclouds.compute.domain.OsFamily;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.logging.Logger;
import org.jclouds.vcloud.domain.Task;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.vcloud.compute.BaseVCloudComputeClient;
import org.jclouds.vcloud.domain.VApp;
import org.jclouds.vcloud.domain.VAppStatus;
import org.jclouds.vcloud.terremark.TerremarkVCloudClient;
@ -42,7 +38,6 @@ import org.jclouds.vcloud.terremark.domain.InternetService;
import org.jclouds.vcloud.terremark.domain.Node;
import org.jclouds.vcloud.terremark.domain.Protocol;
import org.jclouds.vcloud.terremark.domain.PublicIpAddress;
import org.jclouds.vcloud.terremark.options.TerremarkInstantiateVAppTemplateOptions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
@ -50,98 +45,39 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
/**
*
* @author Adrian Cole
*/
public class TerremarkVCloudComputeClient {
@Resource
@Named(ComputeServiceConstants.COMPUTE_LOGGER)
Logger logger = Logger.NULL;
@Singleton
public class TerremarkVCloudComputeClient extends BaseVCloudComputeClient {
private final Predicate<String> taskTester;
private final TerremarkVCloudClient tmClient;
private final TerremarkVCloudClient client;
@Inject
public TerremarkVCloudComputeClient(TerremarkVCloudClient tmClient,
Predicate<String> successTester) {
this.tmClient = tmClient;
this.taskTester = successTester;
protected TerremarkVCloudComputeClient(TerremarkVCloudClient client,
Predicate<String> successTester, @Named("NOT_FOUND") Predicate<VApp> notFoundTester,
Map<VAppStatus, NodeState> vAppStatusToNodeState) {
super(client, successTester, notFoundTester, vAppStatusToNodeState);
this.client = client;
}
private Map<OsFamily, String> imageCatalogIdMap = ImmutableMap
.<OsFamily, String> builder().put(OsFamily.CENTOS, "6").put(
OsFamily.RHEL, "8").put(OsFamily.UBUNTU, "10").build();
// .put( OperatingSystem.UBUNTU_JEOS_90, "11").put(OperatingSystem.WEBAPPVM_93, "29")
public String start(String name, OsFamily image, int minCores, int minMegs,
Map<String, String> properties) {
checkArgument(imageCatalogIdMap.containsKey(image), "image not configured: " + image);
return start(name, imageCatalogIdMap.get(image), minCores, minMegs, properties);
@Override
protected Map<String, String> parseResponse(VApp vAppResponse) {
return ImmutableMap.<String, String> of("id", vAppResponse.getId(), "username", "vcloud",
"password", "p4ssw0rd");
}
public String start(String name, String templateId, int minCores, int minMegs,
Map<String, String> properties) {
return this.start(tmClient.getDefaultVDC().getId(), name, templateId, minCores, minMegs,
properties);
@Override
public Map<String, String> start(String vDCId, String name, String templateId, int minCores,
int minMegs, Long diskSize, Map<String, String> properties, int... portsToOpen) {
Map<String, String> response = super.start(vDCId, name, templateId, minCores, minMegs, null,
properties, portsToOpen);// trmk does not support resizing the primary disk
if (portsToOpen.length > 0)
createPublicAddressMappedToPorts(response.get("id"), portsToOpen);
return response;
}
public String start(String vDCId, String name, String templateId, int minCores, int minMegs,
Map<String, String> properties) {
logger
.debug(
">> instantiating vApp vDC(%s) template(%s) name(%s) minCores(%d) minMegs(%d) properties(%s)",
vDCId, templateId, name, minCores, minMegs, properties);
VApp vApp = tmClient.instantiateVAppTemplateInVDC(vDCId, name, templateId,
TerremarkInstantiateVAppTemplateOptions.Builder.processorCount(minCores).memory(
minMegs).productProperties(properties));
logger.debug("<< instantiated VApp(%s)", vApp.getId());
logger.debug(">> deploying vApp(%s)", vApp.getId());
vApp = blockUntilVAppStatusOrThrowException(vApp, tmClient.deployVApp(vApp.getId()),
"deploy", VAppStatus.OFF);
logger.debug("<< deployed vApp(%s)", vApp.getId());
logger.debug(">> powering vApp(%s)", vApp.getId());
vApp = blockUntilVAppStatusOrThrowException(vApp, tmClient.powerOnVApp(vApp.getId()),
"powerOn", VAppStatus.ON);
logger.debug("<< on vApp(%s)", vApp.getId());
return vApp.getId();
}
/**
*
* @throws ElementNotFoundException
* if no address is configured
*/
public InetAddress getAnyPrivateAddress(String id) {
VApp vApp = tmClient.getVApp(id);
return Iterables.getLast(vApp.getNetworkToAddresses().values());
}
public Set<InetAddress> getPublicAddresses(String id) {
VApp vApp = tmClient.getVApp(id);
Set<InetAddress> ipAddresses = Sets.newHashSet();
for (InternetService service : tmClient.getAllInternetServicesInVDC(vApp.getVDC().getId())) {
for (Node node : tmClient.getNodes(service.getId())) {
if (vApp.getNetworkToAddresses().containsValue(node.getIpAddress())) {
ipAddresses.add(service.getPublicIpAddress().getAddress());
}
}
}
return ipAddresses;
}
public void reboot(String id) {
VApp vApp = tmClient.getVApp(id);
logger.debug(">> rebooting vApp(%s)", vApp.getId());
blockUntilVAppStatusOrThrowException(vApp, tmClient.resetVApp(vApp.getId()), "reset",
VAppStatus.ON);
logger.debug("<< on vApp(%s)", vApp.getId());
}
public InetAddress createPublicAddressMappedToPorts(VApp vApp, int... ports) {
public InetAddress createPublicAddressMappedToPorts(String vAppId, int... ports) {
VApp vApp = client.getVApp(vAppId);
PublicIpAddress ip = null;
InetAddress privateAddress = Iterables.getLast(vApp.getNetworkToAddresses().values());
for (int port : ports) {
@ -165,15 +101,15 @@ public class TerremarkVCloudComputeClient {
if (ip == null) {
logger.debug(">> creating InternetService in vDC %s:%s:%d", vApp.getVDC().getId(),
protocol, port);
is = tmClient.addInternetServiceToVDC(vApp.getVDC().getId(), vApp.getName() + "-"
+ port, protocol, port,
withDescription(String.format("port %d access to serverId: %s name: %s", port,
vApp.getId(), vApp.getName())));
is = client.addInternetServiceToVDC(vApp.getVDC().getId(), vApp.getName() + "-" + port,
protocol, port, withDescription(String.format(
"port %d access to serverId: %s name: %s", port, vApp.getId(), vApp
.getName())));
ip = is.getPublicIpAddress();
} else {
logger.debug(">> adding InternetService %s:%s:%d", ip.getAddress().getHostAddress(),
protocol, port);
is = tmClient.addInternetServiceToExistingIp(ip.getId(), vApp.getName() + "-" + port,
is = client.addInternetServiceToExistingIp(ip.getId(), vApp.getName() + "-" + port,
protocol, port, withDescription(String.format(
"port %d access to serverId: %s name: %s", port, vApp.getId(), vApp
.getName())));
@ -183,49 +119,37 @@ public class TerremarkVCloudComputeClient {
.getPort());
logger.debug(">> adding Node %s:%d -> %s:%d", is.getPublicIpAddress().getAddress()
.getHostAddress(), is.getPort(), privateAddress.getHostAddress(), port);
Node node = tmClient
.addNode(is.getId(), privateAddress, vApp.getName() + "-" + port, port);
Node node = client.addNode(is.getId(), privateAddress, vApp.getName() + "-" + port, port);
logger.debug("<< added Node(%s)", node.getId());
}
return ip.getAddress();
return ip != null ? ip.getAddress() : null;
}
@Override
public void stop(String id) {
VApp vApp = tmClient.getVApp(id);
VApp vApp = client.getVApp(id);
Set<PublicIpAddress> ipAddresses = deleteInternetServicesAndNodesAssociatedWithVApp(vApp);
deletePublicIpAddressesWithNoServicesAttached(ipAddresses);
if (vApp.getStatus() != VAppStatus.OFF) {
logger.debug(">> powering off vApp(%s), current status: %s", vApp.getId(), vApp
.getStatus());
blockUntilVAppStatusOrThrowException(vApp, tmClient.powerOffVApp(vApp.getId()),
"powerOff", VAppStatus.OFF);
logger.debug("<< off vApp(%s)", vApp.getId());
}
logger.debug(">> deleting vApp(%s)", vApp.getId());
tmClient.deleteVApp(id);
logger.debug("<< deleted vApp(%s)", vApp.getId());
super.stop(id);
}
private Set<PublicIpAddress> deleteInternetServicesAndNodesAssociatedWithVApp(VApp vApp) {
Set<PublicIpAddress> ipAddresses = Sets.newHashSet();
SERVICE: for (InternetService service : tmClient.getAllInternetServicesInVDC(vApp.getVDC()
SERVICE: for (InternetService service : client.getAllInternetServicesInVDC(vApp.getVDC()
.getId())) {
for (Node node : tmClient.getNodes(service.getId())) {
for (Node node : client.getNodes(service.getId())) {
if (vApp.getNetworkToAddresses().containsValue(node.getIpAddress())) {
ipAddresses.add(service.getPublicIpAddress());
logger.debug(">> deleting Node(%s) %s:%d -> %s:%d", node.getId(), service
.getPublicIpAddress().getAddress().getHostAddress(), service.getPort(),
node.getIpAddress().getHostAddress(), node.getPort());
tmClient.deleteNode(node.getId());
client.deleteNode(node.getId());
logger.debug("<< deleted Node(%s)", node.getId());
SortedSet<Node> nodes = tmClient.getNodes(service.getId());
SortedSet<Node> nodes = client.getNodes(service.getId());
if (nodes.size() == 0) {
logger.debug(">> deleting InternetService(%s) %s:%d", service.getId(), service
.getPublicIpAddress().getAddress().getHostAddress(), service.getPort());
tmClient.deleteInternetService(service.getId());
client.deleteInternetService(service.getId());
logger.debug("<< deleted InternetService(%s)", service.getId());
continue SERVICE;
}
@ -237,81 +161,35 @@ public class TerremarkVCloudComputeClient {
private void deletePublicIpAddressesWithNoServicesAttached(Set<PublicIpAddress> ipAddresses) {
IPADDRESS: for (PublicIpAddress address : ipAddresses) {
SortedSet<InternetService> services = tmClient.getInternetServicesOnPublicIp(address
.getId());
SortedSet<InternetService> services = client
.getInternetServicesOnPublicIp(address.getId());
if (services.size() == 0) {
logger.debug(">> deleting PublicIpAddress(%s) %s", address.getId(), address
.getAddress().getHostAddress());
tmClient.deletePublicIp(address.getId());
client.deletePublicIp(address.getId());
logger.debug("<< deleted PublicIpAddress(%s)", address.getId());
continue IPADDRESS;
}
}
}
private VApp blockUntilVAppStatusOrThrowException(VApp vApp, Task deployTask, String taskType,
VAppStatus expectedStatus) {
if (!taskTester.apply(deployTask.getId())) {
throw new TaskException(taskType, vApp, deployTask);
}
vApp = tmClient.getVApp(vApp.getId());
if (vApp.getStatus() != expectedStatus) {
throw new VAppException(String.format("vApp %s status %s should be %s after %s", vApp
.getId(), vApp.getStatus(), expectedStatus, taskType), vApp);
}
return vApp;
@Override
public Set<InetAddress> getPrivateAddresses(String id) {
VApp vApp = client.getVApp(id);
return Sets.newHashSet(vApp.getNetworkToAddresses().values());
}
public static class TaskException extends VAppException {
private final Task task;
/** The serialVersionUID */
private static final long serialVersionUID = 251801929573211256L;
public TaskException(String type, VApp vApp, Task task) {
super(String.format("failed to %s vApp %s status %s;task %s status %s", type,
vApp.getId(), vApp.getStatus(), task.getLocation(), task.getStatus()), vApp);
this.task = task;
@Override
public Set<InetAddress> getPublicAddresses(String id) {
VApp vApp = client.getVApp(id);
Set<InetAddress> ipAddresses = Sets.newHashSet();
for (InternetService service : client.getAllInternetServicesInVDC(vApp.getVDC().getId())) {
for (Node node : client.getNodes(service.getId())) {
if (vApp.getNetworkToAddresses().containsValue(node.getIpAddress())) {
ipAddresses.add(service.getPublicIpAddress().getAddress());
}
}
}
public Task getTask() {
return task;
}
}
public static class SocketNotOpenException extends RuntimeException {
private final InetSocketAddress socket;
/** The serialVersionUID */
private static final long serialVersionUID = 251801929573211256L;
public SocketNotOpenException(InetSocketAddress socket) {
super("socket not open: " + socket);
this.socket = socket;
}
public InetSocketAddress getSocket() {
return socket;
}
}
public static class VAppException extends RuntimeException {
private final VApp vApp;
/** The serialVersionUID */
private static final long serialVersionUID = 251801929573211256L;
public VAppException(String message, VApp vApp) {
super(message);
this.vApp = vApp;
}
public VApp getvApp() {
return vApp;
}
return ipAddresses;
}
}

View File

@ -1,221 +0,0 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.vcloud.terremark.compute;
import static org.jclouds.vcloud.terremark.options.AddInternetServiceOptions.Builder.withDescription;
import java.net.InetAddress;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.concurrent.ExecutorService;
import javax.inject.Inject;
import javax.inject.Named;
import javax.inject.Provider;
import javax.inject.Singleton;
import org.jclouds.Constants;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeState;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.domain.internal.NodeMetadataImpl;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.domain.Location;
import org.jclouds.vcloud.compute.VCloudComputeService;
import org.jclouds.vcloud.domain.VApp;
import org.jclouds.vcloud.domain.VAppStatus;
import org.jclouds.vcloud.terremark.TerremarkVCloudClient;
import org.jclouds.vcloud.terremark.domain.InternetService;
import org.jclouds.vcloud.terremark.domain.Node;
import org.jclouds.vcloud.terremark.domain.Protocol;
import org.jclouds.vcloud.terremark.domain.PublicIpAddress;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
/**
* @author Adrian Cole
*/
@Singleton
public class TerremarkVCloudComputeService extends VCloudComputeService {
private final TerremarkVCloudClient client;
@Inject
TerremarkVCloudComputeService(Provider<TemplateBuilder> templateBuilderProvider,
Provider<Map<String, ? extends Image>> images,
Provider<Map<String, ? extends Size>> sizes,
Provider<Map<String, ? extends Location>> locations, ComputeUtils utils,
TerremarkVCloudClient client, Predicate<String> successTester,
@Named("NOT_FOUND") Predicate<VApp> notFoundTester,
Map<VAppStatus, NodeState> vAppStatusToNodeState,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
super(templateBuilderProvider, images, sizes, locations, utils, client, successTester,
notFoundTester, vAppStatusToNodeState, executor);
this.client = client;
}
@Override
protected Map<String, String> parseResponse(VApp vAppResponse) {
return ImmutableMap.<String, String> of("id", vAppResponse.getId(), "username", "vcloud",
"password", "p4ssw0rd");
}
@Override
public Map<String, String> start(String vDCId, String name, String templateId, int minCores,
int minMegs, Long diskSize, Map<String, String> properties, int... portsToOpen) {
Map<String, String> response = super.start(vDCId, name, templateId, minCores, minMegs, null,
properties, portsToOpen);// trmk does not support resizing the primary disk
if (portsToOpen.length > 0)
createPublicAddressMappedToPorts(response.get("id"), portsToOpen);
return response;
}
@Override
protected NodeMetadata getNodeMetadataByIdInVDC(String vDCId, String id) {
VApp vApp = client.getVApp(id);
String tag = vApp.getName().replaceAll("-[0-9]+", "");
return new NodeMetadataImpl(vApp.getId(), vApp.getName(), vDCId, vApp.getLocation(),
ImmutableMap.<String, String> of(), tag,
vAppStatusToNodeState.get(vApp.getStatus()), getPublicAddresses(id), vApp
.getNetworkToAddresses().values(), ImmutableMap.<String, String> of(), null);
}
public InetAddress createPublicAddressMappedToPorts(String vAppId, int... ports) {
VApp vApp = client.getVApp(vAppId);
PublicIpAddress ip = null;
InetAddress privateAddress = Iterables.getLast(vApp.getNetworkToAddresses().values());
for (int port : ports) {
InternetService is = null;
Protocol protocol;
switch (port) {
case 22:
protocol = Protocol.TCP;
break;
case 80:
case 8080:
protocol = Protocol.HTTP;
break;
case 443:
protocol = Protocol.HTTPS;
break;
default:
protocol = Protocol.HTTP;
break;
}
if (ip == null) {
logger.debug(">> creating InternetService in vDC %s:%s:%d", vApp.getVDC().getId(),
protocol, port);
is = client.addInternetServiceToVDC(vApp.getVDC().getId(), vApp.getName() + "-" + port,
protocol, port, withDescription(String.format(
"port %d access to serverId: %s name: %s", port, vApp.getId(), vApp
.getName())));
ip = is.getPublicIpAddress();
} else {
logger.debug(">> adding InternetService %s:%s:%d", ip.getAddress().getHostAddress(),
protocol, port);
is = client.addInternetServiceToExistingIp(ip.getId(), vApp.getName() + "-" + port,
protocol, port, withDescription(String.format(
"port %d access to serverId: %s name: %s", port, vApp.getId(), vApp
.getName())));
}
logger.debug("<< created InternetService(%s) %s:%s:%d", is.getId(), is
.getPublicIpAddress().getAddress().getHostAddress(), is.getProtocol(), is
.getPort());
logger.debug(">> adding Node %s:%d -> %s:%d", is.getPublicIpAddress().getAddress()
.getHostAddress(), is.getPort(), privateAddress.getHostAddress(), port);
Node node = client.addNode(is.getId(), privateAddress, vApp.getName() + "-" + port, port);
logger.debug("<< added Node(%s)", node.getId());
}
return ip != null ? ip.getAddress() : null;
}
@Override
public void stop(String id) {
VApp vApp = client.getVApp(id);
Set<PublicIpAddress> ipAddresses = deleteInternetServicesAndNodesAssociatedWithVApp(vApp);
deletePublicIpAddressesWithNoServicesAttached(ipAddresses);
super.stop(id);
}
private Set<PublicIpAddress> deleteInternetServicesAndNodesAssociatedWithVApp(VApp vApp) {
Set<PublicIpAddress> ipAddresses = Sets.newHashSet();
SERVICE: for (InternetService service : client.getAllInternetServicesInVDC(vApp.getVDC()
.getId())) {
for (Node node : client.getNodes(service.getId())) {
if (vApp.getNetworkToAddresses().containsValue(node.getIpAddress())) {
ipAddresses.add(service.getPublicIpAddress());
logger.debug(">> deleting Node(%s) %s:%d -> %s:%d", node.getId(), service
.getPublicIpAddress().getAddress().getHostAddress(), service.getPort(),
node.getIpAddress().getHostAddress(), node.getPort());
client.deleteNode(node.getId());
logger.debug("<< deleted Node(%s)", node.getId());
SortedSet<Node> nodes = client.getNodes(service.getId());
if (nodes.size() == 0) {
logger.debug(">> deleting InternetService(%s) %s:%d", service.getId(), service
.getPublicIpAddress().getAddress().getHostAddress(), service.getPort());
client.deleteInternetService(service.getId());
logger.debug("<< deleted InternetService(%s)", service.getId());
continue SERVICE;
}
}
}
}
return ipAddresses;
}
private void deletePublicIpAddressesWithNoServicesAttached(Set<PublicIpAddress> ipAddresses) {
IPADDRESS: for (PublicIpAddress address : ipAddresses) {
SortedSet<InternetService> services = client
.getInternetServicesOnPublicIp(address.getId());
if (services.size() == 0) {
logger.debug(">> deleting PublicIpAddress(%s) %s", address.getId(), address
.getAddress().getHostAddress());
client.deletePublicIp(address.getId());
logger.debug("<< deleted PublicIpAddress(%s)", address.getId());
continue IPADDRESS;
}
}
}
@Override
public Set<InetAddress> getPrivateAddresses(String id) {
VApp vApp = client.getVApp(id);
return Sets.newHashSet(vApp.getNetworkToAddresses().values());
}
@Override
public Set<InetAddress> getPublicAddresses(String id) {
VApp vApp = client.getVApp(id);
Set<InetAddress> ipAddresses = Sets.newHashSet();
for (InternetService service : client.getAllInternetServicesInVDC(vApp.getVDC().getId())) {
for (Node node : client.getNodes(service.getId())) {
if (vApp.getNetworkToAddresses().containsValue(node.getIpAddress())) {
ipAddresses.add(service.getPublicIpAddress().getAddress());
}
}
}
return ipAddresses;
}
}

View File

@ -27,16 +27,16 @@ import java.util.concurrent.TimeoutException;
import javax.inject.Named;
import org.jclouds.Constants;
import org.jclouds.compute.ComputeService;
import org.jclouds.compute.domain.Architecture;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.internal.SizeImpl;
import org.jclouds.vcloud.VCloudClient;
import org.jclouds.vcloud.compute.VCloudComputeClient;
import org.jclouds.vcloud.compute.config.VCloudComputeServiceContextModule;
import org.jclouds.vcloud.terremark.TerremarkVCloudClient;
import org.jclouds.vcloud.terremark.compute.TerremarkVCloudComputeService;
import org.jclouds.vcloud.terremark.compute.TerremarkVCloudComputeClient;
import org.jclouds.vcloud.terremark.domain.ComputeOptions;
import com.google.common.base.Function;
@ -45,11 +45,10 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.google.inject.Injector;
/**
* Configures the {@link TerremarkVCloudComputeServiceContext}; requires
* {@link TerremarkVCloudComputeService} bound.
* {@link TerremarkVCloudComputeClientImpl} bound.
*
* @author Adrian Cole
*/
@ -58,11 +57,7 @@ public class TerremarkVCloudComputeServiceContextModule extends VCloudComputeSer
@Override
protected void configure() {
super.configure();
}
@Override
protected ComputeService provideComputeService(Injector injector) {
return injector.getInstance(TerremarkVCloudComputeService.class);
bind(VCloudComputeClient.class).to(TerremarkVCloudComputeClient.class);
}
private static final ComputeOptionsToSize sizeConverter = new ComputeOptionsToSize();

View File

@ -1,167 +0,0 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.vcloud.terremark.compute;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.testng.Assert.assertEquals;
import java.io.IOException;
import java.net.InetAddress;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException;
import org.jclouds.compute.domain.OsFamily;
import org.jclouds.logging.log4j.config.Log4JLoggingModule;
import org.jclouds.ssh.jsch.config.JschSshClientModule;
import org.jclouds.vcloud.domain.ResourceType;
import org.jclouds.vcloud.domain.VApp;
import org.jclouds.vcloud.domain.VAppStatus;
import org.jclouds.vcloud.terremark.TerremarkVCloudClient;
import org.jclouds.vcloud.terremark.TerremarkVCloudContextBuilder;
import org.jclouds.vcloud.terremark.TerremarkVCloudPropertiesBuilder;
import org.testng.annotations.AfterTest;
import org.testng.annotations.BeforeGroups;
import org.testng.annotations.Test;
import com.google.common.base.CaseFormat;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.TypeLiteral;
import com.google.inject.internal.ImmutableMap;
/**
* Tests behavior of {@code TerremarkVCloudClient}
*
* @author Adrian Cole
*/
@Test(groups = "live", sequential = true, testName = "vcloud.TerremarkVCloudClientLiveTest")
public class TerremarkVCloudComputeClientLiveTest {
TerremarkVCloudComputeClient client;
TerremarkVCloudClient tmClient;
private String id;
private InetAddress privateAddress;
public static final String PREFIX = System.getProperty("user.name") + "-terremark";
private static class Expectation {
final long hardDisk;
final String os;
public Expectation(long hardDisk, String os) {
this.hardDisk = hardDisk;
this.os = os;
}
}
private Map<OsFamily, Expectation> expectationMap = ImmutableMap
.<OsFamily, Expectation> builder().put(OsFamily.CENTOS,
new Expectation(10485760, "Red Hat Enterprise Linux 5 (64-bit)")).put(
OsFamily.RHEL,
new Expectation(10485760, "Red Hat Enterprise Linux 5 (64-bit)")).put(
OsFamily.UBUNTU, new Expectation(4194304, "Ubuntu Linux (64-bit)"))
.build();
// .put(OperatingSystem.UBUNTU, new Expectation(4194304, "Ubuntu Linux (32-bit)"))
private Predicate<InetAddress> addressTester;
@Test
public void testPowerOn() throws InterruptedException, ExecutionException, TimeoutException,
IOException {
OsFamily toTest = OsFamily.CENTOS;
String serverName = getCompatibleServerName(toTest);
int processorCount = 1;
int memory = 512;
Map<String, String> properties = ImmutableMap.of();
id = client.start(serverName, toTest, processorCount, memory, properties);
Expectation expectation = expectationMap.get(toTest);
VApp vApp = tmClient.getVApp(id);
verifyConfigurationOfVApp(vApp, serverName, expectation.os, processorCount, memory,
expectation.hardDisk);
assertEquals(vApp.getStatus(), VAppStatus.ON);
}
private String getCompatibleServerName(OsFamily toTest) {
String serverName = CaseFormat.UPPER_UNDERSCORE
.to(CaseFormat.LOWER_HYPHEN, toTest.toString()).substring(0,
toTest.toString().length() <= 15 ? toTest.toString().length() : 14);
return serverName;
}
@Test(dependsOnMethods = "testPowerOn")
public void testGetAnyPrivateAddress() {
privateAddress = client.getAnyPrivateAddress(id);
assert !addressTester.apply(privateAddress);
}
@Test(dependsOnMethods = "testGetAnyPrivateAddress")
public void testSshLoadBalanceIp() {
InetAddress publicIp = client.createPublicAddressMappedToPorts(tmClient.getVApp(id), 22, 80,
443, 8080); // / error 500
assert addressTester.apply(publicIp);
// client.exec(publicIp, "uname -a");
}
private void verifyConfigurationOfVApp(VApp vApp, String serverName, String expectedOs,
int processorCount, int memory, long hardDisk) {
assertEquals(vApp.getName(), serverName);
assertEquals(vApp.getOperatingSystemDescription(), expectedOs);
assertEquals(
Iterables.getOnlyElement(
vApp.getResourceAllocationByType().get(ResourceType.PROCESSOR))
.getVirtualQuantity(), processorCount);
assertEquals(Iterables.getOnlyElement(
vApp.getResourceAllocationByType().get(ResourceType.SCSI_CONTROLLER))
.getVirtualQuantity(), 1);
assertEquals(Iterables.getOnlyElement(
vApp.getResourceAllocationByType().get(ResourceType.MEMORY)).getVirtualQuantity(),
memory);
assertEquals(Iterables.getOnlyElement(
vApp.getResourceAllocationByType().get(ResourceType.DISK_DRIVE))
.getVirtualQuantity(), hardDisk);
assertEquals(vApp.getSize().longValue(), Iterables.getOnlyElement(
vApp.getResourceAllocationByType().get(ResourceType.DISK_DRIVE))
.getVirtualQuantity());
}
@AfterTest
void cleanup() throws InterruptedException, ExecutionException, TimeoutException {
if (id != null)
client.stop(id);
}
@BeforeGroups(groups = { "live" })
public void setupClient() {
String account = checkNotNull(System.getProperty("jclouds.test.user"), "jclouds.test.user");
String key = checkNotNull(System.getProperty("jclouds.test.key"), "jclouds.test.key");
Injector injector = new TerremarkVCloudContextBuilder(new TerremarkVCloudPropertiesBuilder(
account, key).build()).withModules(new Log4JLoggingModule(),
new JschSshClientModule()).buildInjector();
client = injector.getInstance(TerremarkVCloudComputeClient.class);
tmClient = injector.getInstance(TerremarkVCloudClient.class);
addressTester = injector.getInstance(Key.get(new TypeLiteral<Predicate<InetAddress>>() {
}));
}
}

View File

@ -27,7 +27,6 @@ import org.jclouds.compute.domain.Template;
import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.rest.RestContext;
import org.jclouds.ssh.jsch.config.JschSshClientModule;
import org.jclouds.vcloud.compute.VCloudComputeClient;
import org.jclouds.vcloud.terremark.TerremarkVCloudAsyncClient;
import org.jclouds.vcloud.terremark.TerremarkVCloudClient;
import org.testng.annotations.BeforeClass;
@ -61,11 +60,5 @@ public class TerremarkVCloudComputeServiceLiveTest extends BaseComputeServiceLiv
@SuppressWarnings("unused")
RestContext<TerremarkVCloudAsyncClient, TerremarkVCloudClient> tmContext = new ComputeServiceContextFactory()
.createContext(service, user, password).getProviderSpecificContext();
TerremarkVCloudComputeService computeService = TerremarkVCloudComputeService.class
.cast(client);
@SuppressWarnings("unused")
VCloudComputeClient computeClient = VCloudComputeClient.class.cast(computeService);
}
}