mirror of https://github.com/apache/jclouds.git
Issue 156: refactored compute api classes and added reboot support
This commit is contained in:
parent
52e40c4651
commit
df86359e65
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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()))));
|
||||
}
|
||||
|
||||
}
|
|
@ -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)));
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -58,6 +58,11 @@ public class EC2ContextModuleTest {
|
|||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
void testM1SMALLIsSmallest() {
|
||||
// TODO
|
||||
}
|
||||
|
||||
@Test
|
||||
void testContextImpl() {
|
||||
RestContext<EC2AsyncClient, EC2Client> handler = createInjector().getInstance(
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
||||
}
|
|
@ -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();
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -153,7 +153,7 @@ public abstract class BaseComputeServiceLiveTest {
|
|||
// credentials aren't always the same
|
||||
// assertEquals(node1.getCredentials(), node2.getCredentials());
|
||||
assert !node1.getId().equals(node2.getId());
|
||||
|
||||
|
||||
// run one more
|
||||
nodes.addAll(client.runNodesWithTag(tag, 1, template).values());
|
||||
assertEquals(nodes.size(), 3);
|
||||
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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>>() {
|
||||
}));
|
||||
|
|
|
@ -1,317 +1,195 @@
|
|||
/**
|
||||
*
|
||||
* 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.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 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.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 org.jclouds.vcloud.terremark.options.TerremarkInstantiateVAppTemplateOptions;
|
||||
|
||||
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
|
||||
*/
|
||||
public class TerremarkVCloudComputeClient {
|
||||
@Resource
|
||||
@Named(ComputeServiceConstants.COMPUTE_LOGGER)
|
||||
Logger logger = Logger.NULL;
|
||||
|
||||
private final Predicate<String> taskTester;
|
||||
private final TerremarkVCloudClient tmClient;
|
||||
|
||||
@Inject
|
||||
public TerremarkVCloudComputeClient(TerremarkVCloudClient tmClient,
|
||||
Predicate<String> successTester) {
|
||||
this.tmClient = tmClient;
|
||||
this.taskTester = successTester;
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
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) {
|
||||
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 = 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())));
|
||||
ip = is.getPublicIpAddress();
|
||||
} else {
|
||||
logger.debug(">> adding InternetService %s:%s:%d", ip.getAddress().getHostAddress(),
|
||||
protocol, port);
|
||||
is = tmClient.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 = tmClient
|
||||
.addNode(is.getId(), privateAddress, vApp.getName() + "-" + port, port);
|
||||
logger.debug("<< added Node(%s)", node.getId());
|
||||
}
|
||||
return ip.getAddress();
|
||||
}
|
||||
|
||||
public void stop(String id) {
|
||||
VApp vApp = tmClient.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());
|
||||
}
|
||||
|
||||
private Set<PublicIpAddress> deleteInternetServicesAndNodesAssociatedWithVApp(VApp vApp) {
|
||||
Set<PublicIpAddress> ipAddresses = Sets.newHashSet();
|
||||
SERVICE: 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());
|
||||
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());
|
||||
logger.debug("<< deleted Node(%s)", node.getId());
|
||||
SortedSet<Node> nodes = tmClient.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());
|
||||
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 = tmClient.getInternetServicesOnPublicIp(address
|
||||
.getId());
|
||||
if (services.size() == 0) {
|
||||
logger.debug(">> deleting PublicIpAddress(%s) %s", address.getId(), address
|
||||
.getAddress().getHostAddress());
|
||||
tmClient.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;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
/**
|
||||
*
|
||||
* 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 javax.inject.Inject;
|
||||
import javax.inject.Named;
|
||||
import javax.inject.Singleton;
|
||||
|
||||
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;
|
||||
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 TerremarkVCloudComputeClient extends BaseVCloudComputeClient {
|
||||
|
||||
private final TerremarkVCloudClient client;
|
||||
|
||||
@Inject
|
||||
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;
|
||||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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>>() {
|
||||
}));
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue